From 937f7d1341db0a19bce8dfc0e61f6f218e20090c Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Tue, 27 Aug 2024 14:44:13 +0800 Subject: [PATCH 01/19] [feature]init --- .../seatunnel/hbase/catalog/HbaseCatalog.java | 214 +++++++++++++ .../hbase/catalog/HbaseCatalogFactory.java | 47 +++ .../seatunnel/hbase/client/HbaseClient.java | 294 ++++++++++++++++++ .../seatunnel/hbase/config/HbaseConfig.java | 21 ++ .../hbase/config/HbaseParameters.java | 23 +- .../exception/HbaseConnectorErrorCode.java | 13 +- .../seatunnel/hbase/sink/HbaseSink.java | 133 +++++--- .../hbase/sink/HbaseSinkFactory.java | 9 + .../seatunnel/hbase/sink/HbaseSinkWriter.java | 59 ++-- .../seatunnel/hbase/source/HbaseSource.java | 2 - .../hbase/source/HbaseSourceReader.java | 34 +- .../source/HbaseSourceSplitEnumerator.java | 29 +- .../state/HbaseAggregatedCommitInfo.java | 22 ++ .../hbase/state/HbaseCommitInfo.java | 22 ++ .../seatunnel/hbase/state/HbaseSinkState.java | 22 ++ .../hbase/utils/HbaseConnectionUtil.java | 48 --- 16 files changed, 813 insertions(+), 179 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseAggregatedCommitInfo.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseCommitInfo.java create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseSinkState.java delete mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java new file mode 100644 index 00000000000..174a3ce51a7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hbase.catalog; + +import org.apache.seatunnel.api.configuration.util.ConfigUtil; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.InfoPreviewResult; +import org.apache.seatunnel.api.table.catalog.PreviewResult; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; +import org.apache.seatunnel.connectors.seatunnel.hbase.client.HbaseClient; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * Hbase catalog implementation. + * + */ +@Slf4j +public class HbaseCatalog implements Catalog { + + private static final Logger LOGGER = LoggerFactory.getLogger(HbaseCatalog.class); + + private final String catalogName; + private final String defaultDatabase; + private final HbaseParameters hbaseParameters; + + private HbaseClient hbaseClient; + + // todo: do we need default database? + public HbaseCatalog( + String catalogName, String defaultDatabase, HbaseParameters hbaseParameters) { + this.catalogName = checkNotNull(catalogName, "catalogName cannot be null"); + this.defaultDatabase = defaultDatabase; + this.hbaseParameters = checkNotNull(hbaseParameters, "Hbase Config cannot be null"); + } + + @Override + public void open() throws CatalogException { + try { + hbaseClient = HbaseClient.createInstance(hbaseParameters); + } catch (Exception e) { + throw new CatalogException(String.format("Failed to open catalog %s", catalogName), e); + } + } + + @Override + public void close() throws CatalogException { + hbaseClient.close(); + } + + @Override + public String name() { + return catalogName; + } + + @Override + public String getDefaultDatabase() throws CatalogException { + return defaultDatabase; + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + return hbaseClient.databaseExists(databaseName); + } + + @Override + public List listDatabases() throws CatalogException { + return hbaseClient.listDatabases(); + } + + @Override + public List listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(catalogName, databaseName); + } + return hbaseClient.listTables(databaseName); + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + checkNotNull(tablePath); + return hbaseClient.tableExists(tablePath.getTableName()); + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + checkNotNull(tablePath, "tablePath cannot be null"); + // Hbase cannot retrieve the columns of the table + return CatalogTable.of( + TableIdentifier.of( + catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), + TableSchema.builder().build(), + buildTableOptions(tablePath), + Collections.emptyList(), + ""); + } + + @Override + public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + hbaseClient.createTable( + tablePath.getDatabaseName(), + tablePath.getTableName(), + hbaseParameters.getFamilyNames().values().stream() + .filter(value -> !"all_columns".equals(value)) + .collect(Collectors.toList()), + ignoreIfExists); + } + + @Override + public void dropTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath); + if (!tableExists(tablePath) && !ignoreIfNotExists) { + throw new TableNotExistException(catalogName, tablePath); + } + hbaseClient.dropTable(tablePath.getDatabaseName(), tablePath.getTableName()); + } + + @Override + public void createDatabase(TablePath tablePath, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + if (databaseExists(tablePath.getDatabaseName()) && !ignoreIfExists) { + throw new DatabaseAlreadyExistException(catalogName, tablePath.getDatabaseName()); + } + hbaseClient.createNamespace(tablePath.getDatabaseName()); + } + + @Override + public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + if (!databaseExists(tablePath.getDatabaseName()) && !ignoreIfNotExists) { + throw new DatabaseNotExistException(catalogName, tablePath.getDatabaseName()); + } + hbaseClient.deleteNamespace(tablePath.getDatabaseName()); + } + + @Override + public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) { + if (!tableExists(tablePath) && !ignoreIfNotExists) { + throw new TableNotExistException(catalogName, tablePath); + } + hbaseClient.truncateTable(tablePath.getDatabaseName(), tablePath.getTableName()); + } + + @Override + public boolean isExistsData(TablePath tablePath) { + return hbaseClient.isExistsData(tablePath.getDatabaseName(), tablePath.getTableName()); + } + + private Map buildTableOptions(TablePath tablePath) { + Map options = new HashMap<>(); + options.put("connector", "hbase"); + options.put("config", ConfigUtil.convertToJsonString(tablePath)); + return options; + } + + @Override + public PreviewResult previewAction( + ActionType actionType, TablePath tablePath, Optional catalogTable) { + if (actionType == ActionType.CREATE_TABLE) { + return new InfoPreviewResult("create index " + tablePath.getTableName()); + } else if (actionType == ActionType.DROP_TABLE) { + return new InfoPreviewResult("delete index " + tablePath.getTableName()); + } else if (actionType == ActionType.TRUNCATE_TABLE) { + return new InfoPreviewResult("delete and create index " + tablePath.getTableName()); + } else if (actionType == ActionType.CREATE_DATABASE) { + return new InfoPreviewResult("create index " + tablePath.getTableName()); + } else if (actionType == ActionType.DROP_DATABASE) { + return new InfoPreviewResult("delete index " + tablePath.getTableName()); + } else { + throw new UnsupportedOperationException("Unsupported action type: " + actionType); + } + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java new file mode 100644 index 00000000000..73ec3206777 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java @@ -0,0 +1,47 @@ +/* + * 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.hbase.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class HbaseCatalogFactory implements CatalogFactory { + + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + HbaseParameters hbaseParameters = HbaseParameters.buildWithSourceConfig(options.toConfig()); + return new HbaseCatalog(catalogName, hbaseParameters.getNamespace(), hbaseParameters); + } + + @Override + public String factoryIdentifier() { + return "Hbase"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().build(); + } +} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java new file mode 100644 index 00000000000..031828bb70b --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java @@ -0,0 +1,294 @@ +/* + * 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.hbase.client; + +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; +import org.apache.seatunnel.connectors.seatunnel.hbase.source.HbaseSourceSplit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.BufferedMutatorParams; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionLocator; +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.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.util.Bytes; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorErrorCode.CONNECTION_FAILED_FOR_ADMIN; + +@Slf4j +public class HbaseClient { + + private static final int CONNECTION_REQUEST_TIMEOUT = 10 * 1000; + + private static final int SOCKET_TIMEOUT = 5 * 60 * 1000; + + private final Connection connection; + private final Admin admin; + private final BufferedMutator hbaseMutator; + public static Configuration hbaseConfiguration; + + private HbaseClient(Connection connection, HbaseParameters hbaseParameters) { + System.out.println("V1.000"); + this.connection = connection; + try { + this.admin = connection.getAdmin(); + + BufferedMutatorParams bufferedMutatorParams = + new BufferedMutatorParams( + TableName.valueOf( + hbaseParameters.getNamespace(), + hbaseParameters.getTable())) + .pool(HTable.getDefaultExecutor(hbaseConfiguration)) + .writeBufferSize(hbaseParameters.getWriteBufferSize()); + hbaseMutator = connection.getBufferedMutator(bufferedMutatorParams); + } catch (IOException e) { + throw new HbaseConnectorException( + CONNECTION_FAILED_FOR_ADMIN, CONNECTION_FAILED_FOR_ADMIN.getDescription(), e); + } + } + + public static HbaseClient createInstance(HbaseParameters hbaseParameters) { + return new HbaseClient(getHbaseConnection(hbaseParameters), hbaseParameters); + } + + private static Connection getHbaseConnection(HbaseParameters hbaseParameters) { + hbaseConfiguration = HBaseConfiguration.create(); + hbaseConfiguration.set("hbase.zookeeper.quorum", hbaseParameters.getZookeeperQuorum()); + if (hbaseParameters.getHbaseExtraConfig() != null) { + hbaseParameters.getHbaseExtraConfig().forEach(hbaseConfiguration::set); + } + try { + Connection connection = ConnectionFactory.createConnection(hbaseConfiguration); + return connection; + } catch (IOException e) { + String errorMsg = "Build Hbase connection failed."; + throw new HbaseConnectorException( + HbaseConnectorErrorCode.CONNECTION_FAILED, errorMsg, e); + } + } + + public boolean databaseExists(String databaseName) { + try { + return Arrays.stream(admin.listNamespaceDescriptors()) + .anyMatch(descriptor -> descriptor.getName().equals(databaseName)); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.DATABASE_QUERY_EXCEPTION, + HbaseConnectorErrorCode.DATABASE_QUERY_EXCEPTION.getErrorMessage(), + e); + } + } + + public List listDatabases() { + try { + return Arrays.stream(admin.listNamespaceDescriptors()) + .map(NamespaceDescriptor::getName) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.DATABASE_QUERY_EXCEPTION, + HbaseConnectorErrorCode.DATABASE_QUERY_EXCEPTION.getErrorMessage(), + e); + } + } + + public List listTables(String databaseName) { + try { + return Arrays.stream(admin.listTableNamesByNamespace(databaseName)) + .map(tableName -> tableName.getNameAsString()) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.DATABASE_QUERY_EXCEPTION, + HbaseConnectorErrorCode.DATABASE_QUERY_EXCEPTION.getErrorMessage(), + e); + } + } + + public boolean tableExists(String tableName) { + try { + return admin.tableExists(TableName.valueOf(tableName)); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.TABLE_QUERY_EXCEPTION, + HbaseConnectorErrorCode.TABLE_QUERY_EXCEPTION.getErrorMessage(), + e); + } + } + + public void createTable( + String databaseName, + String tableName, + List columnFamilies, + boolean ignoreIfExists) { + try { + if (!databaseExists(databaseName)) { + admin.createNamespace(NamespaceDescriptor.create(databaseName).build()); + } + TableName table = TableName.valueOf(databaseName, tableName); + if (tableExists(table.getNameAsString())) { + log.info("Table {} already exists.", table.getNameAsString()); + if (!ignoreIfExists) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.TABLE_EXISTS_EXCEPTION, + HbaseConnectorErrorCode.TABLE_EXISTS_EXCEPTION.getErrorMessage()); + } + return; + } + TableDescriptorBuilder hbaseTableDescriptor = TableDescriptorBuilder.newBuilder(table); + columnFamilies.forEach( + family -> + hbaseTableDescriptor.setColumnFamily( + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family)) + .build())); + admin.createTable(hbaseTableDescriptor.build()); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.TABLE_CREATE_EXCEPTION, + HbaseConnectorErrorCode.TABLE_CREATE_EXCEPTION.getErrorMessage(), + e); + } + } + + public void dropTable(String databaseName, String tableName) { + try { + TableName table = TableName.valueOf(databaseName, tableName); + admin.disableTable(table); + admin.deleteTable(table); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.TABLE_DELETE_EXCEPTION, + HbaseConnectorErrorCode.TABLE_DELETE_EXCEPTION.getErrorMessage(), + e); + } + } + + public void createNamespace(String namespace) { + try { + admin.createNamespace(NamespaceDescriptor.create(namespace).build()); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.NAMESPACE_CREATE_EXCEPTION, + HbaseConnectorErrorCode.NAMESPACE_CREATE_EXCEPTION.getErrorMessage(), + e); + } + } + + public void deleteNamespace(String namespace) { + try { + admin.deleteNamespace(namespace); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.NAMESPACE_DELETE_EXCEPTION, + HbaseConnectorErrorCode.NAMESPACE_DELETE_EXCEPTION.getErrorMessage(), + e); + } + } + + public void truncateTable(String databaseName, String tableName) { + try { + TableName table = TableName.valueOf(databaseName, tableName); + admin.truncateTable(table, true); + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.TABLE_TRUNCATE_EXCEPTION, + HbaseConnectorErrorCode.TABLE_TRUNCATE_EXCEPTION.getErrorMessage(), + e); + } + } + + public boolean isExistsData(String databaseName, String tableName) { + try { + Table table = + connection.getTable( + org.apache.hadoop.hbase.TableName.valueOf(databaseName, tableName)); + Scan scan = new Scan(); + scan.setCaching(1); + scan.setLimit(1); + try (ResultScanner scanner = table.getScanner(scan)) { + Result result = scanner.next(); + return result != null && result.isEmpty(); + } + } catch (IOException e) { + throw new HbaseConnectorException( + HbaseConnectorErrorCode.TABLE_QUERY_EXCEPTION, + HbaseConnectorErrorCode.TABLE_QUERY_EXCEPTION.getErrorMessage(), + e); + } + } + + public void close() { + try { + if (admin != null) { + admin.close(); + } + if (connection != null) { + connection.close(); + } + } catch (IOException e) { + log.error("Close Hbase connection failed.", e); + } + } + + public void mutate(Put put) throws IOException { + hbaseMutator.mutate(put); + } + + public ResultScanner scan( + HbaseSourceSplit split, HbaseParameters hbaseParameters, List columnNames) + throws IOException { + 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 : columnNames) { + String[] columnNameSplit = columnName.split(":"); + scan.addColumn(Bytes.toBytes(columnNameSplit[0]), Bytes.toBytes(columnNameSplit[1])); + } + return this.connection + .getTable(TableName.valueOf(hbaseParameters.getTable())) + .getScanner(scan); + } + + public RegionLocator getRegionLocator(String tableName) throws IOException { + return this.connection.getRegionLocator(TableName.valueOf(tableName)); + } +} 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 44a5640ffed..2921e1f91c8 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 @@ -19,10 +19,17 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.SchemaSaveMode; +import java.util.Arrays; import java.util.List; import java.util.Map; +import static org.apache.seatunnel.api.sink.DataSaveMode.APPEND_DATA; +import static org.apache.seatunnel.api.sink.DataSaveMode.DROP_DATA; +import static org.apache.seatunnel.api.sink.DataSaveMode.ERROR_WHEN_DATA_EXISTS; + public class HbaseConfig { private static final Integer DEFAULT_BUFFER_SIZE = 8 * 1024 * 1024; @@ -119,6 +126,20 @@ public class HbaseConfig { .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 static final Option SCHEMA_SAVE_MODE = + Options.key("schema_save_mode") + .enumType(SchemaSaveMode.class) + .defaultValue(SchemaSaveMode.CREATE_SCHEMA_WHEN_NOT_EXIST) + .withDescription("schema_save_mode"); + + public static final Option DATA_SAVE_MODE = + Options.key("data_save_mode") + .singleChoice( + DataSaveMode.class, + Arrays.asList(DROP_DATA, APPEND_DATA, ERROR_WHEN_DATA_EXISTS)) + .defaultValue(APPEND_DATA) + .withDescription("data_save_mode"); + 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 c25f04b3753..5e237319624 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 @@ -50,6 +50,8 @@ public class HbaseParameters implements Serializable { private String zookeeperQuorum; + private String namespace; + private String table; private List rowkeyColumns; @@ -85,7 +87,16 @@ public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { // required parameters builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); - builder.table(pluginConfig.getString(TABLE.key())); + String table = pluginConfig.getString(TABLE.key()); + int colonIndex = table.indexOf(':'); + if (colonIndex != -1) { + String namespace = table.substring(0, colonIndex); + builder.namespace(namespace); + builder.table(table.substring(colonIndex + 1)); + } else { + builder.table(table); + } + builder.rowkeyColumns(pluginConfig.getStringList(ROWKEY_COLUMNS.key())); builder.familyNames( TypesafeConfigUtils.configToMap(pluginConfig.getConfig(FAMILY_NAME.key()))); @@ -126,7 +137,15 @@ public static HbaseParameters buildWithSourceConfig(Config pluginConfig) { // required parameters builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); - builder.table(pluginConfig.getString(TABLE.key())); + String table = pluginConfig.getString(TABLE.key()); + int colonIndex = table.indexOf(':'); + if (colonIndex != -1) { + String namespace = table.substring(0, colonIndex); + builder.namespace(namespace); + builder.table(table.substring(colonIndex + 1)); + } else { + builder.table(table); + } if (pluginConfig.hasPath(HBASE_EXTRA_CONFIG.key())) { Config extraConfig = pluginConfig.getConfig(HBASE_EXTRA_CONFIG.key()); diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java index 5717c933b0d..7f6a60f9558 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/exception/HbaseConnectorErrorCode.java @@ -21,8 +21,17 @@ import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; public enum HbaseConnectorErrorCode implements SeaTunnelErrorCode { - CONNECTION_FAILED("Hbase-01", "Build Hbase connection failed"); - + CONNECTION_FAILED("Hbase-01", "Build Hbase connection failed"), + CONNECTION_FAILED_FOR_ADMIN("Hbase-02", "Build Hbase Admin failed"), + DATABASE_QUERY_EXCEPTION("Hbase-03", "Hbase namespace query failed"), + TABLE_QUERY_EXCEPTION("Hbase-04", "Hbase table query failed"), + TABLE_CREATE_EXCEPTION("Hbase-05", "Hbase table create failed"), + TABLE_DELETE_EXCEPTION("Hbase-06", "Hbase table delete failed"), + TABLE_EXISTS_EXCEPTION("Hbase-07", "Hbase table exists failed"), + NAMESPACE_CREATE_EXCEPTION("Hbase-08", "Hbase namespace create failed"), + NAMESPACE_DELETE_EXCEPTION("Hbase-09", "Hbase namespace delete failed"), + TABLE_TRUNCATE_EXCEPTION("Hbase-10", "Hbase table truncate failed"), + ; private final String code; private final String description; 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 4f7b929223f..77308081618 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 @@ -17,76 +17,102 @@ package org.apache.seatunnel.connectors.seatunnel.hbase.sink; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; +import org.apache.seatunnel.api.sink.SaveModeHandler; +import org.apache.seatunnel.api.sink.SchemaSaveMode; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.factory.CatalogFactory; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.config.CheckConfigUtil; -import org.apache.seatunnel.common.config.CheckResult; -import org.apache.seatunnel.common.constants.PluginType; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; -import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; - -import com.google.auto.service.AutoService; +import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseAggregatedCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseSinkState; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Optional; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.FAMILY_NAME; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ROWKEY_COLUMNS; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; -import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.ZOOKEEPER_QUORUM; +import static org.apache.seatunnel.api.table.factory.FactoryUtil.discoverFactory; -@AutoService(SeaTunnelSink.class) -public class HbaseSink extends AbstractSimpleSink { +public class HbaseSink + implements SeaTunnelSink< + SeaTunnelRow, HbaseSinkState, HbaseCommitInfo, HbaseAggregatedCommitInfo>, + SupportMultiTableSink, + SupportSaveMode { - private Config pluginConfig; + private ReadonlyConfig config; - private SeaTunnelRowType seaTunnelRowType; + private CatalogTable catalogTable; - private HbaseParameters hbaseParameters; + private final HbaseParameters hbaseParameters; + + private SeaTunnelRowType seaTunnelRowType; private List rowkeyColumnIndexes = new ArrayList<>(); private int versionColumnIndex = -1; + public HbaseSink(ReadonlyConfig config, CatalogTable catalogTable) { + this.hbaseParameters = HbaseParameters.buildWithSinkConfig(config.toConfig()); + this.config = config; + this.catalogTable = catalogTable; + // this.seaTunnelRowType = seaTunnelRowType; + // for (String rowkeyColumn : hbaseParameters.getRowkeyColumns()) { + // this.rowkeyColumnIndexes.add(seaTunnelRowType.indexOf(rowkeyColumn)); + // } + if (hbaseParameters.getVersionColumn() != null) { + this.versionColumnIndex = seaTunnelRowType.indexOf(hbaseParameters.getVersionColumn()); + } + } + @Override public String getPluginName() { return HbaseSinkFactory.IDENTIFIER; } @Override - public void prepare(Config pluginConfig) throws PrepareFailException { - this.pluginConfig = pluginConfig; - CheckResult result = - CheckConfigUtil.checkAllExists( - pluginConfig, - ZOOKEEPER_QUORUM.key(), - TABLE.key(), - ROWKEY_COLUMNS.key(), - FAMILY_NAME.key()); - if (!result.isSuccess()) { - throw new HbaseConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format( - "PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SINK, result.getMsg())); - } - this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); - if (hbaseParameters.getFamilyNames().size() == 0) { - throw new HbaseConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - "The corresponding field options should be configured and should not be empty Refer to the hbase sink document"); - } + public HbaseSinkWriter createWriter(SinkWriter.Context context) throws IOException { + return new HbaseSinkWriter( + seaTunnelRowType, hbaseParameters, rowkeyColumnIndexes, versionColumnIndex); } + // @Override + // public void prepare(Config pluginConfig) throws PrepareFailException { + // this.pluginConfig = pluginConfig; + // CheckResult result = + // CheckConfigUtil.checkAllExists( + // pluginConfig, + // ZOOKEEPER_QUORUM.key(), + // TABLE.key(), + // ROWKEY_COLUMNS.key(), + // FAMILY_NAME.key()); + // if (!result.isSuccess()) { + // throw new HbaseConnectorException( + // SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + // String.format( + // "PluginName: %s, PluginType: %s, Message: %s", + // getPluginName(), PluginType.SINK, result.getMsg())); + // } + // this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); + // if (hbaseParameters.getFamilyNames().size() == 0) { + // throw new HbaseConnectorException( + // SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + // "The corresponding field options should be configured and should not be + // empty Refer to the hbase sink document"); + // } + // } + @Override public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) { this.seaTunnelRowType = seaTunnelRowType; @@ -99,9 +125,22 @@ public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) { } @Override - public AbstractSinkWriter createWriter(SinkWriter.Context context) - throws IOException { - return new HbaseSinkWriter( - seaTunnelRowType, hbaseParameters, rowkeyColumnIndexes, versionColumnIndex); + public Optional getSaveModeHandler() { + CatalogFactory catalogFactory = + discoverFactory( + Thread.currentThread().getContextClassLoader(), + CatalogFactory.class, + getPluginName()); + if (catalogFactory == null) { + return Optional.empty(); + } + Catalog catalog = catalogFactory.createCatalog(catalogFactory.factoryIdentifier(), config); + SchemaSaveMode schemaSaveMode = config.get(HbaseConfig.SCHEMA_SAVE_MODE); + DataSaveMode dataSaveMode = config.get(HbaseConfig.DATA_SAVE_MODE); + + TablePath tablePath = TablePath.of("", catalogTable.getTableId().getTableName()); + return Optional.of( + new DefaultSaveModeHandler( + schemaSaveMode, dataSaveMode, catalog, tablePath, null, null)); } } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java index 3038473c4ed..44120936238 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java @@ -17,9 +17,12 @@ package org.apache.seatunnel.connectors.seatunnel.hbase.sink; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; 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; @@ -59,4 +62,10 @@ public OptionRule optionRule() { HBASE_EXTRA_CONFIG) .build(); } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + ReadonlyConfig readonlyConfig = context.getOptions(); + return () -> new HbaseSink(readonlyConfig, context.getCatalogTable()); + } } diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java index 7683d6aab0b..9504415d68e 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java @@ -17,24 +17,20 @@ package org.apache.seatunnel.connectors.seatunnel.hbase.sink; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.hbase.client.HbaseClient; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; +import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseSinkState; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.BufferedMutator; -import org.apache.hadoop.hbase.client.BufferedMutatorParams; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.util.Bytes; @@ -42,18 +38,17 @@ import java.nio.charset.Charset; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.IntStream; -public class HbaseSinkWriter extends AbstractSinkWriter { +public class HbaseSinkWriter + implements SinkWriter, + SupportMultiTableSinkWriter { private static final String ALL_COLUMNS = "all_columns"; - private final Configuration hbaseConfiguration = HBaseConfiguration.create(); - - private final Connection hbaseConnection; - - private final BufferedMutator hbaseMutator; + private final HbaseClient hbaseClient; private final SeaTunnelRowType seaTunnelRowType; @@ -69,8 +64,7 @@ public HbaseSinkWriter( SeaTunnelRowType seaTunnelRowType, HbaseParameters hbaseParameters, List rowkeyColumnIndexes, - int versionColumnIndex) - throws IOException { + int versionColumnIndex) { this.seaTunnelRowType = seaTunnelRowType; this.hbaseParameters = hbaseParameters; this.rowkeyColumnIndexes = rowkeyColumnIndexes; @@ -80,34 +74,27 @@ public HbaseSinkWriter( this.writeAllColumnFamily = hbaseParameters.getFamilyNames().get(ALL_COLUMNS); } - // initialize hbase configuration - hbaseConfiguration.set("hbase.zookeeper.quorum", hbaseParameters.getZookeeperQuorum()); - if (hbaseParameters.getHbaseExtraConfig() != null) { - hbaseParameters.getHbaseExtraConfig().forEach(hbaseConfiguration::set); - } - // initialize hbase connection - hbaseConnection = ConnectionFactory.createConnection(hbaseConfiguration); - // initialize hbase mutator - BufferedMutatorParams bufferedMutatorParams = - new BufferedMutatorParams(TableName.valueOf(hbaseParameters.getTable())) - .pool(HTable.getDefaultExecutor(hbaseConfiguration)) - .writeBufferSize(hbaseParameters.getWriteBufferSize()); - hbaseMutator = hbaseConnection.getBufferedMutator(bufferedMutatorParams); + this.hbaseClient = HbaseClient.createInstance(hbaseParameters); } @Override public void write(SeaTunnelRow element) throws IOException { Put put = convertRowToPut(element); - hbaseMutator.mutate(put); + hbaseClient.mutate(put); } + @Override + public Optional prepareCommit() throws IOException { + return Optional.empty(); + } + + @Override + public void abortPrepare() {} + @Override public void close() throws IOException { - if (hbaseMutator != null) { - hbaseMutator.close(); - } - if (hbaseConnection != null) { - hbaseConnection.close(); + if (hbaseClient != null) { + hbaseClient.close(); } } 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 3aca3161516..aefc071a8ea 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 @@ -53,7 +53,6 @@ public class HbaseSource SupportColumnProjection { private static final Logger LOG = LoggerFactory.getLogger(HbaseSource.class); public static final String PLUGIN_NAME = "Hbase"; - private Config pluginConfig; private SeaTunnelRowType seaTunnelRowType; private HbaseParameters hbaseParameters; @@ -65,7 +64,6 @@ public String getPluginName() { } HbaseSource(Config pluginConfig) { - this.pluginConfig = pluginConfig; CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, ZOOKEEPER_QUORUM.key(), TABLE.key()); if (!result.isSuccess()) { 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 526ac826db1..9a8445d395b 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 @@ -22,16 +22,12 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.hbase.client.HbaseClient; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import org.apache.seatunnel.connectors.seatunnel.hbase.format.HBaseDeserializationFormat; -import org.apache.seatunnel.connectors.seatunnel.hbase.utils.HbaseConnectionUtil; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Connection; 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; @@ -58,10 +54,10 @@ public class HbaseSourceReader implements SourceReader columnNames; - private final transient Connection connection; private HBaseDeserializationFormat hbaseDeserializationFormat = new HBaseDeserializationFormat(); @@ -85,8 +81,7 @@ public HbaseSourceReader( Preconditions.checkArgument( column.contains(":") && column.split(":").length == 2, "Invalid column names, it should be [ColumnFamily:Column] format")); - - connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); + hbaseClient = HbaseClient.createInstance(hbaseParameters); } @Override @@ -103,9 +98,9 @@ public void close() throws IOException { throw new IOException("Failed to close HBase Scanner.", e); } } - if (this.connection != null) { + if (this.hbaseClient != null) { try { - this.connection.close(); + this.hbaseClient.close(); } catch (Exception e) { throw new IOException("Failed to close HBase connection.", e); } @@ -119,23 +114,8 @@ public void pollNext(Collector output) throws Exception { final HbaseSourceSplit split = sourceSplits.poll(); if (Objects.nonNull(split)) { // read logic - if (this.currentScanner == null) { - 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())) - .getScanner(scan); + if (currentScanner == null) { + currentScanner = hbaseClient.scan(split, hbaseParameters, this.columnNames); } for (Result result : currentScanner) { SeaTunnelRow seaTunnelRow = diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java index 094128b1747..f5508c9037d 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/source/HbaseSourceSplitEnumerator.java @@ -18,14 +18,10 @@ package org.apache.seatunnel.connectors.seatunnel.hbase.source; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.hbase.client.HbaseClient; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; -import org.apache.seatunnel.connectors.seatunnel.hbase.utils.HbaseConnectionUtil; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionLocator; import lombok.extern.slf4j.Slf4j; @@ -43,7 +39,6 @@ public class HbaseSourceSplitEnumerator /** Source split enumerator context */ private final Context context; - private Config pluginConfig; /** The splits that has assigned */ private final Set assignedSplit; @@ -51,24 +46,29 @@ public class HbaseSourceSplitEnumerator private Set pendingSplit; private HbaseParameters hbaseParameters; - private Connection connection; + + private HbaseClient hbaseClient; public HbaseSourceSplitEnumerator( Context context, HbaseParameters hbaseParameters) { - this.context = context; - this.hbaseParameters = hbaseParameters; - this.assignedSplit = new HashSet<>(); - connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); + this(context, hbaseParameters, new HashSet<>()); } public HbaseSourceSplitEnumerator( Context context, HbaseParameters hbaseParameters, HbaseSourceState sourceState) { + this(context, hbaseParameters, sourceState.getAssignedSplits()); + } + + private HbaseSourceSplitEnumerator( + Context context, + HbaseParameters hbaseParameters, + Set assignedSplit) { this.context = context; this.hbaseParameters = hbaseParameters; - this.assignedSplit = sourceState.getAssignedSplits(); - connection = HbaseConnectionUtil.getHbaseConnection(hbaseParameters); + this.assignedSplit = assignedSplit; + this.hbaseClient = HbaseClient.createInstance(hbaseParameters); } @Override @@ -157,8 +157,7 @@ private Set getTableSplits() { List splits = new ArrayList<>(); try { - RegionLocator regionLocator = - connection.getRegionLocator(TableName.valueOf(hbaseParameters.getTable())); + RegionLocator regionLocator = hbaseClient.getRegionLocator(hbaseParameters.getTable()); byte[][] startKeys = regionLocator.getStartKeys(); byte[][] endKeys = regionLocator.getEndKeys(); if (startKeys.length != endKeys.length) { diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseAggregatedCommitInfo.java new file mode 100644 index 00000000000..c1996dc057e --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseAggregatedCommitInfo.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hbase.state; + +import java.io.Serializable; + +public class HbaseAggregatedCommitInfo implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseCommitInfo.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseCommitInfo.java new file mode 100644 index 00000000000..39999ceddcd --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseCommitInfo.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hbase.state; + +import java.io.Serializable; + +public class HbaseCommitInfo implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseSinkState.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseSinkState.java new file mode 100644 index 00000000000..6e1f068cf68 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/state/HbaseSinkState.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hbase.state; + +import java.io.Serializable; + +public class HbaseSinkState implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java deleted file mode 100644 index f006986e660..00000000000 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/utils/HbaseConnectionUtil.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.hbase.utils; - -import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; -import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; - -import java.io.IOException; - -public class HbaseConnectionUtil { - public static Connection getHbaseConnection(HbaseParameters hbaseParameters) { - Configuration hbaseConfiguration = HBaseConfiguration.create(); - hbaseConfiguration.set("hbase.zookeeper.quorum", hbaseParameters.getZookeeperQuorum()); - if (hbaseParameters.getHbaseExtraConfig() != null) { - hbaseParameters.getHbaseExtraConfig().forEach(hbaseConfiguration::set); - } - // initialize hbase connection - try { - Connection connection = ConnectionFactory.createConnection(hbaseConfiguration); - return connection; - } catch (IOException e) { - String errorMsg = "Build Hbase connection failed."; - throw new HbaseConnectorException(HbaseConnectorErrorCode.CONNECTION_FAILED, errorMsg); - } - } -} From 6864988142a19521d4115f40b731dcfddcb45451 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Wed, 28 Aug 2024 16:18:30 +0800 Subject: [PATCH 02/19] [feature]first version --- .../connector-hbase/pom.xml | 7 ++ .../seatunnel/hbase/catalog/HbaseCatalog.java | 12 +- .../hbase/catalog/HbaseCatalogFactory.java | 7 +- .../seatunnel/hbase/client/HbaseClient.java | 109 +++++++++++++++- .../hbase/config/HbaseParameters.java | 3 +- .../hbase/constant/HbaseIdentifier.java | 22 ++++ .../seatunnel/hbase/sink/HbaseSink.java | 47 ++----- .../hbase/sink/HbaseSinkFactory.java | 15 ++- .../seatunnel/hbase/sink/HbaseSinkWriter.java | 5 +- .../seatunnel/hbase/source/HbaseSource.java | 8 +- .../hbase/source/HbaseSourceFactory.java | 5 +- .../e2e/connector/hbase/HbaseIT.java | 117 +++++++++++++++++- .../fake_to_hbase_with_append_data.conf | 52 ++++++++ ..._to_hbase_with_create_when_not_exists.conf | 51 ++++++++ .../fake_to_hbase_with_drop_data.conf | 52 ++++++++ ..._to_hbase_with_error_when_data_exists.conf | 52 ++++++++ ...e_to_hbase_with_error_when_not_exists.conf | 51 ++++++++ .../fake_to_hbase_with_recreate_schema.conf | 51 ++++++++ 18 files changed, 591 insertions(+), 75 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/constant/HbaseIdentifier.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_append_data.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_create_when_not_exists.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_drop_data.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_data_exists.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_not_exists.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_recreate_schema.conf diff --git a/seatunnel-connectors-v2/connector-hbase/pom.xml b/seatunnel-connectors-v2/connector-hbase/pom.xml index 663bdcfdd30..b2ca1849911 100644 --- a/seatunnel-connectors-v2/connector-hbase/pom.xml +++ b/seatunnel-connectors-v2/connector-hbase/pom.xml @@ -39,6 +39,7 @@ org.apache.seatunnel connector-common ${project.version} + compile @@ -47,6 +48,12 @@ ${hbase.version} + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java index 174a3ce51a7..6659f313552 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java @@ -33,9 +33,6 @@ import org.apache.seatunnel.connectors.seatunnel.hbase.client.HbaseClient; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import lombok.extern.slf4j.Slf4j; import java.util.Collections; @@ -47,22 +44,16 @@ import static com.google.common.base.Preconditions.checkNotNull; -/** - * Hbase catalog implementation. - * - */ +/** Hbase catalog implementation. */ @Slf4j public class HbaseCatalog implements Catalog { - private static final Logger LOGGER = LoggerFactory.getLogger(HbaseCatalog.class); - private final String catalogName; private final String defaultDatabase; private final HbaseParameters hbaseParameters; private HbaseClient hbaseClient; - // todo: do we need default database? public HbaseCatalog( String catalogName, String defaultDatabase, HbaseParameters hbaseParameters) { this.catalogName = checkNotNull(catalogName, "catalogName cannot be null"); @@ -127,6 +118,7 @@ public CatalogTable getTable(TablePath tablePath) return CatalogTable.of( TableIdentifier.of( catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), + // Hbase cannot obtain column names, so TableSchema was directly built here TableSchema.builder().build(), buildTableOptions(tablePath), Collections.emptyList(), diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java index 73ec3206777..6894fba17fc 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.table.factory.CatalogFactory; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import com.google.auto.service.AutoService; @@ -31,13 +32,15 @@ public class HbaseCatalogFactory implements CatalogFactory { @Override public Catalog createCatalog(String catalogName, ReadonlyConfig options) { - HbaseParameters hbaseParameters = HbaseParameters.buildWithSourceConfig(options.toConfig()); + // Create an instance of HbaseCatalog, passing in the catalog name, namespace, and Hbase + // parameters + HbaseParameters hbaseParameters = HbaseParameters.buildWithSinkConfig(options.toConfig()); return new HbaseCatalog(catalogName, hbaseParameters.getNamespace(), hbaseParameters); } @Override public String factoryIdentifier() { - return "Hbase"; + return HbaseIdentifier.IDENTIFIER_NAME; } @Override diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java index 031828bb70b..88897caf89f 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java @@ -63,8 +63,13 @@ public class HbaseClient { private final BufferedMutator hbaseMutator; public static Configuration hbaseConfiguration; + /** + * Constructor for HbaseClient. + * + * @param connection Hbase connection + * @param hbaseParameters Hbase parameters + */ private HbaseClient(Connection connection, HbaseParameters hbaseParameters) { - System.out.println("V1.000"); this.connection = connection; try { this.admin = connection.getAdmin(); @@ -83,10 +88,22 @@ private HbaseClient(Connection connection, HbaseParameters hbaseParameters) { } } + /** + * Create a new instance of HbaseClient. + * + * @param hbaseParameters Hbase parameters + * @return HbaseClient + */ public static HbaseClient createInstance(HbaseParameters hbaseParameters) { return new HbaseClient(getHbaseConnection(hbaseParameters), hbaseParameters); } + /** + * Get Hbase connection. + * + * @param hbaseParameters Hbase parameters + * @return Hbase connection + */ private static Connection getHbaseConnection(HbaseParameters hbaseParameters) { hbaseConfiguration = HBaseConfiguration.create(); hbaseConfiguration.set("hbase.zookeeper.quorum", hbaseParameters.getZookeeperQuorum()); @@ -103,6 +120,12 @@ private static Connection getHbaseConnection(HbaseParameters hbaseParameters) { } } + /** + * Check if a database exists. + * + * @param databaseName database name + * @return true if the database exists, false otherwise + */ public boolean databaseExists(String databaseName) { try { return Arrays.stream(admin.listNamespaceDescriptors()) @@ -115,6 +138,11 @@ public boolean databaseExists(String databaseName) { } } + /** + * List all databases. + * + * @return List of database names + */ public List listDatabases() { try { return Arrays.stream(admin.listNamespaceDescriptors()) @@ -128,6 +156,12 @@ public List listDatabases() { } } + /** + * List all tables in a database. + * + * @param databaseName database name + * @return List of table names + */ public List listTables(String databaseName) { try { return Arrays.stream(admin.listTableNamesByNamespace(databaseName)) @@ -141,6 +175,12 @@ public List listTables(String databaseName) { } } + /** + * Check if a table exists. + * + * @param tableName table name + * @return true if the table exists, false otherwise + */ public boolean tableExists(String tableName) { try { return admin.tableExists(TableName.valueOf(tableName)); @@ -152,6 +192,14 @@ public boolean tableExists(String tableName) { } } + /** + * Create a table. + * + * @param databaseName database name + * @param tableName table name + * @param columnFamilies column families + * @param ignoreIfExists ignore if the table already exists + */ public void createTable( String databaseName, String tableName, @@ -186,6 +234,12 @@ public void createTable( } } + /** + * Drop a table. + * + * @param databaseName database name + * @param tableName table name + */ public void dropTable(String databaseName, String tableName) { try { TableName table = TableName.valueOf(databaseName, tableName); @@ -199,6 +253,11 @@ public void dropTable(String databaseName, String tableName) { } } + /** + * Create a namespace. + * + * @param namespace namespace name + */ public void createNamespace(String namespace) { try { admin.createNamespace(NamespaceDescriptor.create(namespace).build()); @@ -210,6 +269,11 @@ public void createNamespace(String namespace) { } } + /** + * Drop a namespace. + * + * @param namespace namespace name + */ public void deleteNamespace(String namespace) { try { admin.deleteNamespace(namespace); @@ -221,9 +285,16 @@ public void deleteNamespace(String namespace) { } } + /** + * Truncate a table. + * + * @param databaseName database name + * @param tableName table name + */ public void truncateTable(String databaseName, String tableName) { try { TableName table = TableName.valueOf(databaseName, tableName); + admin.disableTable(table); admin.truncateTable(table, true); } catch (IOException e) { throw new HbaseConnectorException( @@ -233,6 +304,13 @@ public void truncateTable(String databaseName, String tableName) { } } + /** + * Check if a table has data. + * + * @param databaseName database name + * @param tableName table name + * @return true if the table has data, false otherwise + */ public boolean isExistsData(String databaseName, String tableName) { try { Table table = @@ -243,7 +321,7 @@ public boolean isExistsData(String databaseName, String tableName) { scan.setLimit(1); try (ResultScanner scanner = table.getScanner(scan)) { Result result = scanner.next(); - return result != null && result.isEmpty(); + return !result.isEmpty(); } } catch (IOException e) { throw new HbaseConnectorException( @@ -253,8 +331,13 @@ public boolean isExistsData(String databaseName, String tableName) { } } + /** Close Hbase connection. */ public void close() { try { + if (hbaseMutator != null) { + hbaseMutator.flush(); + hbaseMutator.close(); + } if (admin != null) { admin.close(); } @@ -266,10 +349,25 @@ public void close() { } } + /** + * Mutate a Put. + * + * @param put Hbase put + * @throws IOException exception + */ public void mutate(Put put) throws IOException { hbaseMutator.mutate(put); } + /** + * Scan a table. + * + * @param split Hbase source split + * @param hbaseParameters Hbase parameters + * @param columnNames column names + * @return ResultScanner + * @throws IOException exception + */ public ResultScanner scan( HbaseSourceSplit split, HbaseParameters hbaseParameters, List columnNames) throws IOException { @@ -288,6 +386,13 @@ public ResultScanner scan( .getScanner(scan); } + /** + * Get a RegionLocator. + * + * @param tableName table name + * @return RegionLocator + * @throws IOException exception + */ public RegionLocator getRegionLocator(String tableName) throws IOException { return this.connection.getRegionLocator(TableName.valueOf(tableName)); } 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 5e237319624..d63c8ef8efd 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 @@ -84,8 +84,6 @@ public class HbaseParameters implements Serializable { public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { HbaseParametersBuilder builder = HbaseParameters.builder(); - - // required parameters builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); String table = pluginConfig.getString(TABLE.key()); int colonIndex = table.indexOf(':'); @@ -95,6 +93,7 @@ public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { builder.table(table.substring(colonIndex + 1)); } else { builder.table(table); + builder.namespace("default"); } builder.rowkeyColumns(pluginConfig.getStringList(ROWKEY_COLUMNS.key())); diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/constant/HbaseIdentifier.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/constant/HbaseIdentifier.java new file mode 100644 index 00000000000..3d84216d660 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/constant/HbaseIdentifier.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.hbase.constant; + +public class HbaseIdentifier { + public static final String IDENTIFIER_NAME = "Hbase"; +} 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 77308081618..14f8ec8c7ef 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 @@ -34,6 +34,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseAggregatedCommitInfo; import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseCommitInfo; import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseSinkState; @@ -67,10 +68,7 @@ public HbaseSink(ReadonlyConfig config, CatalogTable catalogTable) { this.hbaseParameters = HbaseParameters.buildWithSinkConfig(config.toConfig()); this.config = config; this.catalogTable = catalogTable; - // this.seaTunnelRowType = seaTunnelRowType; - // for (String rowkeyColumn : hbaseParameters.getRowkeyColumns()) { - // this.rowkeyColumnIndexes.add(seaTunnelRowType.indexOf(rowkeyColumn)); - // } + this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); if (hbaseParameters.getVersionColumn() != null) { this.versionColumnIndex = seaTunnelRowType.indexOf(hbaseParameters.getVersionColumn()); } @@ -78,50 +76,19 @@ public HbaseSink(ReadonlyConfig config, CatalogTable catalogTable) { @Override public String getPluginName() { - return HbaseSinkFactory.IDENTIFIER; + return HbaseIdentifier.IDENTIFIER_NAME; } @Override public HbaseSinkWriter createWriter(SinkWriter.Context context) throws IOException { - return new HbaseSinkWriter( - seaTunnelRowType, hbaseParameters, rowkeyColumnIndexes, versionColumnIndex); - } - - // @Override - // public void prepare(Config pluginConfig) throws PrepareFailException { - // this.pluginConfig = pluginConfig; - // CheckResult result = - // CheckConfigUtil.checkAllExists( - // pluginConfig, - // ZOOKEEPER_QUORUM.key(), - // TABLE.key(), - // ROWKEY_COLUMNS.key(), - // FAMILY_NAME.key()); - // if (!result.isSuccess()) { - // throw new HbaseConnectorException( - // SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - // String.format( - // "PluginName: %s, PluginType: %s, Message: %s", - // getPluginName(), PluginType.SINK, result.getMsg())); - // } - // this.hbaseParameters = HbaseParameters.buildWithSinkConfig(pluginConfig); - // if (hbaseParameters.getFamilyNames().size() == 0) { - // throw new HbaseConnectorException( - // SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - // "The corresponding field options should be configured and should not be - // empty Refer to the hbase sink document"); - // } - // } - - @Override - public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) { - this.seaTunnelRowType = seaTunnelRowType; for (String rowkeyColumn : hbaseParameters.getRowkeyColumns()) { this.rowkeyColumnIndexes.add(seaTunnelRowType.indexOf(rowkeyColumn)); } if (hbaseParameters.getVersionColumn() != null) { this.versionColumnIndex = seaTunnelRowType.indexOf(hbaseParameters.getVersionColumn()); } + return new HbaseSinkWriter( + seaTunnelRowType, hbaseParameters, rowkeyColumnIndexes, versionColumnIndex); } @Override @@ -137,8 +104,8 @@ public Optional getSaveModeHandler() { Catalog catalog = catalogFactory.createCatalog(catalogFactory.factoryIdentifier(), config); SchemaSaveMode schemaSaveMode = config.get(HbaseConfig.SCHEMA_SAVE_MODE); DataSaveMode dataSaveMode = config.get(HbaseConfig.DATA_SAVE_MODE); - - TablePath tablePath = TablePath.of("", catalogTable.getTableId().getTableName()); + TablePath tablePath = + TablePath.of(hbaseParameters.getNamespace(), hbaseParameters.getTable()); return Optional.of( new DefaultSaveModeHandler( schemaSaveMode, dataSaveMode, catalog, tablePath, null, null)); diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java index 44120936238..8093364e9a5 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java @@ -23,15 +23,18 @@ import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import com.google.auto.service.AutoService; +import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.DATA_SAVE_MODE; 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_EXTRA_CONFIG; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.NULL_MODE; 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.SCHEMA_SAVE_MODE; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.TABLE; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.VERSION_COLUMN; import static org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig.WAL_WRITE; @@ -41,17 +44,21 @@ @AutoService(Factory.class) public class HbaseSinkFactory implements TableSinkFactory { - public static final String IDENTIFIER = "Hbase"; - @Override public String factoryIdentifier() { - return IDENTIFIER; + return HbaseIdentifier.IDENTIFIER_NAME; } @Override public OptionRule optionRule() { return OptionRule.builder() - .required(ZOOKEEPER_QUORUM, TABLE, ROWKEY_COLUMNS, FAMILY_NAME) + .required( + ZOOKEEPER_QUORUM, + TABLE, + ROWKEY_COLUMNS, + FAMILY_NAME, + SCHEMA_SAVE_MODE, + DATA_SAVE_MODE) .optional( ROWKEY_DELIMITER, VERSION_COLUMN, diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java index 9504415d68e..a41a2386297 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java @@ -54,9 +54,9 @@ public class HbaseSinkWriter private final HbaseParameters hbaseParameters; - private final List rowkeyColumnIndexes; + private List rowkeyColumnIndexes; - private final int versionColumnIndex; + private int versionColumnIndex; private String writeAllColumnFamily; @@ -119,7 +119,6 @@ private Put convertRowToPut(SeaTunnelRow row) { .collect(Collectors.toList()); for (Integer writeColumnIndex : writeColumnIndexes) { String fieldName = seaTunnelRowType.getFieldName(writeColumnIndex); - // This is the family of columns that we define to be written through the.conf file Map configurationFamilyNames = hbaseParameters.getFamilyNames(); String familyName = configurationFamilyNames.getOrDefault(fieldName, writeAllColumnFamily); 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 aefc071a8ea..1a597eea133 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 @@ -35,11 +35,9 @@ import org.apache.seatunnel.common.config.CheckResult; import org.apache.seatunnel.common.constants.PluginType; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; +import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.hbase.exception.HbaseConnectorException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.google.common.collect.Lists; import java.util.List; @@ -51,8 +49,6 @@ public class HbaseSource implements SeaTunnelSource, SupportParallelism, SupportColumnProjection { - private static final Logger LOG = LoggerFactory.getLogger(HbaseSource.class); - public static final String PLUGIN_NAME = "Hbase"; private SeaTunnelRowType seaTunnelRowType; private HbaseParameters hbaseParameters; @@ -60,7 +56,7 @@ public class HbaseSource @Override public String getPluginName() { - return PLUGIN_NAME; + return HbaseIdentifier.IDENTIFIER_NAME; } HbaseSource(Config pluginConfig) { 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 2de385dbd18..5e250337d7b 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 @@ -26,18 +26,17 @@ import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig; +import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import com.google.auto.service.AutoService; import java.io.Serializable; -import static org.apache.seatunnel.connectors.seatunnel.hbase.sink.HbaseSinkFactory.IDENTIFIER; - @AutoService(Factory.class) public class HbaseSourceFactory implements TableSourceFactory { @Override public String factoryIdentifier() { - return IDENTIFIER; + return HbaseIdentifier.IDENTIFIER_NAME; } @Override 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 85ceef92353..ae847111ba0 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 @@ -21,6 +21,7 @@ import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestContainerId; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.hadoop.hbase.Cell; @@ -29,10 +30,12 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Put; 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.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.util.Bytes; import org.junit.jupiter.api.AfterAll; @@ -50,9 +53,13 @@ @Slf4j @DisabledOnContainer( - value = {}, - type = {EngineType.SEATUNNEL}, - disabledReason = "The hbase container authentication configuration is incorrect.") + value = { + TestContainerId.FLINK_1_13, + TestContainerId.FLINK_1_14, + TestContainerId.FLINK_1_15 + }, + type = {EngineType.SEATUNNEL, EngineType.SPARK}, + disabledReason = "Test only one engine for first change") public class HbaseIT extends TestSuiteBase implements TestResource { private static final String TABLE_NAME = "seatunnel_test"; @@ -98,6 +105,84 @@ public void testHbaseSink(TestContainer container) throws IOException, Interrupt Assertions.assertEquals(0, sourceExecResult.getExitCode()); } + @TestTemplate + public void testHbaseSinkWithErrorWhenDataExists(TestContainer container) + throws IOException, InterruptedException { + deleteData(table); + insertData(table); + Assertions.assertEquals(5, countData(table)); + Container.ExecResult execResult = + container.executeJob("/fake_to_hbase_with_error_when_data_exists.conf"); + Assertions.assertEquals(1, execResult.getExitCode()); + } + + @TestTemplate + public void testHbaseSinkWithRecreateSchema(TestContainer container) + throws IOException, InterruptedException { + String tableName = "seatunnel_test_with_recreate_schema"; + TableName table = TableName.valueOf(tableName); + hbaseCluster.createTable(tableName, Arrays.asList("test_rs")); + TableDescriptor descriptorBefore = hbaseConnection.getTable(table).getDescriptor(); + String[] familiesBefore = + Arrays.stream(descriptorBefore.getColumnFamilies()) + .map(f -> f.getNameAsString()) + .toArray(String[]::new); + Assertions.assertTrue(Arrays.equals(familiesBefore, new String[] {"test_rs"})); + Container.ExecResult execResult = + container.executeJob("/fake_to_hbase_with_recreate_schema.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + TableDescriptor descriptorAfter = hbaseConnection.getTable(table).getDescriptor(); + String[] familiesAfter = + Arrays.stream(descriptorAfter.getColumnFamilies()) + .map(f -> f.getNameAsString()) + .toArray(String[]::new); + Assertions.assertTrue(!Arrays.equals(familiesBefore, familiesAfter)); + } + + @TestTemplate + public void testHbaseSinkWithDropData(TestContainer container) + throws IOException, InterruptedException { + deleteData(table); + insertData(table); + countData(table); + Assertions.assertEquals(5, countData(table)); + Container.ExecResult execResult = + container.executeJob("/fake_to_hbase_with_drop_data.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(5, countData(table)); + } + + @TestTemplate + public void testHbaseSinkWithCreateWhenNotExists(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = + container.executeJob("/fake_to_hbase_with_create_when_not_exists.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals( + 5, countData(TableName.valueOf("seatunnel_test_with_create_when_not_exists"))); + } + + @TestTemplate + public void testHbaseSinkWithAppendData(TestContainer container) + throws IOException, InterruptedException { + deleteData(table); + insertData(table); + countData(table); + Assertions.assertEquals(5, countData(table)); + Container.ExecResult execResult = + container.executeJob("/fake_to_hbase_with_append_data.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(10, countData(table)); + } + + @TestTemplate + public void testHbaseSinkWithErrorWhenNotExists(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = + container.executeJob("/fake_to_hbase_with_error_when_not_exists.conf"); + Assertions.assertEquals(1, execResult.getExitCode()); + } + @TestTemplate public void testHbaseSinkWithArray(TestContainer container) throws IOException, InterruptedException { @@ -200,4 +285,30 @@ private void deleteData(TableName table) throws IOException { hbaseTable.delete(deleteRow); } } + + private void insertData(TableName table) throws IOException { + Table hbaseTable = hbaseConnection.getTable(table); + for (int i = 0; i < 5; i++) { + String rowKey = "row" + System.currentTimeMillis(); + String value = "value" + i; + hbaseTable.put( + new Put(Bytes.toBytes(rowKey)) + .addColumn( + Bytes.toBytes(FAMILY_NAME), + Bytes.toBytes("name"), + Bytes.toBytes(value))); + } + } + + private int countData(TableName table) throws IOException { + Table hbaseTable = hbaseConnection.getTable(table); + Scan scan = new Scan(); + ResultScanner scanner = hbaseTable.getScanner(scan); + int count = 0; + for (Result result = scanner.next(); result != null; result = scanner.next()) { + count++; + } + scanner.close(); + return count; + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_append_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_append_data.conf new file mode 100644 index 00000000000..0778d8cb36b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_append_data.conf @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema { + fields { + name = string + age = int + c_tinyint = tinyint + c_smallint = smallint + c_bigint = bigint + c_float = float + c_double = double + c_boolean = boolean + } + } + } +} + +sink { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test" + rowkey_column = ["name"] + family_name { + all_columns = info + } + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_create_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_create_when_not_exists.conf new file mode 100644 index 00000000000..21327170824 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_create_when_not_exists.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema { + fields { + name = string + age = int + c_tinyint = tinyint + c_smallint = smallint + c_bigint = bigint + c_float = float + c_double = double + c_boolean = boolean + } + } + } +} + +sink { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test_with_create_when_not_exists" + rowkey_column = ["name"] + family_name { + all_columns = info + } + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_drop_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_drop_data.conf new file mode 100644 index 00000000000..66b3981206a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_drop_data.conf @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema { + fields { + name = string + age = int + c_tinyint = tinyint + c_smallint = smallint + c_bigint = bigint + c_float = float + c_double = double + c_boolean = boolean + } + } + } +} + +sink { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test_with_create_when_not_exists" + rowkey_column = ["name"] + family_name { + all_columns = info + } + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "DROP_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_data_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_data_exists.conf new file mode 100644 index 00000000000..00e0485e3d0 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_data_exists.conf @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema { + fields { + name = string + age = int + c_tinyint = tinyint + c_smallint = smallint + c_bigint = bigint + c_float = float + c_double = double + c_boolean = boolean + } + } + } +} + +sink { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test" + rowkey_column = ["name"] + family_name { + all_columns = info + } + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "ERROR_WHEN_DATA_EXISTS" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_not_exists.conf new file mode 100644 index 00000000000..359b71b79f3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_error_when_not_exists.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema { + fields { + name = string + age = int + c_tinyint = tinyint + c_smallint = smallint + c_bigint = bigint + c_float = float + c_double = double + c_boolean = boolean + } + } + } +} + +sink { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test_with_error_when_not_exists" + rowkey_column = ["name"] + family_name { + all_columns = info + } + schema_save_mode = "ERROR_WHEN_SCHEMA_NOT_EXIST" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_recreate_schema.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_recreate_schema.conf new file mode 100644 index 00000000000..c8a8c43d9ce --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake_to_hbase_with_recreate_schema.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema { + fields { + name = string + age = int + c_tinyint = tinyint + c_smallint = smallint + c_bigint = bigint + c_float = float + c_double = double + c_boolean = boolean + } + } + } +} + +sink { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "seatunnel_test_with_recreate_schema" + rowkey_column = ["name"] + family_name { + all_columns = info + } + schema_save_mode = "RECREATE_SCHEMA" + } +} \ No newline at end of file From 5f9a428118ec57a196802024ba84c78983f8b192 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Wed, 28 Aug 2024 17:18:31 +0800 Subject: [PATCH 03/19] [feature]fix some problem --- .../connectors/seatunnel/hbase/sink/HbaseSinkFactory.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java index 8093364e9a5..9fab21954a7 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; @@ -66,7 +67,8 @@ public OptionRule optionRule() { WAL_WRITE, WRITE_BUFFER_SIZE, ENCODING, - HBASE_EXTRA_CONFIG) + HBASE_EXTRA_CONFIG, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) .build(); } From 75ea71f58dbf728388900cbdf1c83547b0306f05 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Wed, 28 Aug 2024 21:29:27 +0800 Subject: [PATCH 04/19] [feature]fix some problem --- .../connectors/seatunnel/hbase/client/HbaseClient.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java index 88897caf89f..4bd08f637cd 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java @@ -54,10 +54,6 @@ @Slf4j public class HbaseClient { - private static final int CONNECTION_REQUEST_TIMEOUT = 10 * 1000; - - private static final int SOCKET_TIMEOUT = 5 * 60 * 1000; - private final Connection connection; private final Admin admin; private final BufferedMutator hbaseMutator; From b0bfed4b12b92af33bc6a2dbc47fa89621b80e01 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Thu, 29 Aug 2024 19:33:31 +0800 Subject: [PATCH 05/19] [feature]delete disable engine --- .../org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java | 8 -------- 1 file changed, 8 deletions(-) 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 ae847111ba0..15d86c5e210 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 @@ -52,14 +52,6 @@ import java.util.Objects; @Slf4j -@DisabledOnContainer( - value = { - TestContainerId.FLINK_1_13, - TestContainerId.FLINK_1_14, - TestContainerId.FLINK_1_15 - }, - type = {EngineType.SEATUNNEL, EngineType.SPARK}, - disabledReason = "Test only one engine for first change") public class HbaseIT extends TestSuiteBase implements TestResource { private static final String TABLE_NAME = "seatunnel_test"; From 5c5f8b146825f3b4785b8912ced3c5b4d0656290 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Thu, 29 Aug 2024 19:45:04 +0800 Subject: [PATCH 06/19] [feature]format --- .../java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java | 3 --- 1 file changed, 3 deletions(-) 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 15d86c5e210..bb2e1c81fe5 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 @@ -19,10 +19,7 @@ import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; -import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; -import org.apache.seatunnel.e2e.common.container.TestContainerId; -import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; From ee99317d4f2328ef741c0407801305a3a91e5e4f Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 30 Aug 2024 09:01:17 +0800 Subject: [PATCH 07/19] [feature]restore --- .../org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 bb2e1c81fe5..853c745a9ef 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 @@ -19,7 +19,9 @@ 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.junit.DisabledOnContainer; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -49,6 +51,10 @@ import java.util.Objects; @Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SEATUNNEL}, + disabledReason = "The hbase container authentication configuration is incorrect.") public class HbaseIT extends TestSuiteBase implements TestResource { private static final String TABLE_NAME = "seatunnel_test"; From 688cd22af756ed6921ba6d623325df183ef125d1 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 30 Aug 2024 11:22:22 +0800 Subject: [PATCH 08/19] [feature]fix some problem --- .../seatunnel/e2e/connector/hbase/HbaseIT.java | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) 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 853c745a9ef..592632991e1 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 @@ -76,6 +76,7 @@ public class HbaseIT extends TestSuiteBase implements TestResource { public void startUp() throws Exception { hbaseCluster = new HbaseCluster(); hbaseConnection = hbaseCluster.startService(); + admin = hbaseConnection.getAdmin(); // Create table for hbase sink test log.info("initial"); hbaseCluster.createTable(TABLE_NAME, Arrays.asList(FAMILY_NAME)); @@ -116,6 +117,7 @@ public void testHbaseSinkWithRecreateSchema(TestContainer container) throws IOException, InterruptedException { String tableName = "seatunnel_test_with_recreate_schema"; TableName table = TableName.valueOf(tableName); + dropTable(table); hbaseCluster.createTable(tableName, Arrays.asList("test_rs")); TableDescriptor descriptorBefore = hbaseConnection.getTable(table).getDescriptor(); String[] familiesBefore = @@ -150,11 +152,13 @@ public void testHbaseSinkWithDropData(TestContainer container) @TestTemplate public void testHbaseSinkWithCreateWhenNotExists(TestContainer container) throws IOException, InterruptedException { + TableName seatunnelTestWithCreateWhenNotExists = + TableName.valueOf("seatunnel_test_with_create_when_not_exists"); + dropTable(seatunnelTestWithCreateWhenNotExists); Container.ExecResult execResult = container.executeJob("/fake_to_hbase_with_create_when_not_exists.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals( - 5, countData(TableName.valueOf("seatunnel_test_with_create_when_not_exists"))); + Assertions.assertEquals(5, countData(seatunnelTestWithCreateWhenNotExists)); } @TestTemplate @@ -270,6 +274,13 @@ private void fakeToHbase(TestContainer container) throws IOException, Interrupte scanner.close(); } + private void dropTable(TableName tableName) throws IOException { + if (admin.tableExists(tableName)) { + admin.disableTable(tableName); + admin.deleteTable(tableName); + } + } + private void deleteData(TableName table) throws IOException { Table hbaseTable = hbaseConnection.getTable(table); Scan scan = new Scan(); From ddf891baaff628e51a79276bfeadbf1ab184bf87 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 30 Aug 2024 13:11:19 +0800 Subject: [PATCH 09/19] [feature]fix some problem --- .../java/org/apache/seatunnel/e2e/connector/hbase/HbaseIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 592632991e1..50075629249 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 @@ -49,6 +49,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Objects; +import java.util.UUID; @Slf4j @DisabledOnContainer( @@ -295,7 +296,7 @@ private void deleteData(TableName table) throws IOException { private void insertData(TableName table) throws IOException { Table hbaseTable = hbaseConnection.getTable(table); for (int i = 0; i < 5; i++) { - String rowKey = "row" + System.currentTimeMillis(); + String rowKey = "row" + UUID.randomUUID(); String value = "value" + i; hbaseTable.put( new Put(Bytes.toBytes(rowKey)) From 8b4b926c02c13a683aa95306c52f0039b4b5defc Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 14:09:51 +0800 Subject: [PATCH 10/19] init --- .../workflows/labeler/label-scope-conf.yml | 15 +- .github/workflows/publish-docker.yaml | 6 +- config/log4j2.properties | 6 +- config/plugin_config | 4 +- docs/en/concept/sql-config.md | 5 +- docs/en/connector-v2/sink/Clickhouse.md | 6 +- docs/en/connector-v2/sink/Hbase.md | 73 ++ docs/en/connector-v2/sink/Http.md | 6 +- docs/en/connector-v2/sink/Jdbc.md | 2 +- docs/en/connector-v2/sink/Kafka.md | 88 +- docs/en/connector-v2/sink/MongoDB.md | 6 +- docs/en/connector-v2/sink/Mysql.md | 2 +- docs/en/connector-v2/sink/Paimon.md | 2 +- docs/en/connector-v2/sink/Qdrant.md | 70 ++ docs/en/connector-v2/sink/Rabbitmq.md | 39 + docs/en/connector-v2/sink/Typesense.md | 93 ++ docs/en/connector-v2/source/Clickhouse.md | 6 +- docs/en/connector-v2/source/Elasticsearch.md | 150 ++- docs/en/connector-v2/source/Http.md | 6 +- docs/en/connector-v2/source/Jdbc.md | 27 +- docs/en/connector-v2/source/MongoDB-CDC.md | 6 +- docs/en/connector-v2/source/MongoDB.md | 6 +- docs/en/connector-v2/source/MyHours.md | 2 +- docs/en/connector-v2/source/Mysql.md | 8 +- docs/en/connector-v2/source/Opengauss-CDC.md | 170 ++++ docs/en/connector-v2/source/Oracle.md | 6 +- docs/en/connector-v2/source/PostgreSQL.md | 21 +- docs/en/connector-v2/source/Qdrant.md | 81 ++ docs/en/connector-v2/source/Sls.md | 6 +- docs/en/connector-v2/source/SqlServer.md | 6 +- docs/en/connector-v2/source/Tablestore.md | 102 ++ docs/en/connector-v2/source/Typesense.md | 79 ++ docs/en/connector-v2/source/kafka.md | 92 +- docs/en/faq.md | 21 - .../en/seatunnel-engine/checkpoint-storage.md | 1 - docs/en/seatunnel-engine/rest-api.md | 135 +++ docs/en/start-v2/docker/docker.md | 48 +- docs/en/start-v2/locally/deployment.md | 39 +- docs/en/transform-v2/embedding.md | 392 ++++++++ docs/en/transform-v2/llm.md | 168 +++- docs/sidebars.js | 24 +- docs/zh/concept/sql-config.md | 5 +- docs/zh/connector-v2/sink/Clickhouse.md | 6 +- docs/zh/connector-v2/sink/Hbase.md | 72 ++ docs/zh/connector-v2/sink/Http.md | 6 +- docs/zh/connector-v2/sink/Jdbc.md | 2 +- docs/zh/connector-v2/sink/Kafka.md | 87 +- docs/zh/connector-v2/sink/Paimon.md | 31 +- docs/zh/connector-v2/sink/Qdrant.md | 68 ++ docs/zh/connector-v2/sink/Typesense.md | 95 ++ docs/zh/connector-v2/source/Elasticsearch.md | 247 +++++ docs/zh/connector-v2/source/Kafka.md | 288 ++++++ docs/zh/connector-v2/source/Opengauss-CDC.md | 169 ++++ docs/zh/connector-v2/source/Qdrant.md | 79 ++ docs/zh/connector-v2/source/Sls.md | 6 +- docs/zh/connector-v2/source/Typesense.md | 79 ++ docs/zh/faq.md | 21 - .../zh/seatunnel-engine/checkpoint-storage.md | 1 - docs/zh/seatunnel-engine/rest-api.md | 155 ++- docs/zh/start-v2/locally/deployment.md | 41 +- docs/zh/start-v2/locally/quick-start-flink.md | 2 +- .../locally/quick-start-seatunnel-engine.md | 2 +- docs/zh/start-v2/locally/quick-start-spark.md | 2 +- docs/zh/transform-v2/embedding.md | 382 +++++++ docs/zh/transform-v2/llm.md | 164 +++- plugin-mapping.properties | 9 +- pom.xml | 6 +- release-note.md | 2 + .../sink/multitablesink/MultiTableSink.java | 9 +- .../multitablesink/MultiTableSinkWriter.java | 73 +- .../catalog/schema/ReadonlyConfigParser.java | 3 +- .../api/table/type/SeaTunnelRow.java | 3 + .../seatunnel/api/tracing/MDCCallable.java | 50 + .../seatunnel/api/tracing/MDCContext.java | 122 +++ .../seatunnel/api/tracing/MDCExecutor.java | 36 + .../api/tracing/MDCExecutorService.java | 121 +++ .../seatunnel/api/tracing/MDCRunnable.java | 44 + .../tracing/MDCScheduledExecutorService.java | 63 ++ .../seatunnel/api/tracing/MDCTracer.java | 126 +++ .../seatunnel/api/tracing/MDCTracerTest.java | 149 +++ .../common/exception/CommonError.java | 4 +- .../utils/TemporaryClassLoaderContext.java | 53 + .../assertion/excecutor/AssertExecutor.java | 8 + .../connector-cdc-opengauss/pom.xml | 91 ++ .../connection/PostgresConnection.java | 815 +++++++++++++++ .../PostgresReplicationConnection.java | 928 ++++++++++++++++++ .../OpengaussIncrementalSourceFactory.java | 110 +++ seatunnel-connectors-v2/connector-cdc/pom.xml | 1 + .../console/sink/ConsoleSinkWriter.java | 5 +- .../catalog/ElasticSearchCatalog.java | 3 +- .../elasticsearch/client/EsRestClient.java | 28 +- .../elasticsearch/config/SourceConfig.java | 41 +- .../ElasticsearchConnectorErrorCode.java | 6 +- .../DefaultSeaTunnelRowDeserializer.java | 4 +- .../serialize/source/ElasticsearchRecord.java | 2 + .../source/ElasticsearchSource.java | 116 ++- .../source/ElasticsearchSourceFactory.java | 8 +- .../source/ElasticsearchSourceReader.java | 32 +- .../source/ElasticsearchSourceSplit.java | 9 +- .../ElasticsearchSourceSplitEnumerator.java | 61 +- .../fake/source/FakeDataGenerator.java | 72 +- .../fake/source/FakeSourceReader.java | 10 +- .../source/FakeSourceSplitEnumerator.java | 10 +- .../fake/utils/FakeDataRandomUtils.java | 93 +- .../fake/source/FakeDataGeneratorTest.java | 55 ++ .../src/test/resources/complex.schema.conf | 2 +- .../src/test/resources/fake-data.column.conf | 97 ++ .../hbase/config/HbaseParameters.java | 53 +- .../seatunnel/hbase/sink/HbaseSink.java | 7 +- .../hbase/sink/HbaseSinkFactory.java | 7 + .../seatunnel/hbase/sink/HbaseSinkWriter.java | 12 +- .../iceberg/IcebergCatalogLoader.java | 11 +- .../iceberg/catalog/IcebergCatalog.java | 37 +- .../seatunnel/iceberg/data/RowConverter.java | 22 +- .../seatunnel/iceberg/sink/IcebergSink.java | 6 +- .../iceberg/sink/IcebergSinkWriter.java | 10 +- .../sink/writer/IcebergRecordWriter.java | 12 +- .../sink/writer/IcebergWriterFactory.java | 4 - .../iceberg/source/IcebergSource.java | 10 +- .../seatunnel/iceberg/utils/SchemaUtils.java | 43 +- .../jdbc/catalog/JdbcCatalogOptions.java | 6 +- .../jdbc/catalog/oracle/OracleCatalog.java | 25 +- .../catalog/oracle/OracleCatalogFactory.java | 4 +- .../jdbc/catalog/utils/CatalogUtils.java | 19 +- .../jdbc/config/JdbcConnectionConfig.java | 15 + .../seatunnel/jdbc/config/JdbcOptions.java | 7 + .../jdbc/config/JdbcSourceConfig.java | 5 +- .../jdbc/config/JdbcSourceTableConfig.java | 5 +- .../internal/dialect/mysql/MySqlVersion.java | 37 +- .../dialect/oracle/OracleTypeConverter.java | 24 +- .../dialect/oracle/OracleTypeMapper.java | 13 +- .../BufferReducedBatchStatementExecutor.java | 14 +- .../BufferedBatchStatementExecutor.java | 12 +- .../InsertOrUpdateBatchStatementExecutor.java | 17 +- .../jdbc/utils/JdbcCatalogUtils.java | 5 + .../dialect/mysql/MysqlVersionTest.java | 59 ++ .../oracle/OracleTypeConverterTest.java | 239 +++-- .../connector-kafka/pom.xml | 5 + .../seatunnel/kafka/config/Config.java | 14 + .../seatunnel/kafka/config/MessageFormat.java | 3 +- .../DefaultSeaTunnelRowSerializer.java | 52 +- .../seatunnel/kafka/sink/KafkaSinkWriter.java | 12 +- .../kafka/source/ConsumerMetadata.java | 1 - .../kafka/source/KafkaConsumerThread.java | 111 --- .../source/KafkaPartitionSplitReader.java | 458 +++++++++ .../kafka/source/KafkaRecordEmitter.java | 112 +++ .../seatunnel/kafka/source/KafkaSource.java | 33 +- .../kafka/source/KafkaSourceConfig.java | 21 +- .../kafka/source/KafkaSourceReader.java | 343 ++----- .../source/KafkaSourceSplitEnumerator.java | 8 +- .../kafka/source/KafkaSourceSplitState.java | 45 + .../fetch/KafkaSourceFetcherManager.java | 99 ++ .../serde/BsonToRowDataConverters.java | 2 +- .../source/split/SamplingSplitStrategy.java | 7 +- .../serde/BsonToRowDataConvertersTest.java | 55 ++ .../paimon/catalog/PaimonCatalog.java | 30 + .../exception/PaimonConnectorErrorCode.java | 3 +- .../seatunnel/paimon/utils/RowConverter.java | 63 +- .../seatunnel/paimon/utils/SchemaUtil.java | 15 +- .../paimon/utils/RowConverterTest.java | 225 +++-- .../connector-qdrant/pom.xml | 63 ++ .../seatunnel/qdrant/config/QdrantConfig.java | 50 + .../qdrant/config/QdrantParameters.java | 47 + .../exception/QdrantConnectorException.java | 36 + .../qdrant/sink/QdrantBatchWriter.java | 190 ++++ .../seatunnel/qdrant/sink/QdrantSink.java | 50 + .../qdrant/sink/QdrantSinkFactory.java | 55 ++ .../qdrant/sink/QdrantSinkWriter.java | 60 ++ .../seatunnel/qdrant/source/QdrantSource.java | 63 ++ .../qdrant/source/QdrantSourceFactory.java | 63 ++ .../qdrant/source/QdrantSourceReader.java | 181 ++++ .../rabbitmq/client/RabbitmqClient.java | 11 +- .../rabbitmq/config/RabbitmqConfig.java | 36 + .../tablestore/config/TablestoreOptions.java | 19 + .../DefaultSeaTunnelRowDeserializer.java | 38 + .../serialize/SeaTunnelRowDeserializer.java | 26 + .../tablestore/source/TableStoreDBSource.java | 102 ++ .../source/TableStoreDBSourceReader.java | 175 ++++ .../source/TableStoreDBSourceSplit.java | 38 + .../TableStoreDBSourceSplitEnumerator.java | 166 ++++ .../source/TableStoreDBSourceState.java} | 19 +- .../source/TableStoreDbSourceFactory.java | 64 ++ .../source/TableStoreProcessor.java | 95 ++ .../connector-typesense/pom.xml | 90 ++ .../typesense/catalog/TypesenseCatalog.java | 214 ++++ .../catalog/TypesenseCatalogFactory.java | 45 + .../catalog/TypesenseTypeConverter.java | 94 ++ .../typesense/client/TypesenseClient.java | 286 ++++++ .../typesense/client/TypesenseType.java | 38 + .../typesense/config/SinkConfig.java | 78 ++ .../typesense/config/SourceConfig.java | 42 + .../config/TypesenseConnectionConfig.java | 45 + .../typesense/dto/CollectionInfo.java | 40 + .../typesense/dto/SourceCollectionInfo.java | 33 + .../TypesenseConnectorErrorCode.java | 50 + .../TypesenseConnectorException.java | 32 + .../typesense/serialize/KeyExtractor.java | 98 ++ .../sink/SeaTunnelRowSerializer.java | 26 + .../sink/TypesenseRowSerializer.java | 119 +++ .../sink/collection/CollectionSerializer.java | 24 + .../FixedValueCollectionSerializer.java | 34 + .../DefaultSeaTunnelRowDeserializer.java | 253 +++++ .../source/SeaTunnelRowDeserializer.java | 25 + .../serialize/source/TypesenseRecord.java | 31 + .../typesense/sink/TypesenseSink.java | 96 ++ .../typesense/sink/TypesenseSinkFactory.java | 72 ++ .../typesense/sink/TypesenseSinkWriter.java | 139 +++ .../typesense/source/TypesenseSource.java | 87 ++ .../source/TypesenseSourceFactory.java | 59 ++ .../source/TypesenseSourceReader.java | 132 +++ .../source/TypesenseSourceSplit.java | 41 + .../TypesenseSourceSplitEnumerator.java | 187 ++++ .../source/TypesenseSourceState.java | 32 + .../state/TypesenseAggregatedCommitInfo.java | 22 + .../typesense/state/TypesenseCommitInfo.java | 22 + .../typesense/state/TypesenseSinkState.java | 8 +- .../typesense/util/URLParamsConverter.java | 71 ++ .../TypesenseRowSerializerTest.java | 64 ++ .../typesense/sink/TypesenseFactoryTest.java | 32 + .../util/URLParamsConverterTest.java | 35 + seatunnel-connectors-v2/pom.xml | 2 + .../src/main/bin/seatunnel-cluster.cmd | 15 +- .../src/main/bin/seatunnel-cluster.sh | 13 +- .../src/main/bin/seatunnel.cmd | 21 +- .../src/main/bin/seatunnel.sh | 19 +- .../src/main/resources/log4j2.properties | 4 +- seatunnel-dist/pom.xml | 13 + seatunnel-dist/release-docs/LICENSE | 3 + .../licenses/LICENSE-protoc-jar.txt | 201 ++++ .../connector-cdc-opengauss-e2e/pom.xml | 75 ++ .../cdc/postgres/OpengaussCDCIT.java | 727 ++++++++++++++ .../src/test/resources/ddl/inventory.sql | 199 ++++ .../resources/opengausscdc_to_opengauss.conf | 62 ++ ...ngausscdc_to_opengauss_test_add_Filed.conf | 62 ++ ..._to_opengauss_with_custom_primary_key.conf | 69 ++ ...gauss_with_multi_table_mode_one_table.conf | 63 ++ ...gauss_with_multi_table_mode_two_table.conf | 63 ++ ...scdc_to_opengauss_with_no_primary_key.conf | 63 ++ .../elasticsearch/ElasticsearchIT.java | 355 ++++++- ...earch_multi_source_and_sink_by_filter.conf | 92 ++ .../e2e/connector/file/s3/S3Utils.java | 14 - .../e2e/connector/hbase/HbaseIT.java | 47 +- .../fake-to-hbase-with-multipletable.conf | 86 ++ .../hbase-to-assert-with-multipletable.conf | 129 +++ .../seatunnel/jdbc/AbstractJdbcIT.java | 48 +- .../jdbc/InsecureURLClassLoader.java | 59 ++ .../connectors/seatunnel/jdbc/JdbcCase.java | 4 + .../seatunnel/jdbc/JdbcMysqlIT.java | 1 + .../seatunnel/jdbc/JdbcOracleIT.java | 26 + .../jdbc/JdbcOracleMultipleTablesIT.java | 307 ++++++ .../resources/jdbc_oracle_source_to_sink.conf | 4 +- ...dbc_oracle_source_to_sink_use_select1.conf | 4 +- ...dbc_oracle_source_to_sink_use_select2.conf | 2 +- ...dbc_oracle_source_to_sink_use_select3.conf | 2 +- ...o_sink_without_decimal_type_narrowing.conf | 82 ++ ...e_source_with_multiple_tables_to_sink.conf | 65 ++ .../seatunnel/jdbc/JdbcStarRocksdbIT.java | 2 + .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 4 +- .../seatunnel/jdbc/JdbcSqlServerIT.java | 1 + .../connectors/seatunnel/jdbc/JdbcDmIT.java | 1 + .../seatunnel/jdbc/JdbcGBase8aIT.java | 2 +- .../seatunnel/jdbc/JdbcGreenplumIT.java | 1 + .../connectors/seatunnel/jdbc/JdbcXuguIT.java | 1 + .../connector-kafka-e2e/pom.xml | 6 + .../e2e/connector/kafka/KafkaIT.java | 352 ++++++- .../kafka_multi_source_to_pg.conf | 2 +- .../protobuf/fake_to_kafka_protobuf.conf | 100 ++ .../protobuf/kafka_protobuf_to_assert.conf | 177 ++++ .../e2e/connector/paimon/PaimonRecord.java | 1 + .../e2e/connector/paimon/PaimonSinkCDCIT.java | 48 +- .../connector/paimon/PaimonSinkHdfsIT.java | 129 +++ ..._sink_paimon_truncate_with_hdfs_case1.conf | 80 ++ ..._sink_paimon_truncate_with_hdfs_case2.conf | 65 ++ ..._sink_paimon_truncate_with_hive_case1.conf | 82 ++ ..._sink_paimon_truncate_with_hive_case2.conf | 67 ++ ...sink_paimon_truncate_with_local_case1.conf | 71 ++ ...sink_paimon_truncate_with_local_case2.conf | 56 ++ .../connector-qdrant-e2e/pom.xml | 68 ++ .../e2e/connector/v2/qdrant/QdrantIT.java | 145 +++ .../src/test/resources/qdrant-to-qdrant.conf | 51 + .../e2e/connector/rabbitmq/RabbitmqIT.java | 9 + .../test/resources/rabbitmq-to-rabbitmq.conf | 6 + .../connector-typesense-e2e/pom.xml | 51 + .../e2e/connector/typesense/TypesenseIT.java | 265 +++++ .../fake_to_typesense_with_append_data.conf | 52 + ...typesense_with_create_when_not_exists.conf | 51 + .../fake_to_typesense_with_drop_data.conf | 52 + ...typesense_with_error_when_data_exists.conf | 52 + ..._typesense_with_error_when_not_exists.conf | 51 + .../fake_to_typesense_with_primary_keys.conf | 50 + ...ake_to_typesense_with_recreate_schema.conf | 51 + .../resources/typesense_source_and_sink.conf | 54 + .../resources/typesense_to_typesense.conf | 64 ++ .../typesense_to_typesense_with_query.conf | 65 ++ .../seatunnel-connector-v2-e2e/pom.xml | 3 + .../src/test/resources/log4j2.properties | 4 +- .../FakeSourceToConsoleWithEventReportIT.java | 2 +- .../connector-seatunnel-e2e-base/pom.xml | 10 + .../engine/e2e/ClusterFaultToleranceIT.java | 1 - .../engine/e2e/ClusterSeaTunnelContainer.java | 186 ++++ .../WorkerTagClusterTest.java | 161 +++ .../log4j2-test.properties | 4 +- .../log4j2.properties | 4 +- .../log4j2-test.properties | 4 +- .../log4j2.properties | 4 +- .../log4j2-test.properties | 4 +- .../log4j2.properties | 4 +- .../src/test/resources/log4j2-test.properties | 4 +- .../src/test/resources/log4j2.properties | 4 +- .../e2e/transform/TestEmbeddingIT.java | 102 ++ .../seatunnel/e2e/transform/TestLLMIT.java | 7 + .../test/resources/embedding_transform.conf | 295 ++++++ .../resources/embedding_transform_custom.conf | 152 +++ .../test/resources/llm_transform_custom.conf | 94 ++ .../src/test/resources/mock-embedding.json | 75 ++ .../engine/client/SeaTunnelClient.java | 37 +- .../src/test/resources/log4j2-test.properties | 4 +- .../seatunnel/engine/common/Constant.java | 2 + .../SeaTunnelChildFirstClassLoader.java | 2 + .../engine/server/CoordinatorService.java | 15 +- .../seatunnel/engine/server/EventService.java | 100 ++ .../server/LiteNodeDropOutTcpIpJoiner.java | 4 + .../engine/server/SeaTunnelServer.java | 10 +- .../engine/server/TaskExecutionService.java | 76 +- .../checkpoint/CheckpointCoordinator.java | 2 + .../CheckpointBarrierTriggerOperation.java | 2 +- .../operation/CheckpointEndOperation.java | 2 +- .../CheckpointErrorReportOperation.java | 2 +- .../CheckpointFinishedOperation.java | 2 +- .../operation/NotifyTaskRestoreOperation.java | 2 +- .../operation/NotifyTaskStartOperation.java | 2 +- .../operation/TaskAcknowledgeOperation.java | 9 +- .../operation/TaskReportStatusOperation.java | 9 +- ...rSchemaChangeAfterCheckpointOperation.java | 8 +- ...SchemaChangeBeforeCheckpointOperation.java | 6 +- .../dag/execution/ExecutionPlanGenerator.java | 4 +- .../engine/server/execution/Task.java | 1 + .../opeartion/ReleaseSlotOperation.java | 8 +- .../opeartion/RequestSlotOperation.java | 8 +- .../resourcemanager/worker/WorkerProfile.java | 2 + .../engine/server/rest/RestConstant.java | 4 +- .../rest/RestHttpGetCommandProcessor.java | 4 +- .../rest/RestHttpPostCommandProcessor.java | 80 +- .../service/slot/DefaultSlotService.java | 1 + .../task/SourceSplitEnumeratorTask.java | 4 + .../server/task/flow/SinkFlowLifeCycle.java | 17 +- .../server/task/flow/SourceFlowLifeCycle.java | 23 +- .../task/operation/CancelTaskOperation.java | 5 +- .../CheckTaskGroupIsExecutingOperation.java | 5 +- .../CleanTaskGroupContextOperation.java | 5 +- .../task/operation/DeployTaskOperation.java | 5 +- .../GetTaskGroupAddressOperation.java | 6 +- .../operation/NotifyTaskStatusOperation.java | 6 +- .../server/task/operation/TaskOperation.java | 5 +- .../task/operation/TracingOperation.java | 63 ++ .../checkpoint/BarrierFlowOperation.java | 2 +- .../checkpoint/CloseRequestOperation.java | 6 +- .../sink/SinkPrepareCommitOperation.java | 2 +- .../operation/sink/SinkRegisterOperation.java | 6 +- .../source/AssignSplitOperation.java | 8 +- .../source/CloseIdleReaderOperation.java | 9 +- .../source/LastCheckpointNotifyOperation.java | 9 +- .../source/RequestSplitOperation.java | 6 +- .../source/RestoredSplitOperation.java | 2 +- .../source/SourceNoMoreElementOperation.java | 7 +- .../source/SourceReaderEventOperation.java | 2 +- .../source/SourceRegisterOperation.java | 7 +- .../engine/server/utils/RestUtil.java | 21 + .../resourcemanager/FixSlotResourceTest.java | 17 +- .../src/test/resources/log4j2-test.properties | 4 +- .../storage/hdfs/HdfsStorageFactory.java | 1 - .../storage/hdfs/OssFileCheckpointTest.java | 3 - .../src/test/resources/log4j2-test.properties | 4 +- .../storage/file/IMapFileOSSStorageTest.java | 6 - .../src/main/resources/log4j2.properties | 4 +- .../pom.xml | 6 - seatunnel-formats/pom.xml | 1 + .../seatunnel-format-protobuf/pom.xml | 59 ++ .../format/protobuf/CompileDescriptor.java | 107 ++ .../ProtobufDeserializationSchema.java | 68 ++ .../protobuf/ProtobufSerializationSchema.java | 51 + .../protobuf/ProtobufToRowConverter.java | 185 ++++ .../protobuf/RowToProtobufConverter.java | 152 +++ .../exception/ProtobufFormatErrorCode.java | 43 + .../SeaTunnelProtobufFormatException.java | 29 + .../protobuf/ProtobufConverterTest.java | 155 +++ .../nlpmodel/CustomConfigPlaceholder.java | 56 ++ .../transform/nlpmodel/ModelProvider.java | 52 + .../nlpmodel/ModelTransformConfig.java | 115 +++ .../embadding/EmbeddingTransform.java | 212 ++++ .../embadding/EmbeddingTransformConfig.java | 41 + .../embadding/EmbeddingTransformFactory.java | 72 ++ .../embadding/remote/AbstractModel.java | 70 ++ .../nlpmodel/embadding/remote/Model.java | 30 + .../embadding/remote/custom/CustomModel.java | 152 +++ .../embadding/remote/doubao/DoubaoModel.java | 112 +++ .../embadding/remote/openai/OpenAIModel.java | 116 +++ .../remote/qianfan/QianfanModel.java | 173 ++++ .../{ => nlpmodel}/llm/LLMTransform.java | 62 +- .../llm/LLMTransformConfig.java | 35 +- .../llm/LLMTransformFactory.java | 26 +- .../llm/remote}/AbstractModel.java | 2 +- .../model => nlpmodel/llm/remote}/Model.java | 2 +- .../llm/remote/custom/CustomModel.java | 152 +++ .../llm/remote}/openai/OpenAIModel.java | 4 +- .../EmbeddingTransformFactoryTest.java | 32 + .../transform/LLMTransformFactoryTest.java | 2 +- .../embedding/EmbeddingRequestJsonTest.java | 143 +++ .../transform/llm/LLMRequestJsonTest.java | 53 +- .../translation/source/CoordinatedSource.java | 11 + .../translation/source/ParallelSource.java | 8 + .../translation/flink/sink/FlinkSink.java | 7 +- .../flink/sink/FlinkSinkWriter.java | 8 +- .../flink/source/FlinkSourceEnumerator.java | 6 + .../flink/source/FlinkSourceReader.java | 4 + .../spark/sink/writer/SparkDataWriter.java | 7 +- .../sink/writer/SparkDataWriterFactory.java | 2 +- .../sink/write/SeaTunnelSparkDataWriter.java | 7 +- .../SeaTunnelSparkDataWriterFactory.java | 2 +- tools/dependencies/known-dependencies.txt | 8 +- 420 files changed, 22599 insertions(+), 1644 deletions(-) create mode 100644 docs/en/connector-v2/sink/Qdrant.md create mode 100644 docs/en/connector-v2/sink/Typesense.md create mode 100644 docs/en/connector-v2/source/Opengauss-CDC.md create mode 100644 docs/en/connector-v2/source/Qdrant.md create mode 100644 docs/en/connector-v2/source/Tablestore.md create mode 100644 docs/en/connector-v2/source/Typesense.md create mode 100644 docs/en/transform-v2/embedding.md create mode 100644 docs/zh/connector-v2/sink/Qdrant.md create mode 100644 docs/zh/connector-v2/sink/Typesense.md create mode 100644 docs/zh/connector-v2/source/Elasticsearch.md create mode 100644 docs/zh/connector-v2/source/Kafka.md create mode 100644 docs/zh/connector-v2/source/Opengauss-CDC.md create mode 100644 docs/zh/connector-v2/source/Qdrant.md create mode 100644 docs/zh/connector-v2/source/Typesense.md create mode 100644 docs/zh/transform-v2/embedding.md create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java create mode 100644 seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java create mode 100644 seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java delete mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java create mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java create mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java create mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java create mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java create mode 100644 seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/pom.xml create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java rename seatunnel-connectors-v2/{connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java => connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java} (74%) create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java create mode 100644 seatunnel-connectors-v2/connector-typesense/pom.xml create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java => seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java (83%) create mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java create mode 100644 seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java create mode 100644 seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleMultipleTablesIT.java 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_without_decimal_type_narrowing.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform_custom.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_transform_custom.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mock-embedding.json create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/pom.xml create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.java create mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{ => nlpmodel}/llm/LLMTransform.java (57%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{ => nlpmodel}/llm/LLMTransformConfig.java (55%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{ => nlpmodel}/llm/LLMTransformFactory.java (67%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{llm/model => nlpmodel/llm/remote}/AbstractModel.java (98%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{llm/model => nlpmodel/llm/remote}/Model.java (94%) create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{llm/model => nlpmodel/llm/remote}/openai/OpenAIModel.java (96%) create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.java create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java diff --git a/.github/workflows/labeler/label-scope-conf.yml b/.github/workflows/labeler/label-scope-conf.yml index b0a89dfd354..b417d53e72a 100644 --- a/.github/workflows/labeler/label-scope-conf.yml +++ b/.github/workflows/labeler/label-scope-conf.yml @@ -257,6 +257,19 @@ activemq: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-activemq/** - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(activemq)/**' + +qdrant: + - all: + - changed-files: + - any-glob-to-any-file: seatunnel-connectors-v2/connector-qdrant/** + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(qdrant)/**' + +typesense: + - all: + - changed-files: + - any-glob-to-any-file: seatunnel-connectors-v2/connector-typesense/** + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(typesense)/**' + Zeta Rest API: - changed-files: - any-glob-to-any-file: seatunnel-engine/**/server/rest/** @@ -279,4 +292,4 @@ sls: - all: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-sls/** - - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(sls)/**' \ No newline at end of file + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(sls)/**' diff --git a/.github/workflows/publish-docker.yaml b/.github/workflows/publish-docker.yaml index e1041bca2c9..111aabf8511 100644 --- a/.github/workflows/publish-docker.yaml +++ b/.github/workflows/publish-docker.yaml @@ -20,8 +20,6 @@ on: push: tags: - '*' - branches: - - dev paths-ignore: - 'docs/**' - '**/*.md' @@ -72,8 +70,8 @@ jobs: MAVEN_OPTS: -Xmx4096m run: | ./mvnw -B clean install \ - -Dmaven.test.skip \ - -Dmaven.javadoc.skip \ + -Dmaven.test.skip=true \ + -Dmaven.javadoc.skip=true \ -Dlicense.skipAddThirdParty=true \ -D"docker.build.skip"=false \ -D"docker.verify.skip"=false \ diff --git a/config/log4j2.properties b/config/log4j2.properties index d7ff7f99a3f..57dfa5fdf08 100644 --- a/config/log4j2.properties +++ b/config/log4j2.properties @@ -45,7 +45,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -55,7 +55,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT @@ -67,7 +67,7 @@ appender.file.fileName = ${file_path}/${file_name}.log appender.file.filePattern = ${file_path}/${file_name}.log.%d{yyyy-MM-dd}-%i appender.file.append = true appender.file.layout.type = PatternLayout -appender.file.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.file.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.file.policies.type = Policies appender.file.policies.time.type = TimeBasedTriggeringPolicy appender.file.policies.time.modulate = true diff --git a/config/plugin_config b/config/plugin_config index e3ac0f1d046..26eb4cab4a6 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -88,4 +88,6 @@ connector-web3j connector-milvus connector-activemq connector-sls ---end-- \ No newline at end of file +connector-qdrant +connector-typesense +connector-cdc-opengauss diff --git a/docs/en/concept/sql-config.md b/docs/en/concept/sql-config.md index fe148a6f726..dd24b57032e 100644 --- a/docs/en/concept/sql-config.md +++ b/docs/en/concept/sql-config.md @@ -120,7 +120,10 @@ CREATE TABLE sink_table WITH ( INSERT INTO sink_table SELECT id, name, age, email FROM source_table; ``` -* The `SELECT FROM` part is the table name of the source-mapped table. +* The `SELECT FROM` part is the table name of the source-mapped table. If the select field has keyword([refrence](https://github.com/JSQLParser/JSqlParser/blob/master/src/main/jjtree/net/sf/jsqlparser/parser/JSqlParserCC.jjt)),you should use it like \`filedName\`. +```sql +INSERT INTO sink_table SELECT id, name, age, email,`output` FROM source_table; +``` * The `INSERT INTO` part is the table name of the target-mapped table. * Note: This syntax does **not support** specifying fields in `INSERT`, like this: `INSERT INTO sink_table (id, name, age, email) SELECT id, name, age, email FROM source_table;` diff --git a/docs/en/connector-v2/sink/Clickhouse.md b/docs/en/connector-v2/sink/Clickhouse.md index 1528a6d161b..15d92f8c5fc 100644 --- a/docs/en/connector-v2/sink/Clickhouse.md +++ b/docs/en/connector-v2/sink/Clickhouse.md @@ -24,9 +24,9 @@ Used to write data to Clickhouse. In order to use the Clickhouse connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------------------------------------| -| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------| +| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-clickhouse) | ## Data Type Mapping diff --git a/docs/en/connector-v2/sink/Hbase.md b/docs/en/connector-v2/sink/Hbase.md index 0f808f5e52c..3ceba0982d4 100644 --- a/docs/en/connector-v2/sink/Hbase.md +++ b/docs/en/connector-v2/sink/Hbase.md @@ -116,6 +116,79 @@ Hbase { all_columns = seatunnel } } + +``` + +### Multiple Table + +```hocon +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + schema = { + table = "hbase_sink_1" + fields { + name = STRING + c_string = STRING + c_double = DOUBLE + c_bigint = BIGINT + c_float = FLOAT + c_int = INT + c_smallint = SMALLINT + c_boolean = BOOLEAN + time = BIGINT + } + } + rows = [ + { + kind = INSERT + fields = ["label_1", "sink_1", 4.3, 200, 2.5, 2, 5, true, 1627529632356] + } + ] + }, + { + schema = { + table = "hbase_sink_2" + fields { + name = STRING + c_string = STRING + c_double = DOUBLE + c_bigint = BIGINT + c_float = FLOAT + c_int = INT + c_smallint = SMALLINT + c_boolean = BOOLEAN + time = BIGINT + } + } + rows = [ + { + kind = INSERT + fields = ["label_2", "sink_2", 4.3, 200, 2.5, 2, 5, true, 1627529632357] + } + ] + } + ] + } +} + +sink { + Hbase { + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "${table_name}" + rowkey_column = ["name"] + family_name { + all_columns = info + } + } +} ``` ## Writes To The Specified Column Family diff --git a/docs/en/connector-v2/sink/Http.md b/docs/en/connector-v2/sink/Http.md index d756f06aea0..75753b3bf1c 100644 --- a/docs/en/connector-v2/sink/Http.md +++ b/docs/en/connector-v2/sink/Http.md @@ -27,9 +27,9 @@ Used to launch web hooks using data. In order to use the Http connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------------------------------| -| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-http) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------| +| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-http) | ## Sink Options diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md index 99f06891a14..1ddbdd507d9 100644 --- a/docs/en/connector-v2/sink/Jdbc.md +++ b/docs/en/connector-v2/sink/Jdbc.md @@ -235,7 +235,7 @@ there are some reference value for params above. | SQL Server | com.microsoft.sqlserver.jdbc.SQLServerDriver | jdbc:sqlserver://localhost:1433 | com.microsoft.sqlserver.jdbc.SQLServerXADataSource | https://mvnrepository.com/artifact/com.microsoft.sqlserver/mssql-jdbc | | Oracle | oracle.jdbc.OracleDriver | jdbc:oracle:thin:@localhost:1521/xepdb1 | oracle.jdbc.xa.OracleXADataSource | https://mvnrepository.com/artifact/com.oracle.database.jdbc/ojdbc8 | | sqlite | org.sqlite.JDBC | jdbc:sqlite:test.db | / | https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc | -| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar | +| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar | | StarRocks | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | / | https://mvnrepository.com/artifact/mysql/mysql-connector-java | | db2 | com.ibm.db2.jcc.DB2Driver | jdbc:db2://localhost:50000/testdb | com.ibm.db2.jcc.DB2XADataSource | https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc/db2jcc4 | | saphana | com.sap.db.jdbc.Driver | jdbc:sap://localhost:39015 | / | https://mvnrepository.com/artifact/com.sap.cloud.db.jdbc/ngdbc | diff --git a/docs/en/connector-v2/sink/Kafka.md b/docs/en/connector-v2/sink/Kafka.md index 6447faa159a..9868e44f602 100644 --- a/docs/en/connector-v2/sink/Kafka.md +++ b/docs/en/connector-v2/sink/Kafka.md @@ -24,25 +24,28 @@ Write Rows to a Kafka topic. In order to use the Kafka connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Maven | -|------------|--------------------|-------------------------------------------------------------------------------------------------------------| -| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | +| Datasource | Supported Versions | Maven | +|------------|--------------------|-------------------------------------------------------------------------------------| +| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | ## Sink Options -| Name | Type | Required | Default | Description | -|----------------------|--------|----------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| topic | String | Yes | - | When the table is used as sink, the topic name is the topic to write data to. | -| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | -| kafka.config | Map | No | - | In addition to the above parameters that must be specified by the `Kafka producer` client, the user can also specify multiple non-mandatory parameters for the `producer` client, covering [all the producer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#producerconfigs). | -| semantics | String | No | NON | Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. | -| partition_key_fields | Array | No | - | Configure which fields are used as the key of the kafka message. | -| partition | Int | No | - | We can specify the partition, all messages will be sent to this partition. | -| assign_partitions | Array | No | - | We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. | -| transaction_prefix | String | No | - | If semantic is specified as EXACTLY_ONCE, the producer will write all messages in a Kafka transaction,kafka distinguishes different transactions by different transactionId. This parameter is prefix of kafka transactionId, make sure different job use different prefix. | -| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, ogg_json and avro.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | -| field_delimiter | String | No | , | Customize the field delimiter for data format. | -| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../sink-common-options.md) for details | +| Name | Type | Required | Default | Description | +|-----------------------|--------|----------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | Yes | - | When the table is used as sink, the topic name is the topic to write data to. | +| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | +| kafka.config | Map | No | - | In addition to the above parameters that must be specified by the `Kafka producer` client, the user can also specify multiple non-mandatory parameters for the `producer` client, covering [all the producer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#producerconfigs). | +| semantics | String | No | NON | Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. | +| partition_key_fields | Array | No | - | Configure which fields are used as the key of the kafka message. | +| partition | Int | No | - | We can specify the partition, all messages will be sent to this partition. | +| assign_partitions | Array | No | - | We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. | +| transaction_prefix | String | No | - | If semantic is specified as EXACTLY_ONCE, the producer will write all messages in a Kafka transaction,kafka distinguishes different transactions by different transactionId. This parameter is prefix of kafka transactionId, make sure different job use different prefix. | +| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, ogg_json and avro.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | +| field_delimiter | String | No | , | Customize the field delimiter for data format. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../sink-common-options.md) for details | +| protobuf_message_name | String | No | - | Effective when the format is set to protobuf, specifies the Message name | +| protobuf_schema | String | No | - | Effective when the format is set to protobuf, specifies the Schema definition | + ## Parameter Interpretation @@ -213,3 +216,56 @@ sink { } ``` + +### Protobuf Configuration + +Set the `format` to `protobuf` and configure the `protobuf` data structure using the `protobuf_message_name` and `protobuf_schema` parameters. + +Example Usage: + +```hocon +sink { + kafka { + topic = "test_protobuf_topic_fake_source" + bootstrap.servers = "kafkaCluster:9092" + format = protobuf + kafka.request.timeout.ms = 60000 + kafka.config = { + acks = "all" + request.timeout.ms = 60000 + buffer.memory = 33554432 + } + protobuf_message_name = Person + protobuf_schema = """ + syntax = "proto3"; + + package org.apache.seatunnel.format.protobuf; + + option java_outer_classname = "ProtobufE2E"; + + message Person { + int32 c_int32 = 1; + int64 c_int64 = 2; + float c_float = 3; + double c_double = 4; + bool c_bool = 5; + string c_string = 6; + bytes c_bytes = 7; + + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip = 4; + } + + Address address = 8; + + map attributes = 9; + + repeated string phone_numbers = 10; + } + """ + } +} +``` diff --git a/docs/en/connector-v2/sink/MongoDB.md b/docs/en/connector-v2/sink/MongoDB.md index e7e1fd7f877..3d08dea4b43 100644 --- a/docs/en/connector-v2/sink/MongoDB.md +++ b/docs/en/connector-v2/sink/MongoDB.md @@ -27,9 +27,9 @@ This document describes how to set up the MongoDB connector to run data writers In order to use the Mongodb connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|---------------------------------------------------------------------------------------------------------------| -| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-mongodb) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|---------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-mongodb) | ## Data Type Mapping diff --git a/docs/en/connector-v2/sink/Mysql.md b/docs/en/connector-v2/sink/Mysql.md index 1b579aadd9f..6151394b809 100644 --- a/docs/en/connector-v2/sink/Mysql.md +++ b/docs/en/connector-v2/sink/Mysql.md @@ -4,7 +4,7 @@ ## Support Mysql Version -- 5.5/5.6/5.7/8.0/8.4 +- 5.5/5.6/5.7/8.0/8.1/8.2/8.3/8.4 ## Support Those Engines diff --git a/docs/en/connector-v2/sink/Paimon.md b/docs/en/connector-v2/sink/Paimon.md index 6637644bd27..57ba427430e 100644 --- a/docs/en/connector-v2/sink/Paimon.md +++ b/docs/en/connector-v2/sink/Paimon.md @@ -43,7 +43,7 @@ libfb303-xxx.jar | data_save_mode | Enum | No | APPEND_DATA | The data save mode | | paimon.table.primary-keys | String | No | - | Default comma-separated list of columns (primary key) that identify a row in tables.(Notice: The partition field needs to be included in the primary key fields) | | paimon.table.partition-keys | String | No | - | Default comma-separated list of partition fields to use when creating tables. | -| paimon.table.write-props | Map | No | - | Properties passed through to paimon table initialization, [reference](https://paimon.apache.org/docs/0.8/maintenance/configurations/#coreoptions). | +| paimon.table.write-props | Map | No | - | Properties passed through to paimon table initialization, [reference](https://paimon.apache.org/docs/master/maintenance/configurations/#coreoptions). | | paimon.hadoop.conf | Map | No | - | Properties in hadoop conf | | paimon.hadoop.conf-path | String | No | - | The specified loading path for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files | diff --git a/docs/en/connector-v2/sink/Qdrant.md b/docs/en/connector-v2/sink/Qdrant.md new file mode 100644 index 00000000000..e94598d2911 --- /dev/null +++ b/docs/en/connector-v2/sink/Qdrant.md @@ -0,0 +1,70 @@ +# Qdrant + +> Qdrant Sink Connector + +## Description + +[Qdrant](https://qdrant.tech/) is a high-performance vector search engine and vector database. + +This connector can be used to write data into a Qdrant collection. + +## Data Type Mapping + +| SeaTunnel Data Type | Qdrant Data Type | +|---------------------|------------------| +| TINYINT | INTEGER | +| SMALLINT | INTEGER | +| INT | INTEGER | +| BIGINT | INTEGER | +| FLOAT | DOUBLE | +| DOUBLE | DOUBLE | +| BOOLEAN | BOOL | +| STRING | STRING | +| ARRAY | LIST | +| FLOAT_VECTOR | DENSE_VECTOR | +| BINARY_VECTOR | DENSE_VECTOR | +| FLOAT16_VECTOR | DENSE_VECTOR | +| BFLOAT16_VECTOR | DENSE_VECTOR | +| SPARSE_FLOAT_VECTOR | SPARSE_VECTOR | + +The value of the primary key column will be used as point ID in Qdrant. If no primary key is present, a random UUID will be used. + +## Options + +| name | type | required | default value | +|-----------------|--------|----------|---------------| +| collection_name | string | yes | - | +| batch_size | int | no | 64 | +| host | string | no | localhost | +| port | int | no | 6334 | +| api_key | string | no | - | +| use_tls | int | no | false | +| common-options | | no | - | + +### collection_name [string] + +The name of the Qdrant collection to read data from. + +### batch_size [int] + +The batch size of each upsert request to Qdrant. + +### host [string] + +The host name of the Qdrant instance. Defaults to "localhost". + +### port [int] + +The gRPC port of the Qdrant instance. + +### api_key [string] + +The API key to use for authentication if set. + +### use_tls [bool] + +Whether to use TLS(SSL) connection. Required if using Qdrant cloud(https). + +### common options + +Sink plugin common parameters, please refer to [Source Common Options](../sink-common-options.md) for details. diff --git a/docs/en/connector-v2/sink/Rabbitmq.md b/docs/en/connector-v2/sink/Rabbitmq.md index 489287249e0..c7963525fba 100644 --- a/docs/en/connector-v2/sink/Rabbitmq.md +++ b/docs/en/connector-v2/sink/Rabbitmq.md @@ -57,6 +57,21 @@ convenience method for setting the fields in an AMQP URI: host, port, username, the queue to write the message to +### durable [boolean] + +true: The queue will survive a server restart. +false: The queue will be deleted on server restart. + +### exclusive [boolean] + +true: The queue is used only by the current connection and will be deleted when the connection closes. +false: The queue can be used by multiple connections. + +### auto_delete [boolean] + +true: The queue will be deleted automatically when the last consumer unsubscribes. +false: The queue will not be automatically deleted. + ### schema [Config] #### fields [Config] @@ -112,6 +127,30 @@ sink { } ``` +### Example 2 + +queue with durable, exclusive, auto_delete: + +```hocon +sink { + RabbitMQ { + host = "rabbitmq-e2e" + port = 5672 + virtual_host = "/" + username = "guest" + password = "guest" + queue_name = "test1" + durable = "true" + exclusive = "false" + auto_delete = "false" + rabbitmq.config = { + requested-heartbeat = 10 + connection-timeout = 10 + } + } +} +``` + ## Changelog ### next version diff --git a/docs/en/connector-v2/sink/Typesense.md b/docs/en/connector-v2/sink/Typesense.md new file mode 100644 index 00000000000..8700d68dc77 --- /dev/null +++ b/docs/en/connector-v2/sink/Typesense.md @@ -0,0 +1,93 @@ +# Typesense + +## Description + +Outputs data to `Typesense`. + +## Key Features + +- [ ] [Exactly Once](../../concept/connector-v2-features.md) +- [x] [CDC](../../concept/connector-v2-features.md) + +## Options + +| Name | Type | Required | Default Value | +|------------------|--------|----------|------------------------------| +| hosts | array | Yes | - | +| collection | string | Yes | - | +| schema_save_mode | string | Yes | CREATE_SCHEMA_WHEN_NOT_EXIST | +| data_save_mode | string | Yes | APPEND_DATA | +| primary_keys | array | No | | +| key_delimiter | string | No | `_` | +| api_key | string | No | | +| max_retry_count | int | No | 3 | +| max_batch_size | int | No | 10 | +| common-options | | No | - | + +### hosts [array] + +The access address for Typesense, formatted as `host:port`, e.g., `["typesense-01:8108"]`. + +### collection [string] + +The name of the collection to write to, e.g., "seatunnel". + +### primary_keys [array] + +Primary key fields used to generate the document `id`. + +### key_delimiter [string] + +Sets the delimiter for composite keys (default is `_`). + +### api_key [config] + +The `api_key` for secure access to Typesense. + +### max_retry_count [int] + +The maximum number of retry attempts for batch requests. + +### max_batch_size [int] + +The maximum size of document batches. + +### common options + +Common parameters for Sink plugins. Refer to [Common Sink Options](../source-common-options.md) for more details. + +### schema_save_mode + +Choose how to handle the target-side schema before starting the synchronization task: +- `RECREATE_SCHEMA`: Creates the table if it doesn’t exist, and deletes and recreates it if it does. +- `CREATE_SCHEMA_WHEN_NOT_EXIST`: Creates the table if it doesn’t exist, skips creation if it does. +- `ERROR_WHEN_SCHEMA_NOT_EXIST`: Throws an error if the table doesn’t exist. + +### data_save_mode + +Choose how to handle existing data on the target side before starting the synchronization task: +- `DROP_DATA`: Retains the database structure but deletes the data. +- `APPEND_DATA`: Retains both the database structure and the data. +- `ERROR_WHEN_DATA_EXISTS`: Throws an error if data exists. + +## Example + +Simple example: + +```bash +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["localhost:8108"] + collection = "typesense_to_typesense_sink_with_query" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} +``` + diff --git a/docs/en/connector-v2/source/Clickhouse.md b/docs/en/connector-v2/source/Clickhouse.md index d867dbb701b..47907bd3025 100644 --- a/docs/en/connector-v2/source/Clickhouse.md +++ b/docs/en/connector-v2/source/Clickhouse.md @@ -28,9 +28,9 @@ Used to read data from Clickhouse. In order to use the Clickhouse connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------------------------------------| -| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------| +| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-clickhouse) | ## Data Type Mapping diff --git a/docs/en/connector-v2/source/Elasticsearch.md b/docs/en/connector-v2/source/Elasticsearch.md index 9fa1faa5d04..932f2c5a823 100644 --- a/docs/en/connector-v2/source/Elasticsearch.md +++ b/docs/en/connector-v2/source/Elasticsearch.md @@ -19,24 +19,27 @@ support version >= 2.x and <= 8.x. ## Options -| name | type | required | default value | -|-------------------------|---------|----------|-------------------| -| hosts | array | yes | - | -| username | string | no | - | -| password | string | no | - | -| index | string | yes | - | -| source | array | no | - | -| query | json | no | {"match_all": {}} | -| scroll_time | string | no | 1m | -| scroll_size | int | no | 100 | -| tls_verify_certificate | boolean | no | true | -| tls_verify_hostnames | boolean | no | true | -| array_column | map | no | | -| tls_keystore_path | string | no | - | -| tls_keystore_password | string | no | - | -| tls_truststore_path | string | no | - | -| tls_truststore_password | string | no | - | -| common-options | | no | - | +| name | type | required | default value | +| ----------------------- | ------- | -------- | ------------------------------------------------------------ | +| hosts | array | yes | - | +| username | string | no | - | +| password | string | no | - | +| index | string | no | If the index list does not exist, the index must be configured | +| index_list | array | no | used to define a multiple table task | +| source | array | no | - | +| query | json | no | {"match_all": {}} | +| scroll_time | string | no | 1m | +| scroll_size | int | no | 100 | +| tls_verify_certificate | boolean | no | true | +| tls_verify_hostnames | boolean | no | true | +| array_column | map | no | | +| tls_keystore_path | string | no | - | +| tls_keystore_password | string | no | - | +| tls_truststore_path | string | no | - | +| tls_truststore_password | string | no | - | +| common-options | | no | - | + + ### hosts [array] @@ -78,6 +81,10 @@ Amount of time Elasticsearch will keep the search context alive for scroll reque Maximum number of hits to be returned with each Elasticsearch scroll request. +### index_list [array] + +The `index_list` is used to define multi-index synchronization tasks. It is an array that contains the parameters required for single-table synchronization, such as `query`, `source/schema`, `scroll_size`, and `scroll_time`. It is recommended that `index_list` and `query` should not be configured at the same level simultaneously. Please refer to the upcoming multi-table synchronization example for more details. + ### tls_verify_certificate [boolean] Enable certificates validation for HTTPS endpoints @@ -108,46 +115,94 @@ Source plugin common parameters, please refer to [Source Common Options](../sour ## Examples -simple +Demo 1 + +> This case will read data from indices matching the seatunnel-* pattern based on a query. The query will only return documents containing the id, name, age, tags, and phones fields. In this example, the source field configuration is used to specify which fields should be read, and the array_column is used to indicate that tags and phones should be treated as arrays. ```hocon Elasticsearch { hosts = ["localhost:9200"] index = "seatunnel-*" - source = ["_id","name","age"] + array_column = {tags = "array",phones = "array"} + source = ["_id","name","age","tags","phones"] query = {"range":{"firstPacket":{"gte":1669225429990,"lte":1669225429990}}} } ``` -complex +Demo 2 : Multi-table synchronization + +> This example demonstrates how to read different data from ``read_index1`` and ``read_index2`` and write separately to ``read_index1_copy``,``read_index2_copy``. +> in `read_index1`,I used source to specify the fields to be read and specify which fields are array fields using the 'array_column'. ```hocon -Elasticsearch { - hosts = ["elasticsearch:9200"] - index = "st_index" - schema = { - fields { - c_map = "map" - c_array = "array" - c_string = string - c_boolean = boolean - c_tinyint = tinyint - c_smallint = smallint - c_int = int - c_bigint = bigint - c_float = float - c_double = double - c_decimal = "decimal(2, 1)" - c_bytes = bytes - c_date = date - c_timestamp = timestamp - } - } - query = {"range":{"firstPacket":{"gte":1669225429990,"lte":1669225429990}}} +source { + Elasticsearch { + hosts = ["https://elasticsearch:9200"] + username = "elastic" + password = "elasticsearch" + tls_verify_certificate = false + tls_verify_hostname = false + index_list = [ + { + index = "read_index1" + query = {"range": {"c_int": {"gte": 10, "lte": 20}}} + source = [ + c_map, + c_array, + c_string, + c_boolean, + c_tinyint, + c_smallint, + c_bigint, + c_float, + c_double, + c_decimal, + c_bytes, + c_int, + c_date, + c_timestamp] + array_column = { + c_array = "array" + } + } + { + index = "read_index2" + query = {"match_all": {}} + source = [ + c_int2, + c_date2, + c_null + ] + + } + + ] + + } +} + +transform { +} + +sink { + Elasticsearch { + hosts = ["https://elasticsearch:9200"] + username = "elastic" + password = "elasticsearch" + tls_verify_certificate = false + tls_verify_hostname = false + + index = "${table_name}_copy" + index_type = "st" + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + } } ``` -SSL (Disable certificates validation) + + +Demo 3 : SSL (Disable certificates validation) ```hocon source { @@ -161,7 +216,7 @@ source { } ``` -SSL (Disable hostname validation) +Demo 4 :SSL (Disable hostname validation) ```hocon source { @@ -175,7 +230,7 @@ source { } ``` -SSL (Enable certificates validation) +Demo 5 :SSL (Enable certificates validation) ```hocon source { @@ -196,5 +251,4 @@ source { - Add Elasticsearch Source Connector - [Feature] Support https protocol & compatible with opensearch ([3997](https://github.com/apache/seatunnel/pull/3997)) -- [Feature] Support DSL - +- [Feature] Support DSL \ No newline at end of file diff --git a/docs/en/connector-v2/source/Http.md b/docs/en/connector-v2/source/Http.md index 318b8cf00a1..9c60b4c9aa4 100644 --- a/docs/en/connector-v2/source/Http.md +++ b/docs/en/connector-v2/source/Http.md @@ -36,9 +36,9 @@ Supported DataSource Info In order to use the Http connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------------------------------| -| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-http) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------| +| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-http) | ## Source Options diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md index 31257d85b12..27b3d875580 100644 --- a/docs/en/connector-v2/source/Jdbc.md +++ b/docs/en/connector-v2/source/Jdbc.md @@ -39,7 +39,7 @@ supports query SQL and can achieve projection effect. ## Options -| name | type | required | default value | description | +| 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`. | @@ -52,6 +52,7 @@ supports query SQL and can achieve projection effect. | 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. | +| decimal_type_narrowing | Boolean | No | true | Decimal type narrowing, if true, the decimal type will be narrowed to the int or long type if without loss of precision. Only support for Oracle at now. Please refer to `decimal_type_narrowing` below | | 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. | @@ -66,6 +67,28 @@ supports query SQL and can achieve projection effect. | 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](../source-common-options.md) for details. | +### decimal_type_narrowing + +Decimal type narrowing, if true, the decimal type will be narrowed to the int or long type if without loss of precision. Only support for Oracle at now. + +eg: + +decimal_type_narrowing = true + +| Oracle | SeaTunnel | +|---------------|-----------| +| NUMBER(1, 0) | Boolean | +| NUMBER(6, 0) | INT | +| NUMBER(10, 0) | BIGINT | + +decimal_type_narrowing = false + +| Oracle | SeaTunnel | +|---------------|----------------| +| NUMBER(1, 0) | Decimal(1, 0) | +| NUMBER(6, 0) | Decimal(6, 0) | +| NUMBER(10, 0) | Decimal(10, 0) | + ## Parallel Reader The JDBC Source connector supports parallel reading of data from tables. SeaTunnel will use certain rules to split the data in the table, which will be handed over to readers for reading. The number of readers is determined by the `parallelism` option. @@ -99,7 +122,7 @@ there are some reference value for params above. | sqlserver | com.microsoft.sqlserver.jdbc.SQLServerDriver | jdbc:sqlserver://localhost:1433 | https://mvnrepository.com/artifact/com.microsoft.sqlserver/mssql-jdbc | | oracle | oracle.jdbc.OracleDriver | jdbc:oracle:thin:@localhost:1521/xepdb1 | https://mvnrepository.com/artifact/com.oracle.database.jdbc/ojdbc8 | | sqlite | org.sqlite.JDBC | jdbc:sqlite:test.db | https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc | -| gbase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar | +| gbase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar | | starrocks | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | https://mvnrepository.com/artifact/mysql/mysql-connector-java | | db2 | com.ibm.db2.jcc.DB2Driver | jdbc:db2://localhost:50000/testdb | https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc/db2jcc4 | | tablestore | com.alicloud.openservices.tablestore.jdbc.OTSDriver | "jdbc:ots:http s://myinstance.cn-hangzhou.ots.aliyuncs.com/myinstance" | https://mvnrepository.com/artifact/com.aliyun.openservices/tablestore-jdbc | diff --git a/docs/en/connector-v2/source/MongoDB-CDC.md b/docs/en/connector-v2/source/MongoDB-CDC.md index a35c1966663..301d7075738 100644 --- a/docs/en/connector-v2/source/MongoDB-CDC.md +++ b/docs/en/connector-v2/source/MongoDB-CDC.md @@ -25,9 +25,9 @@ The MongoDB CDC connector allows for reading snapshot data and incremental data In order to use the Mongodb CDC connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|-------------------------------------------------------------------------------------------------------------------| -| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-cdc-mongodb) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|-------------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-cdc-mongodb) | ## Availability Settings diff --git a/docs/en/connector-v2/source/MongoDB.md b/docs/en/connector-v2/source/MongoDB.md index 694cce53525..6605aacce5f 100644 --- a/docs/en/connector-v2/source/MongoDB.md +++ b/docs/en/connector-v2/source/MongoDB.md @@ -27,9 +27,9 @@ This document describes how to set up the MongoDB connector to run data reads ag In order to use the Mongodb connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|---------------------------------------------------------------------------------------------------------------| -| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-mongodb) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|---------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-mongodb) | ## Data Type Mapping diff --git a/docs/en/connector-v2/source/MyHours.md b/docs/en/connector-v2/source/MyHours.md index 62dffbd352f..fc2805b45c5 100644 --- a/docs/en/connector-v2/source/MyHours.md +++ b/docs/en/connector-v2/source/MyHours.md @@ -37,7 +37,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor | Datasource | Supported Versions | Dependency | |------------|--------------------|---------------------------------------------------------------------------------------------| -| My Hours | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2) | +| My Hours | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel) | ## Source Options diff --git a/docs/en/connector-v2/source/Mysql.md b/docs/en/connector-v2/source/Mysql.md index 45a6bce18d7..57f40ac7b19 100644 --- a/docs/en/connector-v2/source/Mysql.md +++ b/docs/en/connector-v2/source/Mysql.md @@ -8,7 +8,7 @@ Read external data source data through JDBC. ## Support Mysql Version -- 5.5/5.6/5.7/8.0/8.4 +- 5.5/5.6/5.7/8.0/8.1/8.2/8.3/8.4 ## Support Those Engines @@ -67,7 +67,7 @@ Read external data source data through JDBC. ## Source Options -| Name | Type | Required | Default | Description | +| Name | Type | Required | Default | Description | |--------------------------------------------|------------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/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`. | @@ -81,8 +81,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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 | Int | No | 0 | 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" | -| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| 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" | +| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/Opengauss-CDC.md b/docs/en/connector-v2/source/Opengauss-CDC.md new file mode 100644 index 00000000000..81691ea1ff4 --- /dev/null +++ b/docs/en/connector-v2/source/Opengauss-CDC.md @@ -0,0 +1,170 @@ +# Opengauss CDC + +> Opengauss CDC source connector + +## Support Those Engines + +> SeaTunnel Zeta
+> Flink
+ +## Key features + +- [ ] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +## Description + +The Opengauss CDC connector allows for reading snapshot data and incremental data from Opengauss database. This document +describes how to set up the Opengauss CDC connector to run SQL queries against Opengauss databases. + +## Using steps + +> Here are the steps to enable CDC (Change Data Capture) in Opengauss: + +1. Ensure the wal_level is set to logical, you can use SQL commands to modify the configuration directly: + +```sql +ALTER SYSTEM SET wal_level TO 'logical'; +SELECT pg_reload_conf(); +``` + +2. Change the REPLICA policy of the specified table to FULL + +```sql +ALTER TABLE your_table_name REPLICA IDENTITY FULL; +``` + +If you have multi tables,you can use the result of this sql to change the REPLICA policy of all tables to FULL + +```sql +select 'ALTER TABLE ' || schemaname || '.' || tablename || ' REPLICA IDENTITY FULL;' from pg_tables where schemaname = 'YourTableSchema' +``` + +## Data Type Mapping + +| Opengauss Data type | SeaTunnel Data type | +|-----------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------| +| BOOL
| BOOLEAN | +| BYTEA
| BYTES | +| INT2
SMALLSERIAL
INT4
SERIAL
| INT | +| INT8
BIGSERIAL
| BIGINT | +| FLOAT4
| FLOAT | +| FLOAT8
| DOUBLE | +| NUMERIC(Get the designated column's specified column size>0) | DECIMAL(Get the designated column's specified column size,Gets the number of digits in the specified column to the right of the decimal point) | +| NUMERIC(Get the designated column's specified column size<0) | DECIMAL(38, 18) | +| BPCHAR
CHARACTER
VARCHAR
TEXT
GEOMETRY
GEOGRAPHY
JSON
JSONB | STRING | +| TIMESTAMP
| TIMESTAMP | +| TIME
| TIME | +| DATE
| DATE | +| OTHER DATA TYPES | NOT SUPPORTED YET | + +## Source Options + +| Name | Type | Required | Default | Description | +|------------------------------------------------|----------|----------|----------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| base-url | String | Yes | - | The URL of the JDBC connection. Refer to a case: `jdbc:postgresql://localhost:5432/postgres_cdc?loggerLevel=OFF`. | +| username | String | Yes | - | Username of the database to use when connecting to the database server. | +| password | String | Yes | - | Password to use when connecting to the database server. | +| database-names | List | No | - | Database name of the database to monitor. | +| table-names | List | Yes | - | Table name of the database to monitor. The table name needs to include the database name, for example: `database_name.table_name` | +| table-names-config | List | No | - | Table config list. for example: [{"table": "db1.schema1.table1","primaryKeys":["key1"]}] | +| startup.mode | Enum | No | INITIAL | Optional startup mode for Opengauss CDC consumer, valid enumerations are `initial`, `earliest`, `latest` and `specific`.
`initial`: Synchronize historical data at startup, and then synchronize incremental data.
`earliest`: Startup from the earliest offset possible.
`latest`: Startup from the latest offset.
`specific`: Startup from user-supplied specific offsets. | +| snapshot.split.size | Integer | No | 8096 | The split size (number of rows) of table snapshot, captured tables are split into multiple splits when read the snapshot of table. | +| snapshot.fetch.size | Integer | No | 1024 | The maximum fetch size for per poll when read table snapshot. | +| slot.name | String | No | - | The name of the Opengauss logical decoding slot that was created for streaming changes from a particular plug-in for a particular database/schema. The server uses this slot to stream events to the connector that you are configuring. Default is seatunnel. | +| decoding.plugin.name | String | No | pgoutput | The name of the Postgres logical decoding plug-in installed on the server,Supported values are decoderbufs, wal2json, wal2json_rds, wal2json_streaming,wal2json_rds_streaming and pgoutput. | +| server-time-zone | String | No | UTC | The session time zone in database server. If not set, then ZoneId.systemDefault() is used to determine the server time zone. | +| connect.timeout.ms | Duration | No | 30000 | The maximum time that the connector should wait after trying to connect to the database server before timing out. | +| connect.max-retries | Integer | No | 3 | The max retry times that the connector should retry to build database server connection. | +| connection.pool.size | Integer | No | 20 | The jdbc connection pool size. | +| chunk-key.even-distribution.factor.upper-bound | Double | No | 100 | 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. | +| chunk-key.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | +| 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. | +| format | Enum | No | DEFAULT | Optional output format for Opengauss 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/postgresql.adoc#connector-configuration-properties) to Debezium Embedded Engine which is used to capture data changes from Opengauss server. | +| common-options | | no | - | Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details | + +## Task Example + +### Simple + +> Support multi-table reading + +``` + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1","opengauss_cdc.inventory.opengauss_cdc_table_2"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" + decoding.plugin.name = "pgoutput" + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = "opengauss_cdc" + schema = "inventory" + tablePrefix = "sink_" + primary_keys = ["id"] + } +} + +``` + +### Support custom primary key for table + +``` +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + exactly_once = true + table-names-config = [ + { + table = "opengauss_cdc.inventory.full_types_no_primary_key" + primaryKeys = ["id"] + } + ] + } +} +``` + diff --git a/docs/en/connector-v2/source/Oracle.md b/docs/en/connector-v2/source/Oracle.md index b7a28225206..1756a76db1d 100644 --- a/docs/en/connector-v2/source/Oracle.md +++ b/docs/en/connector-v2/source/Oracle.md @@ -81,7 +81,7 @@ Read external data source data through JDBC. | 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 toimprove 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. | -| Name | Type | Required | Default | Description | +| Name | Type | Required | Default | Description | |--------------------------------------------|------------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/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`. | @@ -95,8 +95,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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 | Int | No | 0 | 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" | -| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| 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" | +| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/PostgreSQL.md b/docs/en/connector-v2/source/PostgreSQL.md index b6e95c8ad7d..101902d3618 100644 --- a/docs/en/connector-v2/source/PostgreSQL.md +++ b/docs/en/connector-v2/source/PostgreSQL.md @@ -74,24 +74,9 @@ Read external data source data through JDBC. ## Options -| Name | Type | Required | Default | Description | -|------------------------------|------------|----------|-----------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost:5432/test | -| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use PostgreSQL the value is `org.postgresql.Driver`. | -| user | String | No | - | Connection instance user name | -| password | String | No | - | Connection instance password | -| query | String | Yes | - | Query statement | -| 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 parallelism's partition, only support numeric type,Only support numeric type primary key, and only can config one column. | -| partition_lower_bound | BigDecimal | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | -| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. | -| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | -| 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 toimprove 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. | - | Name | Type | Required | Default | Description | |--------------------------------------------|------------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/test | +| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost:5432/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 | - | Connection instance user name | | password | String | No | - | Connection instance password | @@ -103,8 +88,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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 | Int | No | 0 | 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" | -| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| 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" | +| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/Qdrant.md b/docs/en/connector-v2/source/Qdrant.md new file mode 100644 index 00000000000..c523cf1da69 --- /dev/null +++ b/docs/en/connector-v2/source/Qdrant.md @@ -0,0 +1,81 @@ +# Qdrant + +> Qdrant source connector + +## Description + +[Qdrant](https://qdrant.tech/) is a high-performance vector search engine and vector database. + +This connector can be used to read data from a Qdrant collection. + +## Options + +| name | type | required | default value | +|-----------------|--------|----------|---------------| +| collection_name | string | yes | - | +| schema | config | yes | - | +| host | string | no | localhost | +| port | int | no | 6334 | +| api_key | string | no | - | +| use_tls | int | no | false | +| common-options | | no | - | + +### collection_name [string] + +The name of the Qdrant collection to read data from. + +### schema [config] + +The schema of the table to read data into. + +Eg: + +```hocon +schema = { + fields { + age = int + address = string + some_vector = float_vector + } +} +``` + +Each entry in Qdrant is called a point. + +The `float_vector` type columns are read from the vectors of each point, others are read from the JSON payload associated with the point. + +If a column is marked as primary key, the ID of the Qdrant point is written into it. It can be of type `"string"` or `"int"`. Since Qdrant only [allows](https://qdrant.tech/documentation/concepts/points/#point-ids) positive integers and UUIDs as point IDs. + +If the collection was created with a single default/unnamed vector, use `default_vector` as the vector name. + +```hocon +schema = { + fields { + age = int + address = string + default_vector = float_vector + } +} +``` + +The ID of the point in Qdrant will be written into the column which is marked as the primary key. It can be of type `int` or `string`. + +### host [string] + +The host name of the Qdrant instance. Defaults to "localhost". + +### port [int] + +The gRPC port of the Qdrant instance. + +### api_key [string] + +The API key to use for authentication if set. + +### use_tls [bool] + +Whether to use TLS(SSL) connection. Required if using Qdrant cloud(https). + +### common options + +Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details. diff --git a/docs/en/connector-v2/source/Sls.md b/docs/en/connector-v2/source/Sls.md index 6468f397ab7..609676e8ff9 100644 --- a/docs/en/connector-v2/source/Sls.md +++ b/docs/en/connector-v2/source/Sls.md @@ -26,9 +26,9 @@ Source connector for Aliyun Sls. In order to use the Sls connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Maven | -|------------|--------------------|-----------------------------------------------------------------------------------------------------------| -| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-sls) | +| Datasource | Supported Versions | Maven | +|------------|--------------------|-----------------------------------------------------------------------------------| +| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-sls) | ## Source Options diff --git a/docs/en/connector-v2/source/SqlServer.md b/docs/en/connector-v2/source/SqlServer.md index bbcd6751a21..2905f45fbdb 100644 --- a/docs/en/connector-v2/source/SqlServer.md +++ b/docs/en/connector-v2/source/SqlServer.md @@ -67,7 +67,7 @@ Read external data source data through JDBC. ## Source Options -| name | type | required | default | Description | +| name | type | required | default | Description | |--------------------------------------------|--------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:sqlserver://127.0.0.1:1434;database=TestDB | | driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use SQLserver the value is `com.microsoft.sqlserver.jdbc.SQLServerDriver`. | @@ -81,8 +81,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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 | Int | No | 0 | 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" | -| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| 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" | +| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/Tablestore.md b/docs/en/connector-v2/source/Tablestore.md new file mode 100644 index 00000000000..8e0d1aeebc7 --- /dev/null +++ b/docs/en/connector-v2/source/Tablestore.md @@ -0,0 +1,102 @@ +# Tablestore + +> Tablestore source connector + +## Description + +Read data from Alicloud Tablestore,support full and CDC. + + +## Key features + +- [ ] [batch](../../concept/connector-v2-features.md) +- [X] [stream](../../concept/connector-v2-features.md) +- [ ] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [ ] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) + +## Options + +| name | type | required | default value | +|-----------------------|--------|----------|---------------| +| end_point | string | yes | - | +| instance_name | string | yes | - | +| access_key_id | string | yes | - | +| access_key_secret | string | yes | - | +| table | string | yes | - | +| primary_keys | array | yes | - | +| schema | config | yes | - | + + +### end_point [string] + +The endpoint of Tablestore. + +### instance_name [string] + +The intance name of Tablestore. + +### access_key_id [string] + +The access id of Tablestore. + +### access_key_secret [string] + +The access secret of Tablestore. + +### table [string] + +The table name of Tablestore. + +### primary_keys [array] + +The primarky key of table,just add a unique primary key. + +### schema [Config] + + + +## Example + +```bash +env { + parallelism = 1 + job.mode = "STREAMING" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Tablestore { + end_point = "https://****.cn-zhangjiakou.tablestore.aliyuncs.com" + instance_name = "****" + access_key_id="***************2Ag5" + access_key_secret="***********2Dok" + table="test" + primary_keys=["id"] + schema={ + fields { + id = string + name = string + } + } + } +} + + +sink { + MongoDB{ + uri = "mongodb://localhost:27017" + database = "test" + collection = "test" + primary-key = ["id"] + schema = { + fields { + id = string + name = string + } + } + } +} +``` + diff --git a/docs/en/connector-v2/source/Typesense.md b/docs/en/connector-v2/source/Typesense.md new file mode 100644 index 00000000000..9f88a58d4a5 --- /dev/null +++ b/docs/en/connector-v2/source/Typesense.md @@ -0,0 +1,79 @@ +# Typesense + +> Typesense Source Connector + +## Description + +Reads data from Typesense. + +## Key Features + +- [x] [Batch Processing](../../concept/connector-v2-features.md) +- [ ] [Stream Processing](../../concept/connector-v2-features.md) +- [ ] [Exactly-Once](../../concept/connector-v2-features.md) +- [x] [Schema](../../concept/connector-v2-features.md) +- [x] [Parallelism](../../concept/connector-v2-features.md) +- [ ] [User-Defined Splits Support](../../concept/connector-v2-features.md) + +## Options + +| Name | Type | Required | Default | +|------------|--------|----------|---------| +| hosts | array | yes | - | +| collection | string | yes | - | +| schema | config | yes | - | +| api_key | string | no | - | +| query | string | no | - | +| batch_size | int | no | 100 | + +### hosts [array] + +The access address of Typesense, for example: `["typesense-01:8108"]`. + +### collection [string] + +The name of the collection to write to, for example: `"seatunnel"`. + +### schema [config] + +The columns to be read from Typesense. For more information, please refer to the [guide](../../concept/schema-feature.md#how-to-declare-type-supported). + +### api_key [config] + +The `api_key` for Typesense security authentication. + +### batch_size + +The number of records to query per batch when reading data. + +### Common Options + +For common parameters of Source plugins, please refer to [Source Common Options](../source-common-options.md). + +## Example + +```bash +source { + Typesense { + hosts = ["localhost:8108"] + collection = "companies" + api_key = "xyz" + query = "q=*&filter_by=num_employees:>9000" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} +``` + diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index 42106fc7d5f..e9259fae484 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -26,30 +26,32 @@ Source connector for Apache Kafka. In order to use the Kafka connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Maven | -|------------|--------------------|-------------------------------------------------------------------------------------------------------------| -| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | +| Datasource | Supported Versions | Maven | +|------------|--------------------|-------------------------------------------------------------------------------------| +| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | ## Source Options -| Name | Type | Required | Default | Description | -|-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| topic | String | Yes | - | Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by comma like 'topic-1,topic-2'. | -| table_list | Map | No | - | Topic list config You can configure only one `table_list` and one `topic` at the same time | -| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | -| pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | -| consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | -| commit_on_checkpoint | Boolean | No | true | If true the consumer's offset will be periodically committed in the background. | -| kafka.config | Map | No | - | In addition to the above necessary parameters that must be specified by the `Kafka consumer` client, users can also specify multiple `consumer` client non-mandatory parameters, covering [all consumer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#consumerconfigs). | -| schema | Config | No | - | The structure of the data, including field names and field types. | -| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, ogg_json and avro.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | -| format_error_handle_way | String | No | fail | The processing method of data format error. The default value is fail, and the optional value is (fail, skip). When fail is selected, data format error will block and an exception will be thrown. When skip is selected, data format error will skip this line data. | -| field_delimiter | String | No | , | Customize the field delimiter for data format. | -| start_mode | StartMode[earliest],[group_offsets],[latest],[specific_offsets],[timestamp] | No | group_offsets | The initial consumption pattern of consumers. | -| start_mode.offsets | Config | No | - | The offset required for consumption mode to be specific_offsets. | -| start_mode.timestamp | Long | No | - | The time required for consumption mode to be "timestamp". | -| partition-discovery.interval-millis | Long | No | -1 | The interval for dynamically discovering topics and partitions. | -| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details | +| Name | Type | Required | Default | Description | +|-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | Yes | - | Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by comma like 'topic-1,topic-2'. | +| table_list | Map | No | - | Topic list config You can configure only one `table_list` and one `topic` at the same time | +| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | +| pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | +| consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | +| commit_on_checkpoint | Boolean | No | true | If true the consumer's offset will be periodically committed in the background. | +| kafka.config | Map | No | - | In addition to the above necessary parameters that must be specified by the `Kafka consumer` client, users can also specify multiple `consumer` client non-mandatory parameters, covering [all consumer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#consumerconfigs). | +| schema | Config | No | - | The structure of the data, including field names and field types. | +| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, maxwell_json, ogg_json, avro and protobuf. If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. Some format details please refer [formats](../formats) | +| format_error_handle_way | String | No | fail | The processing method of data format error. The default value is fail, and the optional value is (fail, skip). When fail is selected, data format error will block and an exception will be thrown. When skip is selected, data format error will skip this line data. | +| field_delimiter | String | No | , | Customize the field delimiter for data format. | +| start_mode | StartMode[earliest],[group_offsets],[latest],[specific_offsets],[timestamp] | No | group_offsets | The initial consumption pattern of consumers. | +| start_mode.offsets | Config | No | - | The offset required for consumption mode to be specific_offsets. | +| start_mode.timestamp | Long | No | - | The time required for consumption mode to be "timestamp". | +| partition-discovery.interval-millis | Long | No | -1 | The interval for dynamically discovering topics and partitions. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details | +| protobuf_message_name | String | No | - | Effective when the format is set to protobuf, specifies the Message name | +| protobuf_schema | String | No | - | Effective when the format is set to protobuf, specifies the Schema definition | ## Task Example @@ -242,3 +244,51 @@ sink { } ``` +### Protobuf configuration + +Set `format` to `protobuf`, configure `protobuf` data structure, `protobuf_message_name` and `protobuf_schema` parameters + +Example: + +```hocon +source { + Kafka { + topic = "test_protobuf_topic_fake_source" + format = protobuf + protobuf_message_name = Person + protobuf_schema = """ + syntax = "proto3"; + + package org.apache.seatunnel.format.protobuf; + + option java_outer_classname = "ProtobufE2E"; + + message Person { + int32 c_int32 = 1; + int64 c_int64 = 2; + float c_float = 3; + double c_double = 4; + bool c_bool = 5; + string c_string = 6; + bytes c_bytes = 7; + + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip = 4; + } + + Address address = 8; + + map attributes = 9; + + repeated string phone_numbers = 10; + } + """ + bootstrap.servers = "kafkaCluster:9092" + start_mode = "earliest" + result_table_name = "kafka_table" + } +} +``` \ No newline at end of file diff --git a/docs/en/faq.md b/docs/en/faq.md index 2e50c9d4618..02c125ad4fd 100644 --- a/docs/en/faq.md +++ b/docs/en/faq.md @@ -203,23 +203,6 @@ spark { } ``` -## How do I specify a different JDK version for SeaTunnel on YARN? - -For example, if you want to set the JDK version to JDK8, there are two cases: - -- The YARN cluster has deployed JDK8, but the default JDK is not JDK8. Add two configurations to the SeaTunnel config file: - - ``` - env { - ... - spark.executorEnv.JAVA_HOME="/your/java_8_home/directory" - spark.yarn.appMasterEnv.JAVA_HOME="/your/java_8_home/directory" - ... - } - ``` -- YARN cluster does not deploy JDK8. At this time, start SeaTunnel attached with JDK8. For detailed operations, see: - https://www.cnblogs.com/jasondan/p/spark-specific-jdk-version.html - ## What should I do if OOM always appears when running SeaTunnel in Spark local[*] mode? If you run in local mode, you need to modify the `start-seatunnel.sh` startup script. After `spark-submit`, add a parameter `--driver-memory 4g` . Under normal circumstances, local mode is not used in the production environment. Therefore, this parameter generally does not need to be set during On YARN. See: [Application Properties](https://spark.apache.org/docs/latest/configuration.html#application-properties) for details. @@ -334,10 +317,6 @@ spark-submit --verbose ... ``` -## How do I use SeaTunnel to synchronize data across HDFS clusters? - -Just configure hdfs-site.xml properly. Refer to: https://www.cnblogs.com/suanec/p/7828139.html. - ## I want to learn the source code of SeaTunnel. Where should I start? SeaTunnel has a completely abstract and structured code implementation, and many people have chosen SeaTunnel As a way to learn Spark. You can learn the source code from the main program entry: SeaTunnel.java diff --git a/docs/en/seatunnel-engine/checkpoint-storage.md b/docs/en/seatunnel-engine/checkpoint-storage.md index f5dd44e3af6..7027f8067fb 100644 --- a/docs/en/seatunnel-engine/checkpoint-storage.md +++ b/docs/en/seatunnel-engine/checkpoint-storage.md @@ -67,7 +67,6 @@ seatunnel: fs.oss.accessKeyId: your-access-key fs.oss.accessKeySecret: your-secret-key fs.oss.endpoint: endpoint address - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` For additional reading on the Hadoop Credential Provider API, you can see: [Credential Provider API](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/CredentialProviderAPI.html). diff --git a/docs/en/seatunnel-engine/rest-api.md b/docs/en/seatunnel-engine/rest-api.md index 8b6cdb231b5..c2c3e06b79c 100644 --- a/docs/en/seatunnel-engine/rest-api.md +++ b/docs/en/seatunnel-engine/rest-api.md @@ -389,6 +389,106 @@ When we can't get the job info, the response will be: ------------------------------------------------------------------------------------------ +### Batch Submit Jobs + +
+POST /hazelcast/rest/maps/submit-jobs (Returns jobId and jobName if the job is successfully submitted.) + +#### Parameters (add in the `params` field in the request body) + +> | Parameter Name | Required | Type | Description | +> |----------------------|--------------|---------|---------------------------------------| +> | jobId | optional | string | job id | +> | jobName | optional | string | job name | +> | isStartWithSavePoint | optional | string | if the job is started with save point | + +#### Request Body + +```json +[ + { + "params":{ + "jobId":"123456", + "jobName":"SeaTunnel-01" + }, + "env": { + "job.mode": "batch" + }, + "source": [ + { + "plugin_name": "FakeSource", + "result_table_name": "fake", + "row.num": 1000, + "schema": { + "fields": { + "name": "string", + "age": "int", + "card": "int" + } + } + } + ], + "transform": [ + ], + "sink": [ + { + "plugin_name": "Console", + "source_table_name": ["fake"] + } + ] + }, + { + "params":{ + "jobId":"1234567", + "jobName":"SeaTunnel-02" + }, + "env": { + "job.mode": "batch" + }, + "source": [ + { + "plugin_name": "FakeSource", + "result_table_name": "fake", + "row.num": 1000, + "schema": { + "fields": { + "name": "string", + "age": "int", + "card": "int" + } + } + } + ], + "transform": [ + ], + "sink": [ + { + "plugin_name": "Console", + "source_table_name": ["fake"] + } + ] + } +] +``` + +#### Response + +```json +[ + { + "jobId": "123456", + "jobName": "SeaTunnel-01" + },{ + "jobId": "1234567", + "jobName": "SeaTunnel-02" + } +] +``` + +
+ +------------------------------------------------------------------------------------------ + ### Stop A Job
@@ -414,7 +514,42 @@ When we can't get the job info, the response will be:
------------------------------------------------------------------------------------------ +### Batch Stop Jobs + +
+POST /hazelcast/rest/maps/stop-jobs (Returns jobId if the job is successfully stopped.) + +#### Request Body + +```json +[ + { + "jobId": 881432421482889220, + "isStopWithSavePoint": false + }, + { + "jobId": 881432456517910529, + "isStopWithSavePoint": false + } +] +``` + +#### Response + +```json +[ + { + "jobId": 881432421482889220 + }, + { + "jobId": 881432456517910529 + } +] +``` + +
+------------------------------------------------------------------------------------------ ### Encrypt Config
diff --git a/docs/en/start-v2/docker/docker.md b/docs/en/start-v2/docker/docker.md index 5c2c8422224..67f57c590ae 100644 --- a/docs/en/start-v2/docker/docker.md +++ b/docs/en/start-v2/docker/docker.md @@ -2,7 +2,7 @@ sidebar_position: 3 --- -# Set Up with Docker in local mode +# Set Up With Docker In Local Mode ## Zeta Engine @@ -15,17 +15,53 @@ docker pull apache/seatunnel: How to submit job in local mode ```shell -docker run --rm -it apache/seatunnel bash ./bin/seatunnel.sh -e local -c +# Run fake source to console sink +docker run --rm -it apache/seatunnel: ./bin/seatunnel.sh -m local -c config/v2.batch.config.template +# Run job with custom config file +docker run --rm -it -v //:/config apache/seatunnel: ./bin/seatunnel.sh -m local -c /config/fake_to_console.conf -# eg: a fake source to console sink -docker run --rm -it apache/seatunnel bash ./bin/seatunnel.sh -e local -c config/v2.batch.config.template +# Example +# If you config file is in /tmp/job/fake_to_console.conf +docker run --rm -it -v /tmp/job/:/config apache/seatunnel: ./bin/seatunnel.sh -m local -c /config/fake_to_console.conf +# Set JVM options when running +docker run --rm -it -v /tmp/job/:/config apache/seatunnel: ./bin/seatunnel.sh -DJvmOption="-Xms4G -Xmx4G" -m local -c /config/fake_to_console.conf ``` ### Build Image By Yourself -```Dockerfile +Build from source code. The way of downloading the source code is the same as the way of downloading the binary package. +You can download the source code from the [download page](https://seatunnel.apache.org/download/) or clone the source code from the [GitHub repository](https://github.com/apache/seatunnel/releases) + +#### Build With One Command +```shell +cd seatunnel +# Use already sett maven profile +sh ./mvnw -B clean install -Dmaven.test.skip=true -Dmaven.javadoc.skip=true -Dlicense.skipAddThirdParty=true -D"docker.build.skip"=false -D"docker.verify.skip"=false -D"docker.push.skip"=true -D"docker.tag"=2.3.8 -Dmaven.deploy.skip --no-snapshot-updates -Pdocker,seatunnel + +# Check the docker image +docker images | grep apache/seatunnel +``` + +#### Build Step By Step +```shell +# Build binary package from source code +sh ./mvnw clean package -DskipTests -Dskip.spotless=true + +# Build docker image +cd seatunnel-dist +docker build -f src/main/docker/Dockerfile --build-arg VERSION=2.3.8 -t apache/seatunnel:2.3.8 . + +# If you build from dev branch, you should add SNAPSHOT suffix to the version +docker build -f src/main/docker/Dockerfile --build-arg VERSION=2.3.8-SNAPSHOT -t apache/seatunnel:2.3.8-SNAPSHOT . + +# Check the docker image +docker images | grep apache/seatunnel +``` + +The Dockerfile is like this: +```dockerfile FROM openjdk:8 ARG VERSION @@ -64,7 +100,7 @@ docker run \ Or you can change the `SPARK_HOME`, `FLINK_HOME` environment variable in Dockerfile and re-build your and mount the spark/flink to related path. -```Dockerfile +```dockerfile FROM apache/seatunnel ENV SPARK_HOME= diff --git a/docs/en/start-v2/locally/deployment.md b/docs/en/start-v2/locally/deployment.md index 7940001b7f2..db5dbdd0fd1 100644 --- a/docs/en/start-v2/locally/deployment.md +++ b/docs/en/start-v2/locally/deployment.md @@ -5,15 +5,17 @@ sidebar_position: 2 import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -# Download and Make Installation Packages +# Deployment -## Step 1: Preparation +## Preparation Before starting to download SeaTunnel, you need to ensure that you have installed the following software required by SeaTunnel: * Install [Java](https://www.java.com/en/download/) (Java 8 or 11, and other versions higher than Java 8 can theoretically work) and set `JAVA_HOME`. -## Step 2: Download SeaTunnel +## Download SeaTunnel Release Package + +### Download The Binary Package Visit the [SeaTunnel Download Page](https://seatunnel.apache.org/download) to download the latest binary package `seatunnel--bin.tar.gz`. @@ -25,7 +27,7 @@ wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${ve tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` -## Step 3: Download The Connector Plugins +### Download The Connector Plugins Starting from version 2.2.0-beta, the binary package no longer provides connector dependencies by default. Therefore, the first time you use it, you need to run the following command to install the connectors (Alternatively, you can manually download the connectors from the [Apache Maven Repository](https://repo.maven.apache.org/maven2/org/apache/seatunnel/) and move them to the `connectors/` directory. For versions before 2.3.5, place them in the `connectors/seatunnel` directory): @@ -56,10 +58,33 @@ If you want to install connector plugins by manually downloading connectors, you ::: +## Build SeaTunnel From Source Code + +### Download The Source Code + +Build from source code. The way of downloading the source code is the same as the way of downloading the binary package. +You can download the source code from the [download page](https://seatunnel.apache.org/download/) or clone the source code from the [GitHub repository](https://github.com/apache/seatunnel/releases) + +### Build The Source Code + +```shell +cd seatunnel +sh ./mvnw clean package -DskipTests -Dskip.spotless=true +# get the binary package +cp seatunnel-dist/target/apache-seatunnel-2.3.8-bin.tar.gz /The-Path-You-Want-To-Copy + +cd /The-Path-You-Want-To-Copy +tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" +``` + +When built from the source code, all the connector plugins and some necessary dependencies (eg: mysql driver) are included in the binary package. You can directly use the connector plugins without the need to install them separately. + +# Run SeaTunnel + Now you have downloaded the SeaTunnel binary package and the connector plugins. Next, you can choose different engine option to run synchronization tasks. -If you use Flink to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start of SeaTunnel Flink Engine](quick-start-flink.md) to run your synchronization task. +If you use Flink to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start With Flink](quick-start-flink.md) to run your synchronization task. -If you use Spark to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start of SeaTunnel Spark Engine](quick-start-spark.md) to run your synchronization task. +If you use Spark to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start With Spark](quick-start-spark.md) to run your synchronization task. -If you use the builtin SeaTunnel Engine (Zeta) to run tasks, you need to deploy the SeaTunnel Engine service first. Refer to [Deployment of SeaTunnel Engine (Zeta) Service](quick-start-seatunnel-engine.md). +If you use the builtin SeaTunnel Engine (Zeta) to run tasks, you need to deploy the SeaTunnel Engine service first. Refer to [Quick Start With SeaTunnel Engine](quick-start-seatunnel-engine.md). diff --git a/docs/en/transform-v2/embedding.md b/docs/en/transform-v2/embedding.md new file mode 100644 index 00000000000..046f72789ac --- /dev/null +++ b/docs/en/transform-v2/embedding.md @@ -0,0 +1,392 @@ +# Embedding + +> Embedding Transform Plugin + +## Description + +The `Embedding` transform plugin leverages embedding models to convert text data into vectorized representations. This +transformation can be applied to various fields. The plugin supports multiple model providers and can be integrated with +different API endpoints. + +## Options + +| Name | Type | Required | Default Value | Description | +|--------------------------------|--------|----------|---------------|-------------------------------------------------------------------------------------------------------------| +| model_provider | enum | yes | - | The model provider for embedding. Options may include `QIANFAN`, `OPENAI`, etc. | +| api_key | string | yes | - | The API key required to authenticate with the embedding service. | +| secret_key | string | yes | - | The secret key required for additional authentication with the embedding service. | +| single_vectorized_input_number | int | no | 1 | The number of inputs vectorized in one request. Default is 1. | +| vectorization_fields | map | yes | - | A mapping between input fields and their corresponding output vector fields. | +| model | string | yes | - | The specific model to use for embedding (e.g: `text-embedding-3-small` for OPENAI). | +| api_path | string | no | - | The API endpoint for the embedding service. Typically provided by the model provider. | +| oauth_path | string | no | - | The API endpoint for the oauth service. | +| custom_config | map | no | | Custom configurations for the model. | +| custom_response_parse | string | no | | Specifies how to parse the response from the model using JsonPath. Example: `$.choices[*].message.content`. | +| custom_request_headers | map | no | | Custom headers for the request to the model. | +| custom_request_body | map | no | | Custom body for the request. Supports placeholders like `${model}`, `${input}`. | + +### model_provider + +The providers for generating embeddings include common options such as `DOUBAO`, `QIANFAN`, and `OPENAI`. Additionally, +you can choose `CUSTOM` to implement requests and retrievals for custom embedding models. + +### api_key + +The API key for authenticating requests to the embedding service. This is typically provided by the model provider when +you register for their service. + +### secret_key + +The secret key used for additional authentication. Some providers may require this for secure API requests. + +### single_vectorized_input_number + +Specifies how many inputs are processed in a single vectorization request. The default is 1. Adjust based on your +processing +capacity and the model provider's API limitations. + +### vectorization_fields + +A mapping between input fields and their respective output vector fields. This allows the plugin to understand which +text fields to vectorize and how to store the resulting vectors. + +```hocon +vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography +} +``` + +### model + +The specific embedding model to use. This depends on the `embedding_model_provider`. For example, if using OPENAI, you +might specify `text-embedding-3-small`. + +### api_path + +The API endpoint to use for making requests to the embedding service. This might vary based on the provider and model +used. Generally, this is provided by the model provider. + +### oauth_path + +The API endpoint for the oauth service. Get certification information. This might vary based on the provider and model +used. Generally, this is provided by the model provider. + +### custom_config + +The `custom_config` option allows you to provide additional custom configurations for the model. This is a map where you +can define various settings that might be required by the specific model you're using. + +### custom_response_parse + +The `custom_response_parse` option allows you to specify how to parse the model's response. You can use JsonPath to +extract the specific data you need from the response. For example, by using `$.data[*].embedding`, you can extract +the `embedding` field values from the following JSON and obtain a `List` of nested `List` results. For more details on +using JsonPath, please refer to +the [JsonPath Getting Started guide](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started). + +```json +{ + "object": "list", + "data": [ + { + "object": "embedding", + "index": 0, + "embedding": [ + -0.006929283495992422, + -0.005336422007530928, + -0.00004547132266452536, + -0.024047505110502243 + ] + } + ], + "model": "text-embedding-3-small", + "usage": { + "prompt_tokens": 5, + "total_tokens": 5 + } +} +``` + +### custom_request_headers + +The `custom_request_headers` option allows you to define custom headers that should be included in the request sent to +the model's API. This is useful if the API requires additional headers beyond the standard ones, such as authorization +tokens, content types, etc. + +### custom_request_body + +The `custom_request_body` option supports placeholders: + +- `${model}`: Placeholder for the model name. +- `${input}`: Placeholder to determine input value and define request body request type based on the type of body + value. Example: `["${input}"]` -> ["input"] (list) + +### common options + +Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details. + +## Example Configuration + +```hocon +env { + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + book_id = "int" + book_name = "string" + book_intro = "string" + author_biography = "string" + } + } + rows = [ + {fields = [1, "To Kill a Mockingbird", + "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", + "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." + ], kind = INSERT} + {fields = [2, "1984", + "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", + "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." + ], kind = INSERT} + {fields = [3, "Pride and Prejudice", + "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", + "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." + ], kind = INSERT} + {fields = [4, "The Great GatsbyThe Great Gatsby", + "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", + "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." + ], kind = INSERT} + {fields = [5, "Moby-Dick", + "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", + "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." + ], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + Embedding { + source_table_name = "fake" + embedding_model_provider = QIANFAN + model = bge_large_en + api_key = xxxxxxxxxx + secret_key = xxxxxxxxxx + api_path = xxxxxxxxxx + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + result_table_name = "embedding_output" + } +} + +sink { + Assert { + source_table_name = "embedding_output" + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} +``` + +### Customize the embedding model + +```hocon + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + book_id = "int" + book_name = "string" + book_intro = "string" + author_biography = "string" + } + } + rows = [ + {fields = [1, "To Kill a Mockingbird", + "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", + "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." + ], kind = INSERT} + {fields = [2, "1984", + "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", + "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." + ], kind = INSERT} + {fields = [3, "Pride and Prejudice", + "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", + "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." + ], kind = INSERT} + {fields = [4, "The Great GatsbyThe Great Gatsby", + "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", + "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." + ], kind = INSERT} + {fields = [5, "Moby-Dick", + "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", + "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." + ], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + Embedding { + source_table_name = "fake" + model_provider = CUSTOM + model = text-embedding-3-small + api_key = xxxxxxxx + api_path = "http://mockserver:1080/v1/doubao/embedding" + single_vectorized_input_number = 2 + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + custom_config={ + custom_response_parse = "$.data[*].embedding" + custom_request_headers = { + "Content-Type"= "application/json" + "Authorization"= "Bearer xxxxxxx + } + custom_request_body ={ + modelx = "${model}" + inputx = ["${input}"] + } + } + result_table_name = "embedding_output_1" + } +} + +sink { + Assert { + source_table_name = "embedding_output_1" + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} + +``` diff --git a/docs/en/transform-v2/llm.md b/docs/en/transform-v2/llm.md index d03b8226f06..8caaad00a0e 100644 --- a/docs/en/transform-v2/llm.md +++ b/docs/en/transform-v2/llm.md @@ -10,19 +10,23 @@ 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 | +| 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 | | +| api_path | string | no | | +| custom_config | map | no | | +| custom_response_parse | string | no | | +| custom_request_headers | map | no | | +| custom_request_body | map | no | | ### model_provider The model provider to use. The available options are: -OPENAI +OPENAI、DOUBAO、CUSTOM ### output_data_type @@ -36,7 +40,7 @@ The prompt to send to the LLM. This parameter defines how LLM will process and r The data read from source is a table like this: -| name | age | +| name | age | |---------------|-----| | Jia Fan | 20 | | Hailin Wang | 20 | @@ -51,7 +55,7 @@ Determine whether someone is Chinese or American by their name The result will be: -| name | age | llm_output | +| name | age | llm_output | |---------------|-----|------------| | Jia Fan | 20 | Chinese | | Hailin Wang | 20 | Chinese | @@ -61,16 +65,73 @@ The result will be: ### 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. +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 +### api_path + +The API path to use for the 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. + +### custom_config + +The `custom_config` option allows you to provide additional custom configurations for the model. This is a map where you +can define various settings that might be required by the specific model you're using. + +### custom_response_parse + +The `custom_response_parse` option allows you to specify how to parse the model's response. You can use JsonPath to +extract the specific data you need from the response. For example, by using `$.choices[*].message.content`, you can +extract the `content` field values from the following JSON. For more details on using JsonPath, please refer to +the [JsonPath Getting Started guide](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started). + +```json +{ + "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" +} +``` + +### custom_request_headers + +The `custom_request_headers` option allows you to define custom headers that should be included in the request sent to +the model's API. This is useful if the API requires additional headers beyond the standard ones, such as authorization +tokens, content types, etc. + +### custom_request_body -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. +The `custom_request_body` option supports placeholders: + +- `${model}`: Placeholder for the model name. +- `${input}`: Placeholder to determine input value and define request body request type based on the type of body + value. Example: `"${input}"` -> "input" +- `${prompt}`:Placeholder for LLM model prompts. ### common options [string] @@ -120,3 +181,82 @@ sink { } ``` +### Customize the LLM model + +```hocon +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 = CUSTOM + 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" + custom_config={ + custom_response_parse = "$.choices[*].message.content" + custom_request_headers = { + Content-Type = "application/json" + Authorization = "Bearer xxxxxxxx" + } + custom_request_body ={ + model = "${model}" + messages = [ + { + role = "system" + content = "${prompt}" + }, + { + role = "user" + content = "${input}" + }] + } + } + 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 + } + ] + } + ] + } + } +} +``` + diff --git a/docs/sidebars.js b/docs/sidebars.js index 6061df55084..9f717d28aa2 100644 --- a/docs/sidebars.js +++ b/docs/sidebars.js @@ -84,7 +84,7 @@ const sidebars = { }, { "type": "category", - "label": "Concept", + "label": "Concepts", "items": [ "concept/config", "concept/connector-v2-features", @@ -106,7 +106,7 @@ const sidebars = { "link": { "type": "generated-index", "title": "Source(V2) of SeaTunnel", - "description": "List all source(v2) supported Apache SeaTunnel for now.", + "description": "List all source(v2) supported by Apache SeaTunnel for now.", "slug": "/connector-v2/source", "keywords": ["source"], "image": "/img/favicon.ico" @@ -124,7 +124,7 @@ const sidebars = { "link": { "type": "generated-index", "title": "Sink(V2) of SeaTunnel", - "description": "List all sink(v2) supported Apache SeaTunnel for now.", + "description": "List all sink(v2) supported by Apache SeaTunnel for now.", "slug": "/connector-v2/sink", "keywords": ["sink"], "image": "/img/favicon.ico" @@ -136,6 +136,24 @@ const sidebars = { } ] }, + { + "type": "category", + "label": "Formats", + "link": { + "type": "generated-index", + "title": "Formats", + "description": "List some special formats (not all) supported by Apache SeaTunnel for now.", + "slug": "/connector-v2/formats", + "keywords": ["formats"], + "image": "/img/favicon.ico" + }, + "items": [ + { + "type": "autogenerated", + "dirName": "connector-v2/formats" + } + ] + }, "connector-v2/source-common-options", "connector-v2/sink-common-options", "connector-v2/Error-Quick-Reference-Manual", diff --git a/docs/zh/concept/sql-config.md b/docs/zh/concept/sql-config.md index 7defa0010b2..e080e161feb 100644 --- a/docs/zh/concept/sql-config.md +++ b/docs/zh/concept/sql-config.md @@ -120,7 +120,10 @@ CREATE TABLE sink_table WITH ( INSERT INTO sink_table SELECT id, name, age, email FROM source_table; ``` -* `SELECT FROM` 部分为源端映射表的表名,`SELECT` 部分的语法参考:[SQL-transform](../transform-v2/sql.md) `query` 配置项 +* `SELECT FROM` 部分为源端映射表的表名,`SELECT` 部分的语法参考:[SQL-transform](../transform-v2/sql.md) `query` 配置项。如果select的字段是关键字([参考](https://github.com/JSQLParser/JSqlParser/blob/master/src/main/jjtree/net/sf/jsqlparser/parser/JSqlParserCC.jjt)),你应该像这样使用\`filedName\` +```sql +INSERT INTO sink_table SELECT id, name, age, email,`output` FROM source_table; +``` * `INSERT INTO` 部分为目标端映射表的表名 * 注意:该语法**不支持**在 `INSERT` 中指定字段,如:`INSERT INTO sink_table (id, name, age, email) SELECT id, name, age, email FROM source_table;` diff --git a/docs/zh/connector-v2/sink/Clickhouse.md b/docs/zh/connector-v2/sink/Clickhouse.md index 30002c607cf..61a359f5c0b 100644 --- a/docs/zh/connector-v2/sink/Clickhouse.md +++ b/docs/zh/connector-v2/sink/Clickhouse.md @@ -23,9 +23,9 @@ 为了使用 Clickhouse 连接器,需要以下依赖项。它们可以通过 install-plugin.sh 或从 Maven 中央存储库下载。 -| 数据源 | 支持的版本 | 依赖 | -|------------|-----------|------------------------------------------------------------------------------------------------------------| -| Clickhouse | universal | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | +| 数据源 | 支持的版本 | 依赖 | +|------------|-----------|------------------------------------------------------------------------------------| +| Clickhouse | universal | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-clickhouse) | ## 数据类型映射 diff --git a/docs/zh/connector-v2/sink/Hbase.md b/docs/zh/connector-v2/sink/Hbase.md index edc9e48510e..f028a8c93ee 100644 --- a/docs/zh/connector-v2/sink/Hbase.md +++ b/docs/zh/connector-v2/sink/Hbase.md @@ -119,6 +119,78 @@ Hbase { ``` +### 写入多表 + +```hocon +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + schema = { + table = "hbase_sink_1" + fields { + name = STRING + c_string = STRING + c_double = DOUBLE + c_bigint = BIGINT + c_float = FLOAT + c_int = INT + c_smallint = SMALLINT + c_boolean = BOOLEAN + time = BIGINT + } + } + rows = [ + { + kind = INSERT + fields = ["label_1", "sink_1", 4.3, 200, 2.5, 2, 5, true, 1627529632356] + } + ] + }, + { + schema = { + table = "hbase_sink_2" + fields { + name = STRING + c_string = STRING + c_double = DOUBLE + c_bigint = BIGINT + c_float = FLOAT + c_int = INT + c_smallint = SMALLINT + c_boolean = BOOLEAN + time = BIGINT + } + } + rows = [ + { + kind = INSERT + fields = ["label_2", "sink_2", 4.3, 200, 2.5, 2, 5, true, 1627529632357] + } + ] + } + ] + } +} + +sink { + Hbase { + zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" + table = "${table_name}" + rowkey_column = ["name"] + family_name { + all_columns = info + } + } +} +``` + ## 写入指定列族 ```hocon diff --git a/docs/zh/connector-v2/sink/Http.md b/docs/zh/connector-v2/sink/Http.md index 38d02d9be62..12ce90614ff 100644 --- a/docs/zh/connector-v2/sink/Http.md +++ b/docs/zh/connector-v2/sink/Http.md @@ -25,9 +25,9 @@ 想使用 Http 连接器,需要安装以下必要的依赖。可以通过运行 install-plugin.sh 脚本或者从 Maven 中央仓库下载这些依赖 -| 数据源 | 支持版本 | 依赖 | -|------|------|------------------------------------------------------------------------------------------------------| -| Http | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-http) | +| 数据源 | 支持版本 | 依赖 | +|------|------|------------------------------------------------------------------------------| +| Http | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-http) | ## 接收器选项 diff --git a/docs/zh/connector-v2/sink/Jdbc.md b/docs/zh/connector-v2/sink/Jdbc.md index b05ecbc501c..e1ab422952e 100644 --- a/docs/zh/connector-v2/sink/Jdbc.md +++ b/docs/zh/connector-v2/sink/Jdbc.md @@ -225,7 +225,7 @@ Sink插件常用参数,请参考 [Sink常用选项](../sink-common-options.md) | SQL Server | com.microsoft.sqlserver.jdbc.SQLServerDriver | jdbc:sqlserver://localhost:1433 | com.microsoft.sqlserver.jdbc.SQLServerXADataSource | https://mvnrepository.com/artifact/com.microsoft.sqlserver/mssql-jdbc | | Oracle | oracle.jdbc.OracleDriver | jdbc:oracle:thin:@localhost:1521/xepdb1 | oracle.jdbc.xa.OracleXADataSource | https://mvnrepository.com/artifact/com.oracle.database.jdbc/ojdbc8 | | sqlite | org.sqlite.JDBC | jdbc:sqlite:test.db | / | https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc | -| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar | +| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar | | StarRocks | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | / | https://mvnrepository.com/artifact/mysql/mysql-connector-java | | db2 | com.ibm.db2.jcc.DB2Driver | jdbc:db2://localhost:50000/testdb | com.ibm.db2.jcc.DB2XADataSource | https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc/db2jcc4 | | saphana | com.sap.db.jdbc.Driver | jdbc:sap://localhost:39015 | / | https://mvnrepository.com/artifact/com.sap.cloud.db.jdbc/ngdbc | diff --git a/docs/zh/connector-v2/sink/Kafka.md b/docs/zh/connector-v2/sink/Kafka.md index 45117a962d6..a7273c3b56c 100644 --- a/docs/zh/connector-v2/sink/Kafka.md +++ b/docs/zh/connector-v2/sink/Kafka.md @@ -24,25 +24,27 @@ 为了使用 Kafka 连接器,需要以下依赖项 可以通过 install-plugin.sh 或从 Maven 中央存储库下载 -| 数据源 | 支持版本 | Maven | -|-------|------|-------------------------------------------------------------------------------------------------------| -| Kafka | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | +| 数据源 | 支持版本 | Maven | +|-------|------|-------------------------------------------------------------------------------| +| Kafka | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | ## 接收器选项 -| 名称 | 类型 | 是否需要 | 默认值 | 描述 | -|----------------------|--------|------|------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| topic | String | 是 | - | 当表用作接收器时,topic 名称是要写入数据的 topic | -| bootstrap.servers | String | 是 | - | Kafka brokers 使用逗号分隔 | -| kafka.config | Map | 否 | - | 除了上述 Kafka Producer 客户端必须指定的参数外,用户还可以为 Producer 客户端指定多个非强制参数,涵盖 [Kafka官方文档中指定的所有生产者参数](https://kafka.apache.org/documentation.html#producerconfigs) | -| semantics | String | 否 | NON | 可以选择的语义是 EXACTLY_ONCE/AT_LEAST_ONCE/NON,默认 NON。 | -| partition_key_fields | Array | 否 | - | 配置字段用作 kafka 消息的key | -| partition | Int | 否 | - | 可以指定分区,所有消息都会发送到此分区 | -| assign_partitions | Array | 否 | - | 可以根据消息的内容决定发送哪个分区,该参数的作用是分发信息 | -| transaction_prefix | String | 否 | - | 如果语义指定为EXACTLY_ONCE,生产者将把所有消息写入一个 Kafka 事务中,kafka 通过不同的 transactionId 来区分不同的事务。该参数是kafka transactionId的前缀,确保不同的作业使用不同的前缀 | -| format | String | 否 | json | 数据格式。默认格式是json。可选文本格式,canal-json、debezium-json 和 avro。如果使用 json 或文本格式。默认字段分隔符是`,`。如果自定义分隔符,请添加`field_delimiter`选项。如果使用canal格式,请参考[canal-json](../formats/canal-json.md)。如果使用debezium格式,请参阅 [debezium-json](../formats/debezium-json.md) 了解详细信息 | -| field_delimiter | String | 否 | , | 自定义数据格式的字段分隔符 | -| common-options | | 否 | - | Sink插件常用参数,请参考 [Sink常用选项 ](../sink-common-options.md) 了解详情 | +| 名称 | 类型 | 是否需要 | 默认值 | 描述 | +|----------------------|--------|------|------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | 是 | - | 当表用作接收器时,topic 名称是要写入数据的 topic | +| bootstrap.servers | String | 是 | - | Kafka brokers 使用逗号分隔 | +| kafka.config | Map | 否 | - | 除了上述 Kafka Producer 客户端必须指定的参数外,用户还可以为 Producer 客户端指定多个非强制参数,涵盖 [Kafka官方文档中指定的所有生产者参数](https://kafka.apache.org/documentation.html#producerconfigs) | +| semantics | String | 否 | NON | 可以选择的语义是 EXACTLY_ONCE/AT_LEAST_ONCE/NON,默认 NON。 | +| partition_key_fields | Array | 否 | - | 配置字段用作 kafka 消息的key | +| partition | Int | 否 | - | 可以指定分区,所有消息都会发送到此分区 | +| assign_partitions | Array | 否 | - | 可以根据消息的内容决定发送哪个分区,该参数的作用是分发信息 | +| transaction_prefix | String | 否 | - | 如果语义指定为EXACTLY_ONCE,生产者将把所有消息写入一个 Kafka 事务中,kafka 通过不同的 transactionId 来区分不同的事务。该参数是kafka transactionId的前缀,确保不同的作业使用不同的前缀 | +| format | String | 否 | json | 数据格式。默认格式是json。可选文本格式,canal-json、debezium-json 、 avro 和 protobuf。如果使用 json 或文本格式。默认字段分隔符是`,`。如果自定义分隔符,请添加`field_delimiter`选项。如果使用canal格式,请参考[canal-json](../formats/canal-json.md)。如果使用debezium格式,请参阅 [debezium-json](../formats/debezium-json.md) 了解详细信息 | +| field_delimiter | String | 否 | , | 自定义数据格式的字段分隔符 | +| common-options | | 否 | - | Sink插件常用参数,请参考 [Sink常用选项 ](../sink-common-options.md) 了解详情 | +|protobuf_message_name|String|否|-| format配置为protobuf时生效,取Message名称 | +|protobuf_schema|String|否|-| format配置为protobuf时生效取Schema名称 | ## 参数解释 @@ -194,3 +196,56 @@ sink { } ``` +### Protobuf配置 + +`format` 设置为 `protobuf`,配置`protobuf`数据结构,`protobuf_message_name`和`protobuf_schema`参数 + +使用样例: + +```hocon +sink { + kafka { + topic = "test_protobuf_topic_fake_source" + bootstrap.servers = "kafkaCluster:9092" + format = protobuf + kafka.request.timeout.ms = 60000 + kafka.config = { + acks = "all" + request.timeout.ms = 60000 + buffer.memory = 33554432 + } + protobuf_message_name = Person + protobuf_schema = """ + syntax = "proto3"; + + package org.apache.seatunnel.format.protobuf; + + option java_outer_classname = "ProtobufE2E"; + + message Person { + int32 c_int32 = 1; + int64 c_int64 = 2; + float c_float = 3; + double c_double = 4; + bool c_bool = 5; + string c_string = 6; + bytes c_bytes = 7; + + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip = 4; + } + + Address address = 8; + + map attributes = 9; + + repeated string phone_numbers = 10; + } + """ + } +} +``` + diff --git a/docs/zh/connector-v2/sink/Paimon.md b/docs/zh/connector-v2/sink/Paimon.md index 11e299eb0d8..41fa395eb7e 100644 --- a/docs/zh/connector-v2/sink/Paimon.md +++ b/docs/zh/connector-v2/sink/Paimon.md @@ -30,21 +30,22 @@ libfb303-xxx.jar ## 连接器选项 -| 名称 | 类型 | 是否必须 | 默认值 | 描述 | -|-----------------------------|-----|------|------------------------------|---------------------------------------------------------------------------------------------------|---| -| warehouse | 字符串 | 是 | - | Paimon warehouse路径 | -| catalog_type | 字符串 | 否 | filesystem | Paimon的catalog类型,目前支持filesystem和hive | -| catalog_uri | 字符串 | 否 | - | Paimon catalog的uri,仅当catalog_type为hive时需要配置 | | -| database | 字符串 | 是 | - | 数据库名称 | -| table | 字符串 | 是 | - | 表名 | -| hdfs_site_path | 字符串 | 否 | - | hdfs-site.xml文件路径 | -| schema_save_mode | 枚举 | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | Schema保存模式 | -| data_save_mode | 枚举 | 否 | APPEND_DATA | 数据保存模式 | -| paimon.table.primary-keys | 字符串 | 否 | - | 主键字段列表,联合主键使用逗号分隔(注意:分区字段需要包含在主键字段中) | -| paimon.table.partition-keys | 字符串 | 否 | - | 分区字段列表,多字段使用逗号分隔 | -| paimon.table.write-props | Map | 否 | - | Paimon表初始化指定的属性, [参考](https://paimon.apache.org/docs/0.8/maintenance/configurations/#coreoptions) | -| paimon.hadoop.conf | Map | 否 | - | Hadoop配置文件属性信息 | -| paimon.hadoop.conf-path | 字符串 | 否 | - | Hadoop配置文件目录,用于加载'core-site.xml', 'hdfs-site.xml', 'hive-site.xml'文件配置 | +| 名称 | 类型 | 是否必须 | 默认值 | 描述 | +|-----------------------------|-------|----------|------------------------------|------------------------------------------------------------------------------------------------------| +| warehouse | 字符串 | 是 | - | Paimon warehouse路径 | +| catalog_type | 字符串 | 否 | filesystem | Paimon的catalog类型,目前支持filesystem和hive | +| catalog_uri | 字符串 | 否 | - | Paimon catalog的uri,仅当catalog_type为hive时需要配置 | +| database | 字符串 | 是 | - | 数据库名称 | +| table | 字符串 | 是 | - | 表名 | +| hdfs_site_path | 字符串 | 否 | - | hdfs-site.xml文件路径 | +| schema_save_mode | 枚举 | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | Schema保存模式 | +| data_save_mode | 枚举 | 否 | APPEND_DATA | 数据保存模式 | +| paimon.table.primary-keys | 字符串 | 否 | - | 主键字段列表,联合主键使用逗号分隔(注意:分区字段需要包含在主键字段中) | +| paimon.table.partition-keys | 字符串 | 否 | - | 分区字段列表,多字段使用逗号分隔 | +| paimon.table.write-props | Map | 否 | - | Paimon表初始化指定的属性, [参考](https://paimon.apache.org/docs/master/maintenance/configurations/#coreoptions) | +| paimon.hadoop.conf | Map | 否 | - | Hadoop配置文件属性信息 | +| paimon.hadoop.conf-path | 字符串 | 否 | - | Hadoop配置文件目录,用于加载'core-site.xml', 'hdfs-site.xml', 'hive-site.xml'文件配置 | + ## 示例 diff --git a/docs/zh/connector-v2/sink/Qdrant.md b/docs/zh/connector-v2/sink/Qdrant.md new file mode 100644 index 00000000000..7394eb85414 --- /dev/null +++ b/docs/zh/connector-v2/sink/Qdrant.md @@ -0,0 +1,68 @@ +# Qdrant + +> Qdrant 数据连接器 + +[Qdrant](https://qdrant.tech/) 是一个高性能的向量搜索引擎和向量数据库。 + +该连接器可用于将数据写入 Qdrant 集合。 + +## 数据类型映射 + +| SeaTunnel 数据类型 | Qdrant 数据类型 | +|---------------------|---------------| +| TINYINT | INTEGER | +| SMALLINT | INTEGER | +| INT | INTEGER | +| BIGINT | INTEGER | +| FLOAT | DOUBLE | +| DOUBLE | DOUBLE | +| BOOLEAN | BOOL | +| STRING | STRING | +| ARRAY | LIST | +| FLOAT_VECTOR | DENSE_VECTOR | +| BINARY_VECTOR | DENSE_VECTOR | +| FLOAT16_VECTOR | DENSE_VECTOR | +| BFLOAT16_VECTOR | DENSE_VECTOR | +| SPARSE_FLOAT_VECTOR | SPARSE_VECTOR | + +主键列的值将用作 Qdrant 中的点 ID。如果没有主键,则将使用随机 UUID。 + +## 选项 + +| 名称 | 类型 | 必填 | 默认值 | +|-----------------|--------|----|-----------| +| collection_name | string | 是 | - | +| batch_size | int | 否 | 64 | +| host | string | 否 | localhost | +| port | int | 否 | 6334 | +| api_key | string | 否 | - | +| use_tls | bool | 否 | false | +| common-options | | 否 | - | + +### collection_name [string] + +要从中读取数据的 Qdrant 集合的名称。 + +### batch_size [int] + +每个 upsert 请求到 Qdrant 的批量大小。 + +### host [string] + +Qdrant 实例的主机名。默认为 "localhost"。 + +### port [int] + +Qdrant 实例的 gRPC 端口。 + +### api_key [string] + +用于身份验证的 API 密钥(如果设置)。 + +### use_tls [bool] + +是否使用 TLS(SSL)连接。如果使用 Qdrant 云(https),则需要。 + +### 通用选项 + +接收插件的通用参数,请参考[源通用选项](../sink-common-options.md)了解详情。 diff --git a/docs/zh/connector-v2/sink/Typesense.md b/docs/zh/connector-v2/sink/Typesense.md new file mode 100644 index 00000000000..99017f32cb5 --- /dev/null +++ b/docs/zh/connector-v2/sink/Typesense.md @@ -0,0 +1,95 @@ +# Typesense + +## 描述 + +输出数据到 `Typesense` + +## 主要特性 + +- [ ] [精确一次](../../concept/connector-v2-features.md) +- [x] [cdc](../../concept/connector-v2-features.md) + +## 选项 + +| 名称 | 类型 | 是否必须 | 默认值 | +|------------------|--------|------|------------------------------| +| hosts | array | 是 | - | +| collection | string | 是 | - | +| schema_save_mode | string | 是 | CREATE_SCHEMA_WHEN_NOT_EXIST | +| data_save_mode | string | 是 | APPEND_DATA | +| primary_keys | array | 否 | | +| key_delimiter | string | 否 | `_` | +| api_key | string | 否 | | +| max_retry_count | int | 否 | 3 | +| max_batch_size | int | 否 | 10 | +| common-options | | 否 | - | + +### hosts [array] + +Typesense的访问地址,格式为 `host:port`,例如:["typesense-01:8108"] + +### collection [string] + +要写入的集合名,例如:“seatunnel” + +### primary_keys [array] + +主键字段用于生成文档 `id`。 + +### key_delimiter [string] + +设定复合键的分隔符(默认为 `_`)。 + +### api_key [config] + +typesense 安全认证的 api_key。 + +### max_retry_count [int] + +批次批量请求最大尝试大小 + +### max_batch_size [int] + +批次批量文档最大大小 + +### common options + +Sink插件常用参数,请参考 [Sink常用选项](../sink-common-options.md) 了解详情 + +### schema_save_mode + +在启动同步任务之前,针对目标侧已有的表结构选择不同的处理方案
+选项介绍:
+`RECREATE_SCHEMA` :当表不存在时会创建,当表已存在时会删除并重建
+`CREATE_SCHEMA_WHEN_NOT_EXIST` :当表不存在时会创建,当表已存在时则跳过创建
+`ERROR_WHEN_SCHEMA_NOT_EXIST` :当表不存在时将抛出错误
+ +### data_save_mode + +在启动同步任务之前,针对目标侧已存在的数据选择不同的处理方案
+选项介绍:
+`DROP_DATA`: 保留数据库结构,删除数据
+`APPEND_DATA`:保留数据库结构,保留数据
+`ERROR_WHEN_DATA_EXISTS`:当有数据时抛出错误
+ +## 示例 + +简单示例 + +```bash +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["localhost:8108"] + collection = "typesense_to_typesense_sink_with_query" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} +``` + diff --git a/docs/zh/connector-v2/source/Elasticsearch.md b/docs/zh/connector-v2/source/Elasticsearch.md new file mode 100644 index 00000000000..7a27f2b9371 --- /dev/null +++ b/docs/zh/connector-v2/source/Elasticsearch.md @@ -0,0 +1,247 @@ +# Elasticsearch + +> Elasticsearch source 连接器 + +## 简介 + +支持读取 Elasticsearch2.x 版本和 8.x 版本之间的数据 + +## Key features + +- [x] [批处理](../../concept/connector-v2-features.md) +- [ ] [流处理](../../concept/connector-v2-features.md) +- [ ] [精准一次](../../concept/connector-v2-features.md) +- [x] [column projection](../../concept/connector-v2-features.md) +- [ ] [并行度](../../concept/connector-v2-features.md) +- [ ] [支持用户自定义的分片](../../concept/connector-v2-features.md) + +## 配置参数选项 + +| 参数名称 | 类型 | 是否必须 | 默认值或者描述 | +| ----------------------- | ------- | -------- | ------------------------------------------------------- | +| hosts | 数组 | | - | +| username | string | no | - | +| password | string | no | - | +| index | string | No | 单索引同步配置,如果index_list没有配置,则必须配置index | +| index_list | array | no | 用来定义多索引同步任务 | +| source | array | no | - | +| query | json | no | {"match_all": {}} | +| scroll_time | string | no | 1m | +| scroll_size | int | no | 100 | +| tls_verify_certificate | boolean | no | true | +| tls_verify_hostnames | boolean | no | true | +| array_column | map | no | | +| tls_keystore_path | string | no | - | +| tls_keystore_password | string | no | - | +| tls_truststore_path | string | no | - | +| tls_truststore_password | string | no | - | +| common-options | | no | - | + +### hosts [array] + +Elasticsearch 集群的 HTTP 地址,格式为 `host:port`,允许指定多个主机。例如:`["host1:9200", "host2:9200"]`。 + +### username [string] + +用户名 + +### password [string] + +密码 + +### index [string] + +Elasticsearch 索引名称,支持 * 模糊匹配。比如存在索引index1,index2,可以指定index*同时读取两个索引的数据。 + +### source [array] + +索引的字段 + +你可以通过指定字段 `_id` 来获取文档 ID。如果将 `_id` 写入到其他索引,由于 Elasticsearch 的限制,你需要为 `_id` 指定一个别名。 + +如果你没有配置 `source`,它将自动从索引的映射中获取。 + +### array_column [array] + +由于 Elasticsearch 中没有数组索引,因此需要指定数组类型。 + +假设tags和phones是数组类型: + +```hocon +array_column = {tags = "array",phones = "array"} +``` + +### query [json] + +ElasticsSearch的原生查询语句,用于控制读取哪些数据写入到其他数据源。 + +### scroll_time [String] + +`Seatunnel`底层会使用滚动查询来查询数据,所以需要使用这个参数控制搜索上下文的时间长度。 + +### scroll_size [int] + +滚动查询的最大文档数量。 + +### index_list [array] + +`index_list` 用于定义多索引同步任务。它是一个数组,包含单表同步所需的参数,如 `query`、`source/schema`、`scroll_size` 和 `scroll_time`。建议不要将 `index_list` 和 `query` 配置在同一层级。有关更多详细信息,请参考后面的多表同步示例。 + +### tls_verify_certificate [boolean] + +启用 HTTPS 端点的证书验证 + +### tls_verify_hostname [boolean] + +启用 HTTPS 端点的主机名验证 + +### tls_keystore_path [string] + +PEM 或 JKS 密钥库的路径。该文件必须对运行 SeaTunnel 的操作系统用户可读。 + +### tls_keystore_password [string] + +指定密钥库的密钥密码。 + +### tls_truststore_path [string] + +PEM 或 JKS 信任库的路径。该文件必须对运行 SeaTunnel 的操作系统用户可读。 + +### tls_truststore_password [string] + +指定信任库的密钥密码。 + +### common options + +Source 插件常用参数,具体请参考 [Source 常用选项](../source-common-options.md) + +## 使用案例 + +案例一 + +> 案例一会从满足seatunnel-*匹配的索引中按照query读取数据,查询只会返回文档`id`,`name`,`age`,`tags`,`phones` 三个字段。在这个例子中,使用了source字段配置应该读取哪些字段,使用`array_column`指定了`tags`,`phones`应该被当做数组处理。 + +```hocon +Elasticsearch { + hosts = ["localhost:9200"] + index = "seatunnel-*" + array_column = {tags = "array",phones = "array"} + source = ["_id","name","age","tags","phones"] + query = {"range":{"firstPacket":{"gte":1669225429990,"lte":1669225429990}}} +} +``` + +案例二:多索引同步 + +> 此示例演示了如何从 `read_index1` 和 `read_index2` 中读取不同的数据数据,并将其分别写入 `read_index1_copy`,`read_index12_copy` 索引。 +> 在 `read_index1` 中,我使用 `source` 来指定要读取的字段,并使用`array_column`指明哪些字段是数组字段。 + +```hocon +source { + Elasticsearch { + hosts = ["https://elasticsearch:9200"] + username = "elastic" + password = "elasticsearch" + tls_verify_certificate = false + tls_verify_hostname = false + index_list = [ + { + index = "read_index1" + query = {"range": {"c_int": {"gte": 10, "lte": 20}}} + source = [ + c_map, + c_array, + c_string, + c_boolean, + c_tinyint, + c_smallint, + c_bigint, + c_float, + c_double, + c_decimal, + c_bytes, + c_int, + c_date, + c_timestamp + ] + array_column = { + c_array = "array" + } + } + { + index = "read_index2" + query = {"match_all": {}} + source = [ + c_int2, + c_date2, + c_null + ] + + } + + ] + + } +} + +transform { +} + +sink { + Elasticsearch { + hosts = ["https://elasticsearch:9200"] + username = "elastic" + password = "elasticsearch" + tls_verify_certificate = false + tls_verify_hostname = false + + index = "multi_source_write_test_index" + index_type = "st" + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + } +} +``` + +案例三:SSL(禁用证书验证) + +```hocon +source { + Elasticsearch { + hosts = ["https://localhost:9200"] + username = "elastic" + password = "elasticsearch" + + tls_verify_certificate = false + } +} +``` + +案例四:SSL(禁用主机名验证) + +```hocon +source { + Elasticsearch { + hosts = ["https://localhost:9200"] + username = "elastic" + password = "elasticsearch" + + tls_verify_hostname = false + } +} +``` + +案例五:SSL(启用证书验证) + +```hocon +source { + Elasticsearch { + hosts = ["https://localhost:9200"] + username = "elastic" + password = "elasticsearch" + + tls_keystore_path = "${your elasticsearch home}/config/certs/http.p12" + tls_keystore_password = "${your password}" + } +} +``` \ No newline at end of file diff --git a/docs/zh/connector-v2/source/Kafka.md b/docs/zh/connector-v2/source/Kafka.md new file mode 100644 index 00000000000..8f65e92e924 --- /dev/null +++ b/docs/zh/connector-v2/source/Kafka.md @@ -0,0 +1,288 @@ +# Kafka + +> Kafka 源连接器 + +## 支持以下引擎 + +> Spark
+> Flink
+> Seatunnel Zeta
+ +## 主要功能 + +- [x] [批处理](../../concept/connector-v2-features.md) +- [x] [流处理](../../concept/connector-v2-features.md) +- [x] [精确一次](../../concept/connector-v2-features.md) +- [ ] [列投影](../../concept/connector-v2-features.md) +- [x] [并行度](../../concept/connector-v2-features.md) +- [ ] [支持用户定义拆分](../../concept/connector-v2-features.md) + +## 描述 + +用于 Apache Kafka 的源连接器。 + +## 支持的数据源信息 + +使用 Kafka 连接器需要以下依赖项。 +可以通过 install-plugin.sh 下载或从 Maven 中央仓库获取。 + +| 数据源 | 支持的版本 | Maven 下载链接 | +|-------|-------|-------------------------------------------------------------------------------| +| Kafka | 通用版本 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | + +## 源选项 + +| 名称 | 类型 | 是否必填 | 默认值 | 描述 | +|-------------------------------------|-------------------------------------|------|--------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | 是 | - | 使用表作为数据源时要读取数据的主题名称。它也支持通过逗号分隔的多个主题列表,例如 'topic-1,topic-2'。 | +| table_list | Map | 否 | - | 主题列表配置,你可以同时配置一个 `table_list` 和一个 `topic`。 | +| bootstrap.servers | String | 是 | - | 逗号分隔的 Kafka brokers 列表。 | +| pattern | Boolean | 否 | false | 如果 `pattern` 设置为 `true`,则会使用指定的正则表达式匹配并订阅主题。 | +| consumer.group | String | 否 | SeaTunnel-Consumer-Group | `Kafka 消费者组 ID`,用于区分不同的消费者组。 | +| commit_on_checkpoint | Boolean | 否 | true | 如果为 true,消费者的偏移量将会定期在后台提交。 | +| kafka.config | Map | 否 | - | 除了上述必要参数外,用户还可以指定多个非强制的消费者客户端参数,覆盖 [Kafka 官方文档](https://kafka.apache.org/documentation.html#consumerconfigs) 中指定的所有消费者参数。 | +| schema | Config | 否 | - | 数据结构,包括字段名称和字段类型。 | +| format | String | 否 | json | 数据格式。默认格式为 json。可选格式包括 text, canal_json, debezium_json, ogg_json, maxwell_json, avro 和 protobuf。默认字段分隔符为 ", "。如果自定义分隔符,添加 "field_delimiter" 选项。如果使用 canal 格式,请参考 [canal-json](../formats/canal-json.md) 了解详细信息。如果使用 debezium 格式,请参考 [debezium-json](../formats/debezium-json.md)。一些Format的详细信息请参考 [formats](../formats) | +| format_error_handle_way | String | 否 | fail | 数据格式错误的处理方式。默认值为 fail,可选值为 fail 和 skip。当选择 fail 时,数据格式错误将阻塞并抛出异常。当选择 skip 时,数据格式错误将跳过此行数据。 | +| field_delimiter | String | 否 | , | 自定义数据格式的字段分隔符。 | +| start_mode | StartMode[earliest],[group_offsets] | 否 | group_offsets | 消费者的初始消费模式。 | +| start_mode.offsets | Config | 否 | - | 用于 specific_offsets 消费模式的偏移量。 | +| start_mode.timestamp | Long | 否 | - | 用于 "timestamp" 消费模式的时间。 | +| partition-discovery.interval-millis | Long | 否 | -1 | 动态发现主题和分区的间隔时间。 | +| common-options | | 否 | - | 源插件的常见参数,详情请参考 [Source Common Options](../source-common-options.md)。 | +| protobuf_message_name | String | 否 | - | 当格式设置为 protobuf 时有效,指定消息名称。 | +| protobuf_schema | String | 否 | - | 当格式设置为 protobuf 时有效,指定 Schema 定义。 | + +## 任务示例 + +### 简单示例 + +> 此示例读取 Kafka 的 topic_1、topic_2 和 topic_3 的数据并将其打印到客户端。如果尚未安装和部署 SeaTunnel,请按照 [安装指南](../../start-v2/locally/deployment.md) 进行安装和部署。然后,按照 [快速开始](../../start-v2/locally/quick-start-seatunnel-engine.md) 运行此任务。 + +```hocon +# 定义运行环境 +env { + parallelism = 2 + job.mode = "BATCH" +} +source { + Kafka { + schema = { + fields { + name = "string" + age = "int" + } + } + format = text + field_delimiter = "#" + topic = "topic_1,topic_2,topic_3" + bootstrap.servers = "localhost:9092" + kafka.config = { + client.id = client_1 + max.poll.records = 500 + auto.offset.reset = "earliest" + enable.auto.commit = "false" + } + } +} +sink { + Console {} +} +``` + +### 正则表达式主题 + +```hocon +source { + Kafka { + topic = ".*seatunnel*." + pattern = "true" + bootstrap.servers = "localhost:9092" + consumer.group = "seatunnel_group" + } +} +``` + +### AWS MSK SASL/SCRAM + +将以下 `${username}` 和 `${password}` 替换为 AWS MSK 中的配置值。 + +```hocon +source { + Kafka { + topic = "seatunnel" + bootstrap.servers = "xx.amazonaws.com.cn:9096,xxx.amazonaws.com.cn:9096,xxxx.amazonaws.com.cn:9096" + consumer.group = "seatunnel_group" + kafka.config = { + security.protocol=SASL_SSL + sasl.mechanism=SCRAM-SHA-512 + sasl.jaas.config="org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";" + } + } +} +``` + +### AWS MSK IAM + +从 [此处](https://github.com/aws/aws-msk-iam-auth/releases) 下载 `aws-msk-iam-auth-1.1.5.jar` 并将其放在 `$SEATUNNEL_HOME/plugin/kafka/lib` 目录下。 + +确保 IAM 策略中包含 `"kafka-cluster:Connect"` 权限,如下所示: + +```hocon +"Effect": "Allow", +"Action": [ + "kafka-cluster:Connect", + "kafka-cluster:AlterCluster", + "kafka-cluster:DescribeCluster" +], +``` + +源配置示例: + +```hocon +source { + Kafka { + topic = "seatunnel" + bootstrap.servers = "xx.amazonaws.com.cn:9098,xxx.amazonaws.com.cn:9098,xxxx.amazonaws.com.cn:9098" + consumer.group = "seatunnel_group" + kafka.config = { + security.protocol=SASL_SSL + sasl.mechanism=AWS_MSK_IAM + sasl.jaas.config="software.amazon.msk.auth.iam.IAMLoginModule required;" + sasl.client.callback.handler.class="software.amazon.msk.auth.iam.IAMClientCallbackHandler" + } + } +} +``` + +### Kerberos 认证示例 + +源配置示例: + +```hocon +source { + Kafka { + topic = "seatunnel" + bootstrap.servers = "127.0.0.1:9092" + consumer.group = "seatunnel_group" + kafka.config = { + security.protocol=SASL_PLAINTEXT + sasl.kerberos.service.name=kafka + sasl.mechanism=GSSAPI + java.security.krb5.conf="/etc/krb5.conf" + sasl.jaas.config="com.sun.security.auth.module.Krb5LoginModule required \n useKeyTab=true \n storeKey=true \n keyTab=\"/path/to/xxx.keytab\" \n principal=\"user@xxx.com\";" + } + } +} +``` + +### 多 Kafka 源示例 + +> 根据不同的 Kafka 主题和格式解析数据,并基于 ID 执行 upsert 操作。 + +```hocon +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Kafka { + + + bootstrap.servers = "kafka_e2e:9092" + table_list = [ + { + topic = "^test-ogg-sou.*" + pattern = "true" + consumer.group = "ogg_multi_group" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = ogg_json + }, + { + topic = "test-cdc_mds" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + }, + format = canal_json + } + ] + } +} + +sink { + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://postgresql:5432/test?loggerLevel=OFF" + user = test + password = test + generate_sink_sql = true + database = test + table = public.sink + primary_keys = ["id"] + } +} +``` + +### Protobuf配置 + +`format` 设置为 `protobuf`,配置`protobuf`数据结构,`protobuf_message_name`和`protobuf_schema`参数 + +使用样例: + +```hocon +source { + Kafka { + topic = "test_protobuf_topic_fake_source" + format = protobuf + protobuf_message_name = Person + protobuf_schema = """ + syntax = "proto3"; + + package org.apache.seatunnel.format.protobuf; + + option java_outer_classname = "ProtobufE2E"; + + message Person { + int32 c_int32 = 1; + int64 c_int64 = 2; + float c_float = 3; + double c_double = 4; + bool c_bool = 5; + string c_string = 6; + bytes c_bytes = 7; + + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip = 4; + } + + Address address = 8; + + map attributes = 9; + + repeated string phone_numbers = 10; + } + """ + bootstrap.servers = "kafkaCluster:9092" + start_mode = "earliest" + result_table_name = "kafka_table" + } +} +``` \ No newline at end of file diff --git a/docs/zh/connector-v2/source/Opengauss-CDC.md b/docs/zh/connector-v2/source/Opengauss-CDC.md new file mode 100644 index 00000000000..83da40b363e --- /dev/null +++ b/docs/zh/connector-v2/source/Opengauss-CDC.md @@ -0,0 +1,169 @@ +# Opengauss CDC + +> Opengauss CDC源连接器 + +## 支持这些引擎 + +> SeaTunnel Zeta
+> Flink
+ +## 主要功能 + +- [ ] [批处理](../../concept/connector-v2-features.md) +- [x] [流处理](../../concept/connector-v2-features.md) +- [x] [精确一次](../../concept/connector-v2-features.md) +- [ ] [列投影](../../concept/connector-v2-features.md) +- [x] [并行度](../../concept/connector-v2-features.md) +- [x] [支持用户定义的拆分](../../concept/connector-v2-features.md) + +## 描述 + +Opengauss CDC连接器允许从Opengauss数据库读取快照数据和增量数据。这个文档描述如何设置Opengauss CDC连接器以在Opengauss database中运行SQL查询。 + +## 使用步骤 + +> 这里是启用Opengauss CDC的步骤: + +1. 确保wal_level被设置为logical, 你可以直接使用SQL命令来修改这个配置: + +```sql +ALTER SYSTEM SET wal_level TO 'logical'; +SELECT pg_reload_conf(); +``` + +2. 改变指定表的REPLICA策略为FULL + +```sql +ALTER TABLE your_table_name REPLICA IDENTITY FULL; +``` + +如果你有很多表,你可以使用下面SQL的结果集来改变所有表的REPLICA策略 + +```sql +select 'ALTER TABLE ' || schemaname || '.' || tablename || ' REPLICA IDENTITY FULL;' from pg_tables where schemaname = 'YourTableSchema' +``` + +## 数据类型映射 + +| Opengauss Data type | SeaTunnel Data type | +|-----------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------| +| BOOL
| BOOLEAN | +| BYTEA
| BYTES | +| INT2
SMALLSERIAL
INT4
SERIAL
| INT | +| INT8
BIGSERIAL
| BIGINT | +| FLOAT4
| FLOAT | +| FLOAT8
| DOUBLE | +| NUMERIC(Get the designated column's specified column size>0) | DECIMAL(Get the designated column's specified column size,Gets the number of digits in the specified column to the right of the decimal point) | +| NUMERIC(Get the designated column's specified column size<0) | DECIMAL(38, 18) | +| BPCHAR
CHARACTER
VARCHAR
TEXT
GEOMETRY
GEOGRAPHY
JSON
JSONB | STRING | +| TIMESTAMP
| TIMESTAMP | +| TIME
| TIME | +| DATE
| DATE | +| OTHER DATA TYPES | NOT SUPPORTED YET | + +## 源端可选项 + +| Name | Type | Required | Default | Description | +|------------------------------------------------|------|----------|----------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| base-url | 字符串 | 是 | - | JDBC连接的URL. 参考: `jdbc:postgresql://localhost:5432/postgres_cdc?loggerLevel=OFF`. | +| username | 字符串 | 是 | - | 连接数据库的用户名 | +| password | 字符串 | 是 | - | 连接数据库的密码 | +| database-names | 列表 | 否 | - | 监控的数据库名称 | +| table-names | 列表 | 是 | - | 监控的数据表名称. 表名需要包含数据库名称, 例如: `database_name.table_name` | +| table-names-config | 列表 | 否 | - | 表配置的列表集合. 例如: [{"table": "db1.schema1.table1","primaryKeys":["key1"]}] | +| startup.mode | 枚举 | 否 | INITIAL | Opengauss CDC消费者的可选启动模式, 有效的枚举是`initial`, `earliest`, `latest` and `specific`.
`initial`: 启动时同步历史数据,然后同步增量数据
`earliest`: 从可能的最早偏移量启动
`latest`: 从最近的偏移量启动
`specific`: 从用户指定的偏移量启动 | +| snapshot.split.size | 整型 | 否 | 8096 | 表快照的分割大小(行数),在读取表的快照时,捕获的表被分割成多个split | +| snapshot.fetch.size | 整型 | 否 | 1024 | 读取表快照时,每次轮询的最大读取大小 | +| slot.name | 字符串 | 否 | - | Opengauss逻辑解码插槽的名称,该插槽是为特定数据库/模式的特定插件的流式更改而创建的。服务器使用此插槽将事件流传输到正在配置的连接器。默认值为seatunnel | +| decoding.plugin.name | 字符串 | 否 | pgoutput | 安装在服务器上的Postgres逻辑解码插件的名称,支持的值是decoderbufs、wal2json、wal2json_rds、wal2json_streaming、wal2json_rds_streaming和pgoutput | +| server-time-zone | 字符串 | 否 | UTC | 数据库服务器中的会话时区。如果没有设置,则使用ZoneId.systemDefault()来确定服务器的时区 | +| connect.timeout.ms | 时间间隔 | 否 | 30000 | 在尝试连接数据库服务器之后,连接器在超时之前应该等待的最大时间 | +| connect.max-retries | 整型 | 否 | 3 | 连接器在建立数据库服务器连接时应该重试的最大次数 | +| connection.pool.size | 整型 | 否 | 20 | jdbc连接池的大小 | +| chunk-key.even-distribution.factor.upper-bound | 双浮点型 | 否 | 100 | chunk的key分布因子的上界。该因子用于确定表数据是否均匀分布。如果分布因子被计算为小于或等于这个上界(即(MAX(id) - MIN(id) + 1) /行数),表的所有chunk将被优化以达到均匀分布。否则,如果分布因子更大,则认为表分布不均匀,如果估计的分片数量超过`sample-sharding.threshold`指定的值,则将使用基于采样的分片策略。默认值为100.0。 | +| chunk-key.even-distribution.factor.lower-bound | 双浮点型 | 否 | 0.05 | chunk的key分布因子的下界。该因子用于确定表数据是否均匀分布。如果分布因子的计算结果大于或等于这个下界(即(MAX(id) - MIN(id) + 1) /行数),那么表的所有块将被优化以达到均匀分布。否则,如果分布因子较小,则认为表分布不均匀,如果估计的分片数量超过`sample-sharding.threshold`指定的值,则使用基于采样的分片策略。缺省值为0.05。 | +| sample-sharding.threshold | 整型 | 否 | 1000 | 此配置指定了用于触发采样分片策略的估计分片数的阈值。当分布因子超出了由`chunk-key.even-distribution.factor.upper-bound `和`chunk-key.even-distribution.factor.lower-bound`,并且估计的分片计数(以近似的行数/块大小计算)超过此阈值,则将使用样本分片策略。这有助于更有效地处理大型数据集。默认值为1000个分片。 | +| inverse-sampling.rate | 整型 | 否 | 1000 | 采样分片策略中使用的采样率的倒数。例如,如果该值设置为1000,则意味着在采样过程中应用了1/1000的采样率。该选项提供了控制采样粒度的灵活性,从而影响最终的分片数量。当处理非常大的数据集时,它特别有用,其中首选较低的采样率。缺省值为1000。 | +| exactly_once | 布尔 | 否 | false | 启用exactly once语义 | +| format | 枚举 | 否 | DEFAULT | Opengauss CDC可选的输出格式, 有效的枚举是`DEFAULT`, `COMPATIBLE_DEBEZIUM_JSON`. | +| debezium | 配置 | 否 | - | 将 [Debezium的属性](https://github.com/debezium/debezium/blob/v1.9.8.Final/documentation/modules/ROOT/pages/connectors/postgresql.adoc#connector-configuration-properties) 传递到Debezium嵌入式引擎,该引擎用于捕获来自Opengauss服务的数据更改 | +| common-options | | 否 | - | 源码插件通用参数, 请参考[Source Common Options](../source-common-options.md)获取详情 | + +## 任务示例 + +### 简单 + +> 支持多表读 + +``` + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1","opengauss_cdc.inventory.opengauss_cdc_table_2"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" + decoding.plugin.name = "pgoutput" + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = "opengauss_cdc" + schema = "inventory" + tablePrefix = "sink_" + primary_keys = ["id"] + } +} + +``` + +### 支持自定义主键 + +``` +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + exactly_once = true + table-names-config = [ + { + table = "opengauss_cdc.inventory.full_types_no_primary_key" + primaryKeys = ["id"] + } + ] + } +} +``` + diff --git a/docs/zh/connector-v2/source/Qdrant.md b/docs/zh/connector-v2/source/Qdrant.md new file mode 100644 index 00000000000..140ff36a395 --- /dev/null +++ b/docs/zh/connector-v2/source/Qdrant.md @@ -0,0 +1,79 @@ +# Qdrant + +> Qdrant 数据源连接器 + +[Qdrant](https://qdrant.tech/) 是一个高性能的向量搜索引擎和向量数据库。 + +该连接器可用于从 Qdrant 集合中读取数据。 + +## 选项 + +| 名称 | 类型 | 必填 | 默认值 | +|-----------------|--------|----|-----------| +| collection_name | string | 是 | - | +| schema | config | 是 | - | +| host | string | 否 | localhost | +| port | int | 否 | 6334 | +| api_key | string | 否 | - | +| use_tls | bool | 否 | false | +| common-options | | 否 | - | + +### collection_name [string] + +要从中读取数据的 Qdrant 集合的名称。 + +### schema [config] + +要将数据读取到的表的模式。 + +例如: + +```hocon +schema = { + fields { + age = int + address = string + some_vector = float_vector + } +} +``` + +Qdrant 中的每个条目称为一个点。 + +`float_vector` 类型的列从每个点的向量中读取,其他列从与该点关联的 JSON 有效负载中读取。 + +如果列被标记为主键,Qdrant 点的 ID 将写入其中。它可以是 `"string"` 或 `"int"` 类型。因为 Qdrant 仅[允许](https://qdrant.tech/documentation/concepts/points/#point-ids)使用正整数和 UUID 作为点 ID。 + +如果集合是用单个默认/未命名向量创建的,请使用 `default_vector` 作为向量名称。 + +```hocon +schema = { + fields { + age = int + address = string + default_vector = float_vector + } +} +``` + +Qdrant 中点的 ID 将写入标记为主键的列中。它可以是 `int` 或 `string` 类型。 + +### host [string] + +Qdrant 实例的主机名。默认为 "localhost"。 + +### port [int] + +Qdrant 实例的 gRPC 端口。 + +### api_key [string] + +用于身份验证的 API 密钥(如果设置)。 + +### use_tls [bool] + +是否使用 TLS(SSL)连接。如果使用 Qdrant 云(https),则需要。 + +### 通用选项 + +源插件的通用参数,请参考[源通用选项](../source-common-options.md)了解详情。**** diff --git a/docs/zh/connector-v2/source/Sls.md b/docs/zh/connector-v2/source/Sls.md index d0e10257258..59b00df0850 100644 --- a/docs/zh/connector-v2/source/Sls.md +++ b/docs/zh/connector-v2/source/Sls.md @@ -26,9 +26,9 @@ 为了使用Sls连接器,需要以下依赖关系。 它们可以通过install-plugin.sh或Maven中央存储库下载。 -| 数据源 | 支持的版本 | Maven | -|-----|-----------|-----------------------------------------------------------------------------------------------------------| -| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-sls) | +| 数据源 | 支持的版本 | Maven | +|-----|-----------|-----------------------------------------------------------------------------------| +| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-sls) | ## Source Options diff --git a/docs/zh/connector-v2/source/Typesense.md b/docs/zh/connector-v2/source/Typesense.md new file mode 100644 index 00000000000..35f04e23a27 --- /dev/null +++ b/docs/zh/connector-v2/source/Typesense.md @@ -0,0 +1,79 @@ +# Typesense + +> Typesense 源连接器 + +## 描述 + +从 Typesense 读取数据。 + +## 主要功能 + +- [x] [批处理](../../concept/connector-v2-features.md) +- [ ] [流处理](../../concept/connector-v2-features.md) +- [ ] [精确一次](../../concept/connector-v2-features.md) +- [x] [Schema](../../concept/connector-v2-features.md) +- [x] [并行度](../../concept/connector-v2-features.md) +- [ ] [支持用户定义的拆分](../../concept/connector-v2-features.md) + +## 选项 + +| 名称 | 类型 | 必填 | 默认值 | +|------------|--------|----|-----| +| hosts | array | 是 | - | +| collection | string | 是 | - | +| schema | config | 是 | - | +| api_key | string | 否 | - | +| query | string | 否 | - | +| batch_size | int | 否 | 100 | + +### hosts [array] + +Typesense的访问地址,格式为 `host:port`,例如:["typesense-01:8108"] + +### collection [string] + +要写入的集合名,例如:“seatunnel” + +### schema [config] + +typesense 需要读取的列。有关更多信息,请参阅:[guide](../../concept/schema-feature.md#how-to-declare-type-supported)。 + +### api_key [config] + +typesense 安全认证的 api_key。 + +### batch_size + +读取数据时,每批次查询数量 + +### 常用选项 + +Source 插件常用参数,具体请参考 [Source 常用选项](../source-common-options.md) + +## 示例 + +```bash +source { + Typesense { + hosts = ["localhost:8108"] + collection = "companies" + api_key = "xyz" + query = "q=*&filter_by=num_employees:>9000" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} +``` + diff --git a/docs/zh/faq.md b/docs/zh/faq.md index 3be6ce38e56..4fc24e6a3ad 100644 --- a/docs/zh/faq.md +++ b/docs/zh/faq.md @@ -204,23 +204,6 @@ spark { } ``` -## 如何为 YARN 上的 SeaTunnel 指定不同的 JDK 版本? - -例如要设置JDK版本为JDK8,有两种情况: - -- YARN集群已部署JDK8,但默认JDK不是JDK8。 在 SeaTunnel 配置文件中添加两个配置: - - ``` - env { - ... - spark.executorEnv.JAVA_HOME="/your/java_8_home/directory" - spark.yarn.appMasterEnv.JAVA_HOME="/your/java_8_home/directory" - ... - } - ``` -- YARN集群未部署JDK8。 此时,启动附带JDK8的SeaTunnel。 详细操作参见: - https://www.cnblogs.com/jasondan/p/spark-specific-jdk-version.html - ## Spark local[*]模式运行SeaTunnel时总是出现OOM怎么办? 如果以本地模式运行,则需要修改`start-seatunnel.sh`启动脚本。 在 `spark-submit` 之后添加参数 `--driver-memory 4g` 。 一般情况下,生产环境中不使用本地模式。 因此,On YARN时一般不需要设置该参数。 有关详细信息,请参阅:[应用程序属性](https://spark.apache.org/docs/latest/configuration.html#application-properties)。 @@ -335,10 +318,6 @@ spark-submit --verbose ... ``` -## 如何使用SeaTunnel跨HDFS集群同步数据? - -只需正确配置 hdfs-site.xml 即可。 参考:https://www.cnblogs.com/suanec/p/7828139.html。 - ## 我想学习SeaTunnel的源代码。 我应该从哪里开始? SeaTunnel 拥有完全抽象、结构化的代码实现,很多人都选择 SeaTunnel 作为学习 Spark 的方式。 您可以从主程序入口了解源代码:SeaTunnel.java diff --git a/docs/zh/seatunnel-engine/checkpoint-storage.md b/docs/zh/seatunnel-engine/checkpoint-storage.md index 7dd26ca11f0..86165d5d3be 100644 --- a/docs/zh/seatunnel-engine/checkpoint-storage.md +++ b/docs/zh/seatunnel-engine/checkpoint-storage.md @@ -65,7 +65,6 @@ seatunnel: fs.oss.accessKeyId: your-access-key fs.oss.accessKeySecret: your-secret-key fs.oss.endpoint: endpoint address - fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` 有关Hadoop Credential Provider API的更多信息,请参见: [Credential Provider API](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/CredentialProviderAPI.html). diff --git a/docs/zh/seatunnel-engine/rest-api.md b/docs/zh/seatunnel-engine/rest-api.md index 69199cccc21..6d5a15d3f48 100644 --- a/docs/zh/seatunnel-engine/rest-api.md +++ b/docs/zh/seatunnel-engine/rest-api.md @@ -68,7 +68,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回所有作业及其当前状态的概览。 +### 返回所有作业及其当前状态的概览
GET /hazelcast/rest/maps/running-jobs (返回所有作业及其当前状态的概览。) @@ -107,7 +107,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回作业的详细信息。 +### 返回作业的详细信息
GET /hazelcast/rest/maps/job-info/:jobId (返回作业的详细信息。) @@ -233,7 +233,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回所有已完成的作业信息。 +### 返回所有已完成的作业信息
GET /hazelcast/rest/maps/finished-jobs/:state (返回所有已完成的作业信息。) @@ -265,7 +265,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回系统监控信息。 +### 返回系统监控信息
GET /hazelcast/rest/maps/system-monitoring-information (返回系统监控信息。) @@ -330,7 +330,7 @@ network: ------------------------------------------------------------------------------------------ -### 提交作业。 +### 提交作业
POST /hazelcast/rest/maps/submit-job (如果作业提交成功,返回jobId和jobName。) @@ -388,7 +388,110 @@ network: ------------------------------------------------------------------------------------------ -### 停止作业。 + +### 批量提交作业 + +
+POST /hazelcast/rest/maps/submit-jobs (如果作业提交成功,返回jobId和jobName。) + +#### 参数(在请求体中params字段中添加) + +> | 参数名称 | 是否必传 | 参数类型 | 参数描述 | +> |----------------------|----------|--------|-----------------------------------| +> | jobId | optional | string | job id | +> | jobName | optional | string | job name | +> | isStartWithSavePoint | optional | string | if job is started with save point | + + + +#### 请求体 + +```json +[ + { + "params":{ + "jobId":"123456", + "jobName":"SeaTunnel-01" + }, + "env": { + "job.mode": "batch" + }, + "source": [ + { + "plugin_name": "FakeSource", + "result_table_name": "fake", + "row.num": 1000, + "schema": { + "fields": { + "name": "string", + "age": "int", + "card": "int" + } + } + } + ], + "transform": [ + ], + "sink": [ + { + "plugin_name": "Console", + "source_table_name": ["fake"] + } + ] + }, + { + "params":{ + "jobId":"1234567", + "jobName":"SeaTunnel-02" + }, + "env": { + "job.mode": "batch" + }, + "source": [ + { + "plugin_name": "FakeSource", + "result_table_name": "fake", + "row.num": 1000, + "schema": { + "fields": { + "name": "string", + "age": "int", + "card": "int" + } + } + } + ], + "transform": [ + ], + "sink": [ + { + "plugin_name": "Console", + "source_table_name": ["fake"] + } + ] + } +] +``` + +#### 响应 + +```json +[ + { + "jobId": "123456", + "jobName": "SeaTunnel-01" + },{ + "jobId": "1234567", + "jobName": "SeaTunnel-02" + } +] +``` + +
+ +------------------------------------------------------------------------------------------ + +### 停止作业
POST /hazelcast/rest/maps/stop-job (如果作业成功停止,返回jobId。) @@ -412,9 +515,47 @@ network:
+ +------------------------------------------------------------------------------------------ + +### 批量停止作业 + +
+POST /hazelcast/rest/maps/stop-jobs (如果作业成功停止,返回jobId。) + +#### 请求体 + +```json +[ + { + "jobId": 881432421482889220, + "isStopWithSavePoint": false + }, + { + "jobId": 881432456517910529, + "isStopWithSavePoint": false + } +] +``` + +#### 响应 + +```json +[ + { + "jobId": 881432421482889220 + }, + { + "jobId": 881432456517910529 + } +] +``` + +
+ ------------------------------------------------------------------------------------------ -### 加密配置。 +### 加密配置
POST /hazelcast/rest/maps/encrypt-config (如果配置加密成功,则返回加密后的配置。) diff --git a/docs/zh/start-v2/locally/deployment.md b/docs/zh/start-v2/locally/deployment.md index a21612570e5..e2ae8dadd03 100644 --- a/docs/zh/start-v2/locally/deployment.md +++ b/docs/zh/start-v2/locally/deployment.md @@ -5,15 +5,17 @@ sidebar_position: 1 import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -# 本地部署 +# 部署 -## 步骤 1: 准备工作 +## 准备工作 在开始本地运行前,您需要确保您已经安装了SeaTunnel所需要的以下软件: * 安装[Java](https://www.java.com/en/download/) (Java 8 或 11, 其他高于Java 8的版本理论上也可以工作) 以及设置 `JAVA_HOME`。 -## 步骤 2: 下载 SeaTunnel +## 下载 SeaTunnel 发行包 + +### 下载二进制包 进入[SeaTunnel下载页面](https://seatunnel.apache.org/download)下载最新版本的二进制安装包`seatunnel--bin.tar.gz` @@ -25,7 +27,7 @@ wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${ve tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` -## 步骤 3: 下载连接器插件 +### 下载连接器插件 从2.2.0-beta版本开始,二进制包不再默认提供连接器依赖,因此在第一次使用时,您需要执行以下命令来安装连接器:(当然,您也可以从 [Apache Maven Repository](https://repo.maven.apache.org/maven2/org/apache/seatunnel/) 手动下载连接器,然后将其移动至`connectors/`目录下,如果是2.3.5之前则需要放入`connectors/seatunnel`目录下)。 @@ -56,4 +58,33 @@ connector-console ::: -现在,您已经完成了SeaTunnel部署。您可以按照[快速开始](quick-start-seatunnel-engine.md)来配置并运行数据同步作业了。 +## 从源码构建SeaTunnel + +### 下载源码 + +从源码构建SeaTunnel。下载源码的方式与下载二进制包的方式相同。 +您可以从[下载页面](https://seatunnel.apache.org/download/)下载源码,或者从[GitHub仓库](https://github.com/apache/seatunnel/releases)克隆源码。 + +### 构建源码 + +```shell +cd seatunnel +sh ./mvnw clean package -DskipTests -Dskip.spotless=true +# 获取构建好的二进制包 +cp seatunnel-dist/target/apache-seatunnel-2.3.8-bin.tar.gz /The-Path-You-Want-To-Copy + +cd /The-Path-You-Want-To-Copy +tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" +``` + +当从源码构建时,所有的连接器插件和一些必要的依赖(例如:mysql驱动)都包含在二进制包中。您可以直接使用连接器插件,而无需单独安装它们。 + +# 启动SeaTunnel + +现在您已经下载了SeaTunnel二进制包和连接器插件。接下来,您可以选择不同的引擎选项来运行同步任务。 + +如果您使用Flink来运行同步任务,则无需部署SeaTunnel引擎服务集群。您可以参考[Flink 引擎快速开始](quick-start-flink.md)来运行您的同步任务。 + +如果您使用Spark来运行同步任务,则无需部署SeaTunnel引擎服务集群。您可以参考[Spark 引擎快速开始](quick-start-spark.md)来运行您的同步任务。 + +如果您使用内置的SeaTunnel引擎(Zeta)来运行任务,则需要先部署SeaTunnel引擎服务。请参考[SeaTunnel 引擎快速开始](quick-start-seatunnel-engine.md)。 diff --git a/docs/zh/start-v2/locally/quick-start-flink.md b/docs/zh/start-v2/locally/quick-start-flink.md index f52d8507824..87de2808c4a 100644 --- a/docs/zh/start-v2/locally/quick-start-flink.md +++ b/docs/zh/start-v2/locally/quick-start-flink.md @@ -2,7 +2,7 @@ sidebar_position: 3 --- -# Flink Engine快速开始 +# Flink 引擎快速开始 ## 步骤 1: 部署SeaTunnel及连接器 diff --git a/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md b/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md index 16f02d6c385..bf2515ddb6e 100644 --- a/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md +++ b/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md @@ -2,7 +2,7 @@ sidebar_position: 2 --- -# SeaTunnel Engine快速开始 +# SeaTunnel 引擎快速开始 ## 步骤 1: 部署SeaTunnel及连接器 diff --git a/docs/zh/start-v2/locally/quick-start-spark.md b/docs/zh/start-v2/locally/quick-start-spark.md index e00a7fa68ac..59fbe47049a 100644 --- a/docs/zh/start-v2/locally/quick-start-spark.md +++ b/docs/zh/start-v2/locally/quick-start-spark.md @@ -2,7 +2,7 @@ sidebar_position: 4 --- -# Spark引擎快速开始 +# Spark 引擎快速开始 ## 步骤 1: 部署SeaTunnel及连接器 diff --git a/docs/zh/transform-v2/embedding.md b/docs/zh/transform-v2/embedding.md new file mode 100644 index 00000000000..fbee0ac33e9 --- /dev/null +++ b/docs/zh/transform-v2/embedding.md @@ -0,0 +1,382 @@ +# Embedding + +> Embedding Transform Plugin + +## 描述 + +`Embedding` 转换插件利用 embedding 模型将文本数据转换为向量化表示。此转换可以应用于各种字段。该插件支持多种模型提供商,并且可以与不同的API集成。 + +## 配置选项 + +| 名称 | 类型 | 是否必填 | 默认值 | 描述 | +|--------------------------------|--------|------|-----|------------------------------------------------------------------| +| model_provider | enum | 是 | - | embedding模型的提供商。可选项包括 `QIANFAN`、`OPENAI` 等。 | +| api_key | string | 是 | - | 用于验证embedding服务的API密钥。 | +| secret_key | string | 是 | - | 用于额外验证的密钥。一些提供商可能需要此密钥进行安全的API请求。 | +| single_vectorized_input_number | int | 否 | 1 | 单次请求向量化的输入数量。默认值为1。 | +| vectorization_fields | map | 是 | - | 输入字段和相应的输出向量字段之间的映射。 | +| model | string | 是 | - | 要使用的具体embedding模型。例如,如果提供商为OPENAI,可以指定 `text-embedding-3-small`。 | +| api_path | string | 否 | - | embedding服务的API。通常由模型提供商提供。 | +| oauth_path | string | 否 | - | oauth 服务的 API 。 | +| custom_config | map | 否 | | 模型的自定义配置。 | +| custom_response_parse | string | 否 | | 使用 JsonPath 解析模型响应的方式。示例:`$.choices[*].message.content`。 | +| custom_request_headers | map | 否 | | 发送到模型的请求的自定义头信息。 | +| custom_request_body | map | 否 | | 请求体的自定义配置。支持占位符如 `${model}`、`${input}`。 | + +### embedding_model_provider + +用于生成 embedding 的模型提供商。常见选项包括 `DOUBAO`、`QIANFAN`、`OPENAI` 等,同时可选择 `CUSTOM` 实现自定义 embedding +模型的请求以及获取。 + +### api_key + +用于验证 embedding 服务请求的API密钥。通常由模型提供商在你注册他们的服务时提供。 + +### secret_key + +用于额外验证的密钥。一些提供商可能要求此密钥以确保API请求的安全性。 + +### single_vectorized_input_number + +指定单次请求向量化的输入数量。默认值为1。根据处理能力和模型提供商的API限制进行调整。 + +### vectorization_fields + +输入字段和相应的输出向量字段之间的映射。这使得插件可以理解要向量化的文本字段以及如何存储生成的向量。 + +```hocon +vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography +} +``` + +### model + +要使用的具体 embedding 模型。这取决于`embedding_model_provider`。例如,如果使用 OPENAI ,可以指定 `text-embedding-3-small`。 + +### api_path + +用于向 embedding 服务发送请求的API。根据提供商和所用模型的不同可能有所变化。通常由模型提供商提供。 + +### oauth_path + +用于向oauth服务发送请求的API,获取对应的认证信息。根据提供商和所用模型的不同可能有所变化。通常由模型提供商提供。 + +### custom_config + +`custom_config` 选项允许您为模型提供额外的自定义配置。这是一个映射,您可以在其中定义特定模型可能需要的各种设置。 + +### custom_response_parse + +`custom_response_parse` 选项允许您指定如何解析模型的响应。您可以使用 JsonPath +从响应中提取所需的特定数据。例如,使用 `$.data[*].embedding` 提取如下json中的 `embedding` 字段 +值,获取 `List` 嵌套 `List` 的结果。JsonPath +的使用请参考 [JsonPath 快速入门](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started) + +```json +{ + "object": "list", + "data": [ + { + "object": "embedding", + "index": 0, + "embedding": [ + -0.006929283495992422, + -0.005336422007530928, + -0.00004547132266452536, + -0.024047505110502243 + ] + } + ], + "model": "text-embedding-3-small", + "usage": { + "prompt_tokens": 5, + "total_tokens": 5 + } +} +``` + +### custom_request_headers + +`custom_request_headers` 选项允许您定义应包含在发送到模型 API 的请求中的自定义头信息。如果 API +需要标准头信息之外的额外头信息,例如授权令牌、内容类型等,这个选项会非常有用。 + +### custom_request_body + +`custom_request_body` 选项支持占位符: + +- `${model}`:用于模型名称的占位符。 +- `${input}`:用于确定输入值的占位符,同时根据 body value 的类型定义请求体请求类型。例如:`["${input}"]` -> ["input"] ( + list)。 + +### common options + +转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情 + +## 示例配置 + +```hocon +env { + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + book_id = "int" + book_name = "string" + book_intro = "string" + author_biography = "string" + } + } + rows = [ + {fields = [1, "To Kill a Mockingbird", + "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", + "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." + ], kind = INSERT} + {fields = [2, "1984", + "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", + "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." + ], kind = INSERT} + {fields = [3, "Pride and Prejudice", + "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", + "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." + ], kind = INSERT} + {fields = [4, "The Great GatsbyThe Great Gatsby", + "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", + "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." + ], kind = INSERT} + {fields = [5, "Moby-Dick", + "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", + "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." + ], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + Embedding { + source_table_name = "fake" + embedding_model_provider = QIANFAN + model = bge_large_en + api_key = xxxxxxxxxx + secret_key = xxxxxxxxxx + api_path = xxxxxxxxxx + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + result_table_name = "embedding_output" + } +} + +sink { + Assert { + source_table_name = "embedding_output" + + + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} +``` + +### Customize the embedding model + +```hocon + +env { + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + schema = { + fields { + book_id = "int" + book_name = "string" + book_intro = "string" + author_biography = "string" + } + } + rows = [ + {fields = [1, "To Kill a Mockingbird", + "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", + "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." + ], kind = INSERT} + {fields = [2, "1984", + "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", + "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." + ], kind = INSERT} + {fields = [3, "Pride and Prejudice", + "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", + "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." + ], kind = INSERT} + {fields = [4, "The Great GatsbyThe Great Gatsby", + "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", + "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." + ], kind = INSERT} + {fields = [5, "Moby-Dick", + "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", + "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." + ], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + Embedding { + source_table_name = "fake" + model_provider = CUSTOM + model = text-embedding-3-small + api_key = xxxxxxxx + api_path = "http://mockserver:1080/v1/doubao/embedding" + single_vectorized_input_number = 2 + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + custom_config={ + custom_response_parse = "$.data[*].embedding" + custom_request_headers = { + "Content-Type"= "application/json" + "Authorization"= "Bearer xxxxxxx + } + custom_request_body ={ + modelx = "${model}" + inputx = ["${input}"] + } + } + result_table_name = "embedding_output_1" + } +} + +sink { + Assert { + source_table_name = "embedding_output_1" + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} + +``` \ No newline at end of file diff --git a/docs/zh/transform-v2/llm.md b/docs/zh/transform-v2/llm.md index acd3245b8eb..5efcf47125d 100644 --- a/docs/zh/transform-v2/llm.md +++ b/docs/zh/transform-v2/llm.md @@ -8,19 +8,23 @@ ## 属性 -| 名称 | 类型 | 是否必须 | 默认值 | -|------------------|--------|------|--------------------------------------------| -| 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 | enum | yes | | +| output_data_type | enum | no | String | +| prompt | string | yes | | +| model | string | yes | | +| api_key | string | yes | | +| api_path | string | no | | +| custom_config | map | no | | +| custom_response_parse | string | no | | +| custom_request_headers | map | no | | +| custom_request_body | map | no | | ### model_provider 要使用的模型提供者。可用选项为: -OPENAI +OPENAI、DOUBAO、CUSTOM ### output_data_type @@ -34,7 +38,7 @@ STRING,INT,BIGINT,DOUBLE,BOOLEAN. 从源读取的数据是这样的表格: -| name | age | +| name | age | |---------------|-----| | Jia Fan | 20 | | Hailin Wang | 20 | @@ -49,7 +53,7 @@ Determine whether someone is Chinese or American by their name 这将返回: -| name | age | llm_output | +| name | age | llm_output | |---------------|-----|------------| | Jia Fan | 20 | Chinese | | Hailin Wang | 20 | Chinese | @@ -59,16 +63,68 @@ Determine whether someone is Chinese or American by their name ### model 要使用的模型。不同的模型提供者有不同的模型。例如,OpenAI 模型可以是 `gpt-4o-mini`。 -如果使用 OpenAI 模型,请参考 https://platform.openai.com/docs/models/model-endpoint-compatibility 文档的`/v1/chat/completions` 端点。 +如果使用 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 +### api_path -用于 OpenAI 模型提供者的 API 路径。在大多数情况下,您不需要更改此配置。如果使用 API 代理的服务,您可能需要将其配置为代理的 API 地址。 +用于模型提供者的 API 路径。在大多数情况下,您不需要更改此配置。如果使用 API 代理的服务,您可能需要将其配置为代理的 API 地址。 + +### custom_config + +`custom_config` 选项允许您为模型提供额外的自定义配置。这是一个 Map,您可以在其中定义特定模型可能需要的各种设置。 + +### custom_response_parse + +`custom_response_parse` 选项允许您指定如何解析模型的响应。您可以使用 JsonPath +从响应中提取所需的特定数据。例如,使用 `$.choices[*].message.content` 提取如下json中的 `content` 字段 +值。JsonPath 的使用请参考 [JsonPath 快速入门](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started) + +```json +{ + "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" +} +``` + +### custom_request_headers + +`custom_request_headers` 选项允许您定义应包含在发送到模型 API 的请求中的自定义头信息。如果 API +需要标准头信息之外的额外头信息,例如授权令牌、内容类型等,这个选项会非常有用。 + +### custom_request_body + +`custom_request_body` 选项支持占位符: + +- `${model}`:用于模型名称的占位符。 +- `${input}`:用于确定输入值的占位符,同时根据 body value 的类型定义请求体请求类型。例如:`"${input}"` -> "input"。 +- `${prompt}`:用于 LLM 模型提示的占位符。 ### common options [string] @@ -118,3 +174,83 @@ sink { } ``` +### Customize the LLM model + +```hocon +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 = CUSTOM + 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" + custom_config={ + custom_response_parse = "$.choices[*].message.content" + custom_request_headers = { + Content-Type = "application/json" + Authorization = "Bearer xxxxxxxx" + } + custom_request_body ={ + model = "${model}" + messages = [ + { + role = "system" + content = "${prompt}" + }, + { + role = "user" + content = "${input}" + }] + } + } + 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 + } + ] + } + ] + } + } +} +``` + + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index a74b9e1223e..630d56654d4 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -85,6 +85,7 @@ seatunnel.sink.InfluxDB = connector-influxdb seatunnel.source.GoogleSheets = connector-google-sheets seatunnel.sink.GoogleFirestore = connector-google-firestore seatunnel.sink.Tablestore = connector-tablestore +seatunnel.source.Tablestore = connector-tablestore seatunnel.source.Lemlist = connector-http-lemlist seatunnel.source.Klaviyo = connector-http-klaviyo seatunnel.sink.Slack = connector-slack @@ -130,8 +131,12 @@ seatunnel.sink.ObsFile = connector-file-obs seatunnel.source.Milvus = connector-milvus seatunnel.sink.Milvus = connector-milvus seatunnel.sink.ActiveMQ = connector-activemq +seatunnel.source.Qdrant = connector-qdrant +seatunnel.sink.Qdrant = connector-qdrant seatunnel.source.Sls = connector-sls - +seatunnel.source.Typesense = connector-typesense +seatunnel.sink.Typesense = connector-typesense +seatunnel.source.Opengauss-CDC = connector-cdc-opengauss seatunnel.transform.Sql = seatunnel-transforms-v2 seatunnel.transform.FieldMapper = seatunnel-transforms-v2 seatunnel.transform.Filter = seatunnel-transforms-v2 @@ -142,3 +147,5 @@ seatunnel.transform.Split = seatunnel-transforms-v2 seatunnel.transform.Copy = seatunnel-transforms-v2 seatunnel.transform.DynamicCompile = seatunnel-transforms-v2 seatunnel.transform.LLM = seatunnel-transforms-v2 +seatunnel.transform.Embedding = seatunnel-transforms-v2 + diff --git a/pom.xml b/pom.xml index 3815a5c393d..043f8fc0d86 100644 --- a/pom.xml +++ b/pom.xml @@ -789,6 +789,8 @@ --no-cache -t ${docker.hub}/${docker.repo}:${docker.tag} + -t + ${docker.hub}/${docker.repo}:latest ${project.basedir} --build-arg VERSION=${project.version} @@ -842,8 +844,8 @@ linux/amd64,linux/arm64 --no-cache --push - -t - ${docker.hub}/${docker.repo}:${docker.tag} + --all-tags + ${docker.hub}/${docker.repo} ${project.basedir} --build-arg VERSION=${revision} diff --git a/release-note.md b/release-note.md index 32067c22dfa..4ed0d51fed9 100644 --- a/release-note.md +++ b/release-note.md @@ -58,6 +58,7 @@ - [Connector-v2] [Mongodb] Support to convert to double from numeric type that mongodb saved it as numeric internally (#6997) - [Connector-v2] [Redis] Using scan replace keys operation command,support batchWrite in single mode(#7030,#7085) - [Connector-V2] [Clickhouse] Add a new optional configuration `clickhouse.config` to the source connector of ClickHouse (#7143) +- [Connector-V2] [ElasticsSource] Source support multiSource (#6730) ### Zeta(ST-Engine) @@ -87,6 +88,7 @@ - [zeta] dynamically replace the value of the variable at runtime (#4950) - [Zeta] Add from_unixtime function (#5462) - [zeta] Fix CDC task restore throw NPE (#5507) +- [Zeta] Fix a checkpoint storage document with OSS (#7507) ### E2E 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 923ecff8b88..3f7f7fa9c6a 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 @@ -64,6 +64,7 @@ public String getPluginName() { public SinkWriter createWriter( SinkWriter.Context context) throws IOException { Map> writers = new HashMap<>(); + Map sinkWritersContext = new HashMap<>(); for (int i = 0; i < replicaNum; i++) { for (String tableIdentifier : sinks.keySet()) { SeaTunnelSink sink = sinks.get(tableIdentifier); @@ -71,15 +72,18 @@ public SinkWriter createWri writers.put( SinkIdentifier.of(tableIdentifier, index), sink.createWriter(new SinkContextProxy(index, context))); + sinkWritersContext.put(SinkIdentifier.of(tableIdentifier, index), context); } } - return new MultiTableSinkWriter(writers, replicaNum); + return new MultiTableSinkWriter(writers, replicaNum, sinkWritersContext); } @Override public SinkWriter restoreWriter( SinkWriter.Context context, List states) throws IOException { Map> writers = new HashMap<>(); + Map sinkWritersContext = new HashMap<>(); + for (int i = 0; i < replicaNum; i++) { for (String tableIdentifier : sinks.keySet()) { SeaTunnelSink sink = sinks.get(tableIdentifier); @@ -102,9 +106,10 @@ public SinkWriter restoreWr sinkIdentifier, sink.restoreWriter(new SinkContextProxy(index, context), state)); } + sinkWritersContext.put(SinkIdentifier.of(tableIdentifier, index), context); } } - return new MultiTableSinkWriter(writers, replicaNum); + return new MultiTableSinkWriter(writers, replicaNum, sinkWritersContext); } @Override diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java index 3c73435fafb..649417351ae 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java @@ -20,8 +20,10 @@ import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.sink.event.WriterCloseEvent; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.tracing.MDCTracer; import lombok.extern.slf4j.Slf4j; @@ -45,6 +47,7 @@ public class MultiTableSinkWriter implements SinkWriter { private final Map> sinkWriters; + private final Map sinkWritersContext; private final Map> sinkPrimaryKeys = new HashMap<>(); private final List>> sinkWritersWithIndex; private final List runnable = new ArrayList<>(); @@ -55,21 +58,28 @@ public class MultiTableSinkWriter private volatile boolean submitted = false; public MultiTableSinkWriter( - Map> sinkWriters, int queueSize) { + Map> sinkWriters, + int queueSize, + Map sinkWritersContext) { this.sinkWriters = sinkWriters; + this.sinkWritersContext = sinkWritersContext; AtomicInteger cnt = new AtomicInteger(0); executorService = - Executors.newFixedThreadPool( - // we use it in `MultiTableWriterRunnable` and `prepare commit task`, so it - // should be double. - queueSize * 2, - runnable -> { - Thread thread = new Thread(runnable); - thread.setDaemon(true); - thread.setName( - "st-multi-table-sink-writer" + "-" + cnt.incrementAndGet()); - return thread; - }); + MDCTracer.tracing( + Executors.newFixedThreadPool( + // we use it in `MultiTableWriterRunnable` and `prepare commit + // task`, so it + // should be double. + queueSize * 2, + runnable -> { + Thread thread = new Thread(runnable); + thread.setDaemon(true); + thread.setName( + "st-multi-table-sink-writer" + + "-" + + cnt.incrementAndGet()); + return thread; + })); sinkWritersWithIndex = new ArrayList<>(); for (int i = 0; i < queueSize; i++) { BlockingQueue queue = new LinkedBlockingQueue<>(1024); @@ -84,6 +94,7 @@ public MultiTableSinkWriter( entry.getKey().getTableIdentifier(), entry.getValue()); sinkIdentifierMap.put(entry.getKey(), entry.getValue()); }); + sinkWritersWithIndex.add(sinkIdentifierMap); blockingQueues.add(queue); MultiTableWriterRunnable r = new MultiTableWriterRunnable(tableIdWriterMap, queue); @@ -267,26 +278,34 @@ public void abortPrepare() { @Override public void close() throws IOException { - Throwable firstE = null; + // The variables used in lambda expressions should be final or valid final, so they are + // modified to arrays + final Throwable[] firstE = {null}; try { checkQueueRemain(); } catch (Exception e) { - firstE = e; + firstE[0] = e; } executorService.shutdownNow(); for (int i = 0; i < sinkWritersWithIndex.size(); i++) { synchronized (runnable.get(i)) { - for (SinkWriter sinkWriter : - sinkWritersWithIndex.get(i).values()) { - try { - sinkWriter.close(); - } catch (Throwable e) { - if (firstE == null) { - firstE = e; - } - log.error("close error", e); - } - } + Map> sinkIdentifierSinkWriterMap = + sinkWritersWithIndex.get(i); + sinkIdentifierSinkWriterMap.forEach( + (identifier, sinkWriter) -> { + try { + sinkWriter.close(); + sinkWritersContext + .get(identifier) + .getEventListener() + .onEvent(new WriterCloseEvent()); + } catch (Throwable e) { + if (firstE[0] == null) { + firstE[0] = e; + } + log.error("close error", e); + } + }); } } try { @@ -296,8 +315,8 @@ public void close() throws IOException { } catch (Throwable e) { log.error("close resourceManager error", e); } - if (firstE != null) { - throw new RuntimeException(firstE); + if (firstE[0] != null) { + throw new RuntimeException(firstE[0]); } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java index e043c0ecd72..ab85455b34e 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java @@ -95,7 +95,8 @@ public List parse(ReadonlyConfig schemaConfig) { String value = entry.getValue(); SeaTunnelDataType dataType = SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType(key, value); - PhysicalColumn column = PhysicalColumn.of(key, dataType, 0, true, null, null); + PhysicalColumn column = + PhysicalColumn.of(key, dataType, null, null, true, null, null); columns.add(column); } return columns; diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java index 801f037b186..10a5b33a935 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java @@ -295,6 +295,9 @@ private int getBytesForValue(Object v) { size += getBytesForValue(entry.getKey()) + getBytesForValue(entry.getValue()); } return size; + case "HeapByteBuffer": + case "ByteBuffer": + return ((ByteBuffer) v).capacity(); case "SeaTunnelRow": int rowSize = 0; SeaTunnelRow row = (SeaTunnelRow) v; diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java new file mode 100644 index 00000000000..f3cae160dab --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.tracing; + +import java.util.concurrent.Callable; + +/** + * Callable that sets MDC context before calling the delegate and clears it afterwards. + * + * @param + */ +public class MDCCallable implements Callable { + private final MDCContext context; + private final Callable delegate; + + public MDCCallable(Callable delegate) { + this(MDCContext.current(), delegate); + } + + public MDCCallable(MDCContext context, Callable delegate) { + this.context = context; + this.delegate = delegate; + } + + @Override + public V call() throws Exception { + try { + context.put(); + + return delegate.call(); + } finally { + context.clear(); + } + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java new file mode 100644 index 00000000000..4a439374590 --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.tracing; + +import org.slf4j.MDC; + +import lombok.Builder; +import lombok.EqualsAndHashCode; +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; + +/** + * MDC context for tracing. + * + *

reference: https://www.slf4j.org/manual.html#mdc + */ +@Slf4j +@Builder +@EqualsAndHashCode +public class MDCContext implements Serializable { + private static final MDCContext EMPTY = MDCContext.builder().build(); + private static final String EMPTY_TO_STRING = "NA"; + + public static final String JOB_ID = "ST-JID"; + public static final String PIPELINE_ID = "ST-PID"; + public static final String TASK_ID = "ST-TID"; + + private final Long jobId; + private final Long pipelineId; + private final Long taskId; + + public static MDCContext of(long jobId) { + return MDCContext.builder().jobId(jobId).build(); + } + + public static MDCContext of(long jobId, long pipelineId) { + return MDCContext.builder().jobId(jobId).pipelineId(pipelineId).build(); + } + + public static MDCContext of(long jobId, long pipelineId, long taskId) { + return MDCContext.builder().jobId(jobId).pipelineId(pipelineId).taskId(taskId).build(); + } + + public static MDCContext current() { + return MDCContext.builder() + .jobId(MDC.get(JOB_ID) != null ? Long.parseLong(MDC.get(JOB_ID)) : null) + .pipelineId( + MDC.get(PIPELINE_ID) != null ? Long.parseLong(MDC.get(PIPELINE_ID)) : null) + .taskId(MDC.get(TASK_ID) != null ? Long.parseLong(MDC.get(TASK_ID)) : null) + .build(); + } + + public static MDCContext valueOf(String s) { + if (EMPTY_TO_STRING.equals(s)) { + return EMPTY; + } + + String[] arr = s.split("/"); + Long jobId = Long.parseLong(arr[0]); + Long pipelineId = Long.parseLong(arr[1]); + Long taskId = Long.parseLong(arr[2]); + if (pipelineId == 0 || taskId == 0) { + return MDCContext.of(jobId); + } + return MDCContext.of(jobId, pipelineId, taskId); + } + + @Override + public String toString() { + if (jobId != null) { + return String.format( + "%d/%d/%d", + jobId, pipelineId == null ? 0 : pipelineId, taskId == null ? 0 : taskId); + } else { + return EMPTY_TO_STRING; + } + } + + public void put() { + try { + if (jobId != null) { + MDC.put(JOB_ID, String.valueOf(jobId)); + } + if (pipelineId != null) { + MDC.put(PIPELINE_ID, String.valueOf(pipelineId)); + } + if (taskId != null) { + MDC.put(TASK_ID, String.valueOf(taskId)); + } + } catch (Throwable e) { + log.error("Failed to put MDC context", e); + throw e; + } + } + + public void clear() { + try { + MDC.remove(JOB_ID); + MDC.remove(PIPELINE_ID); + MDC.remove(TASK_ID); + } catch (Throwable e) { + log.error("Failed to clear MDC context", e); + throw e; + } + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.java new file mode 100644 index 00000000000..1651f7d6c40 --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.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.api.tracing; + +import java.util.concurrent.Executor; + +/** Executor that sets MDC context before calling the delegate and clears it afterwards. */ +public class MDCExecutor implements Executor { + private final MDCContext context; + private final Executor delegate; + + public MDCExecutor(MDCContext context, Executor delegate) { + this.context = context; + this.delegate = delegate; + } + + @Override + public void execute(Runnable command) { + delegate.execute(new MDCRunnable(context, command)); + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java new file mode 100644 index 00000000000..7ef93f41dfc --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.tracing; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** ExecutorService that sets MDC context before calling the delegate and clears it afterwards. */ +public class MDCExecutorService extends MDCExecutor implements ExecutorService { + private final MDCContext context; + private final ExecutorService delegate; + + public MDCExecutorService(MDCContext context, ExecutorService delegate) { + super(context, delegate); + this.context = context; + this.delegate = delegate; + } + + @Override + public void shutdown() { + delegate.shutdown(); + } + + @Override + public List shutdownNow() { + return delegate.shutdownNow(); + } + + @Override + public boolean isShutdown() { + return delegate.isShutdown(); + } + + @Override + public boolean isTerminated() { + return delegate.isTerminated(); + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { + return delegate.awaitTermination(timeout, unit); + } + + @Override + public Future submit(Callable task) { + return delegate.submit(new MDCCallable<>(context, task)); + } + + @Override + public Future submit(Runnable task, T result) { + return delegate.submit(new MDCRunnable(context, task), result); + } + + @Override + public Future submit(Runnable task) { + return delegate.submit(new MDCRunnable(context, task)); + } + + @Override + public List> invokeAll(Collection> tasks) + throws InterruptedException { + return delegate.invokeAll( + tasks.stream() + .map(task -> new MDCCallable<>(context, task)) + .collect(Collectors.toList())); + } + + @Override + public List> invokeAll( + Collection> tasks, long timeout, TimeUnit unit) + throws InterruptedException { + return delegate.invokeAll( + tasks.stream() + .map(task -> new MDCCallable<>(context, task)) + .collect(Collectors.toList()), + timeout, + unit); + } + + @Override + public T invokeAny(Collection> tasks) + throws InterruptedException, ExecutionException { + return delegate.invokeAny( + tasks.stream() + .map(task -> new MDCCallable<>(context, task)) + .collect(Collectors.toList())); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + return delegate.invokeAny( + tasks.stream() + .map(task -> new MDCCallable<>(context, task)) + .collect(Collectors.toList()), + timeout, + unit); + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java new file mode 100644 index 00000000000..e6d310de10e --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.tracing; + +/** Runnable that sets MDC context before calling the delegate and clears it afterwards. */ +public class MDCRunnable implements Runnable { + private final MDCContext context; + private final Runnable delegate; + + public MDCRunnable(Runnable delegate) { + this(MDCContext.current(), delegate); + } + + public MDCRunnable(MDCContext context, Runnable delegate) { + this.context = context; + this.delegate = delegate; + } + + @Override + public void run() { + try { + context.put(); + + delegate.run(); + } finally { + context.clear(); + } + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java new file mode 100644 index 00000000000..804e953aced --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.tracing; + +import java.util.concurrent.Callable; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + * ScheduledExecutorService that sets MDC context before calling the delegate and clears it + * afterwards. + */ +public class MDCScheduledExecutorService extends MDCExecutorService + implements ScheduledExecutorService { + private final MDCContext context; + private final ScheduledExecutorService delegate; + + public MDCScheduledExecutorService(MDCContext context, ScheduledExecutorService delegate) { + super(context, delegate); + this.context = context; + this.delegate = delegate; + } + + @Override + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { + return delegate.schedule(new MDCRunnable(context, command), delay, unit); + } + + @Override + public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { + return delegate.schedule(new MDCCallable<>(context, callable), delay, unit); + } + + @Override + public ScheduledFuture scheduleAtFixedRate( + Runnable command, long initialDelay, long period, TimeUnit unit) { + return delegate.scheduleAtFixedRate( + new MDCRunnable(context, command), initialDelay, period, unit); + } + + @Override + public ScheduledFuture scheduleWithFixedDelay( + Runnable command, long initialDelay, long delay, TimeUnit unit) { + return delegate.scheduleWithFixedDelay( + new MDCRunnable(context, command), initialDelay, delay, unit); + } +} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java new file mode 100644 index 00000000000..1ce2120cc3c --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.tracing; + +import java.util.concurrent.Callable; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; + +/** + * Tracer for MDC context. + * + *

It wraps the given {@link Runnable}, {@link Callable}, {@link Executor}, {@link + * ExecutorService}, {@link ScheduledExecutorService} to trace the MDC context. + * + *

It is useful to trace the MDC context in the asynchronous execution. For example, when you + * submit a task to the {@link ExecutorService}, the MDC context is not propagated to the worker + * thread. + * + *

It is recommended to use the {@link MDCTracer} to wrap the task to trace the MDC context. + * + *

{@code
+ * MDCContext mdcContext = MDCContext.of(1);
+ * ExecutorService executorService = Executors.newFixedThreadPool(10);
+ * executorService.submit(MDCTracer.tracing(mdcContext, () -> {
+ *    // Your task
+ *    logger.info("Task is running");
+ *    return null;
+ *    }));
+ *
+ * }
+ */ +public class MDCTracer { + + public static MDCRunnable tracing(Runnable delegate) { + return tracing(MDCContext.current(), delegate); + } + + public static MDCRunnable tracing(Long jobId, Runnable delegate) { + return tracing(MDCContext.of(jobId), delegate); + } + + public static MDCRunnable tracing(MDCContext context, Runnable delegate) { + if (delegate instanceof MDCRunnable) { + throw new IllegalArgumentException("Already an MDCRunnable"); + } + return new MDCRunnable(context, delegate); + } + + public static MDCCallable tracing(Callable delegate) { + return tracing(MDCContext.current(), delegate); + } + + public static MDCCallable tracing(Long jobId, Callable delegate) { + return tracing(MDCContext.of(jobId), delegate); + } + + public static MDCCallable tracing(MDCContext context, Callable delegate) { + if (delegate instanceof MDCCallable) { + throw new IllegalArgumentException("Already an MDCCallable"); + } + return new MDCCallable<>(context, delegate); + } + + public static MDCExecutor tracing(Executor delegate) { + return tracing(MDCContext.current(), delegate); + } + + public static MDCExecutor tracing(Long jobId, Executor delegate) { + return tracing(MDCContext.of(jobId), delegate); + } + + public static MDCExecutor tracing(MDCContext context, Executor delegate) { + if (delegate instanceof MDCExecutor) { + throw new IllegalArgumentException("Already an MDCExecutor"); + } + return new MDCExecutor(context, delegate); + } + + public static MDCExecutorService tracing(ExecutorService delegate) { + return tracing(MDCContext.current(), delegate); + } + + public static MDCExecutorService tracing(Long jobId, ExecutorService delegate) { + return tracing(MDCContext.of(jobId), delegate); + } + + public static MDCExecutorService tracing(MDCContext context, ExecutorService delegate) { + if (delegate instanceof MDCExecutor) { + throw new IllegalArgumentException("Already an MDCExecutor"); + } + return new MDCExecutorService(context, delegate); + } + + public static MDCScheduledExecutorService tracing(ScheduledExecutorService delegate) { + return tracing(MDCContext.current(), delegate); + } + + public static MDCScheduledExecutorService tracing( + Long jobId, ScheduledExecutorService delegate) { + return tracing(MDCContext.of(jobId), delegate); + } + + public static MDCScheduledExecutorService tracing( + MDCContext context, ScheduledExecutorService delegate) { + if (delegate instanceof MDCExecutor) { + throw new IllegalArgumentException("Already an MDCExecutor"); + } + return new MDCScheduledExecutorService(context, delegate); + } +} diff --git a/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java b/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java new file mode 100644 index 00000000000..88ac95885f6 --- /dev/null +++ b/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.tracing; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.slf4j.MDC; + +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; + +public class MDCTracerTest { + + @Test + public void testMDCTracedRunnable() { + MDCContext mdcContext = MDCContext.of(1, 2, 3); + Runnable tracedRunnable = + MDCTracer.tracing( + mdcContext, + new Runnable() { + @Override + public void run() { + Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); + Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); + } + }); + + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + + tracedRunnable.run(); + + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + } + + @Test + public void testMDCTracedCallable() throws Exception { + MDCContext mdcContext = MDCContext.of(1, 2, 3); + + Callable tracedCallable = + MDCTracer.tracing( + mdcContext, + new Callable() { + @Override + public Void call() throws Exception { + Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); + Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); + return null; + } + }); + + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + + tracedCallable.call(); + + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + } + + @Test + public void testMDCTracedExecutorService() throws Exception { + MDCContext mdcContext = MDCContext.of(1, 2, 3); + + MDCExecutorService tracedExecutorService = + MDCTracer.tracing(mdcContext, Executors.newSingleThreadExecutor()); + + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + tracedExecutorService + .submit( + new Runnable() { + @Override + public void run() { + Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); + Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); + } + }) + .get(); + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + tracedExecutorService + .submit( + new Callable() { + @Override + public Void call() throws Exception { + Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); + Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); + return null; + } + }) + .get(); + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + } + + @Test + public void testMDCContext() throws Exception { + MDCContext.current(); + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + + MDCContext mdcContext = MDCContext.of(1, 2, 3); + mdcContext.put(); + Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); + Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); + + MDCContext currentMDCCOntext = MDCContext.current(); + Assertions.assertEquals(mdcContext, currentMDCCOntext); + + mdcContext.clear(); + Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); + Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); + Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); + } +} 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 e9adf4d70a1..0148e651884 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 @@ -266,8 +266,8 @@ public static SeaTunnelRuntimeException writeRowErrorWithFiledsCountNotMatch( String connector, int sourceFieldsNum, int sinkFieldsNum) { Map params = new HashMap<>(); params.put("connector", connector); - params.put("sourceFiledName", String.valueOf(sourceFieldsNum)); - params.put("sourceFiledType", String.valueOf(sinkFieldsNum)); + params.put("sourceFieldsNum", String.valueOf(sourceFieldsNum)); + params.put("sinkFieldsNum", String.valueOf(sinkFieldsNum)); return new SeaTunnelRuntimeException( WRITE_SEATUNNEL_ROW_ERROR_WITH_FILEDS_NOT_MATCH, params); } diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java new file mode 100644 index 00000000000..d5ddd48106b --- /dev/null +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.common.utils; + +public final class TemporaryClassLoaderContext implements AutoCloseable { + + /** + * Sets the context class loader to the given ClassLoader and returns a resource that sets it + * back to the current context ClassLoader when the resource is closed. + * + *
{@code
+     * try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(classloader)) {
+     *     // code that needs the context class loader
+     * }
+     * }
+ */ + public static TemporaryClassLoaderContext of(ClassLoader cl) { + final Thread t = Thread.currentThread(); + final ClassLoader original = t.getContextClassLoader(); + + t.setContextClassLoader(cl); + + return new TemporaryClassLoaderContext(t, original); + } + + private final Thread thread; + + private final ClassLoader originalContextClassLoader; + + private TemporaryClassLoaderContext(Thread thread, ClassLoader originalContextClassLoader) { + this.thread = thread; + this.originalContextClassLoader = originalContextClassLoader; + } + + @Override + public void close() { + thread.setContextClassLoader(originalContextClassLoader); + } +} diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java index d60fb46c8c9..f6908b989d2 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java @@ -36,6 +36,7 @@ import java.io.IOException; import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -307,6 +308,13 @@ private Boolean checkType(Object value, SeaTunnelDataType fieldType) { return checkDecimalType(value, fieldType); } + if (fieldType.getSqlType() == SqlType.FLOAT_VECTOR + || fieldType.getSqlType() == SqlType.FLOAT16_VECTOR + || fieldType.getSqlType() == SqlType.BFLOAT16_VECTOR + || fieldType.getSqlType() == SqlType.BINARY_VECTOR) { + return value instanceof ByteBuffer; + } + return value.getClass().equals(fieldType.getTypeClass()); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml new file mode 100644 index 00000000000..098c60370d4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml @@ -0,0 +1,91 @@ + + + + 4.0.0 + + org.apache.seatunnel + connector-cdc + ${revision} + + + connector-cdc-opengauss + SeaTunnel : Connectors V2 : CDC : Opengauss + + + 5.1.0 + + + + + org.opengauss + opengauss-jdbc + ${opengauss.version} + + + + org.apache.seatunnel + connector-cdc-postgres + ${project.version} + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + shade + + package + + false + true + false + false + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.postgresql + ${seatunnel.shade.package}.org.postgresql + + + + + + + + + + diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java new file mode 100644 index 00000000000..57c393acfaf --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java @@ -0,0 +1,815 @@ +/* + * 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 io.debezium.connector.postgresql.connection; + +import org.apache.kafka.connect.errors.ConnectException; + +import org.postgresql.core.BaseConnection; +import org.postgresql.jdbc.PgConnection; +import org.postgresql.jdbc.TimestampUtils; +import org.postgresql.replication.LogSequenceNumber; +import org.postgresql.util.PGmoney; +import org.postgresql.util.PSQLState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.DebeziumException; +import io.debezium.annotation.VisibleForTesting; +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.PgOid; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresSchema; +import io.debezium.connector.postgresql.PostgresType; +import io.debezium.connector.postgresql.PostgresValueConverter; +import io.debezium.connector.postgresql.TypeRegistry; +import io.debezium.connector.postgresql.spi.SlotState; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.ColumnEditor; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.schema.DatabaseSchema; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; + +import java.nio.charset.Charset; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Copied from Debezium 1.9.8.Final. {@link JdbcConnection} connection extension used for connecting + * to Postgres instances. + * + *

Line 616 : skip validateServerVersion because the version based pg of opengauss is below 9.4 + */ +public class PostgresConnection extends JdbcConnection { + + public static final String CONNECTION_STREAMING = "Debezium Streaming"; + public static final String CONNECTION_SLOT_INFO = "Debezium Slot Info"; + public static final String CONNECTION_DROP_SLOT = "Debezium Drop Slot"; + public static final String CONNECTION_VALIDATE_CONNECTION = "Debezium Validate Connection"; + public static final String CONNECTION_HEARTBEAT = "Debezium Heartbeat"; + public static final String CONNECTION_GENERAL = "Debezium General"; + + private static Logger LOGGER = LoggerFactory.getLogger(PostgresConnection.class); + + private static final String URL_PATTERN = + "jdbc:postgresql://${" + + JdbcConfiguration.HOSTNAME + + "}:${" + + JdbcConfiguration.PORT + + "}/${" + + JdbcConfiguration.DATABASE + + "}"; + protected static final ConnectionFactory FACTORY = + JdbcConnection.patternBasedFactory( + URL_PATTERN, + org.postgresql.Driver.class.getName(), + PostgresConnection.class.getClassLoader(), + JdbcConfiguration.PORT.withDefault( + PostgresConnectorConfig.PORT.defaultValueAsString())); + + /** + * Obtaining a replication slot may fail if there's a pending transaction. We're retrying to get + * a slot for 30 min. + */ + private static final int MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT = 900; + + private static final Duration PAUSE_BETWEEN_REPLICATION_SLOT_RETRIEVAL_ATTEMPTS = + Duration.ofSeconds(2); + + private final TypeRegistry typeRegistry; + private final PostgresDefaultValueConverter defaultValueConverter; + + /** + * Creates a Postgres connection using the supplied configuration. If necessary this connection + * is able to resolve data type mappings. Such a connection requires a {@link + * PostgresValueConverter}, and will provide its own {@link TypeRegistry}. Usually only one such + * connection per connector is needed. + * + * @param config {@link Configuration} instance, may not be null. + * @param valueConverterBuilder supplies a configured {@link PostgresValueConverter} for a given + * {@link TypeRegistry} + * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools + */ + public PostgresConnection( + JdbcConfiguration config, + PostgresValueConverterBuilder valueConverterBuilder, + String connectionUsage) { + super( + addDefaultSettings(config, connectionUsage), + FACTORY, + PostgresConnection::validateServerVersion, + null, + "\"", + "\""); + + if (Objects.isNull(valueConverterBuilder)) { + this.typeRegistry = null; + this.defaultValueConverter = null; + } else { + this.typeRegistry = new TypeRegistry(this); + + final PostgresValueConverter valueConverter = + valueConverterBuilder.build(this.typeRegistry); + this.defaultValueConverter = + new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils()); + } + } + + /** + * Create a Postgres connection using the supplied configuration and {@link TypeRegistry} + * + * @param config {@link Configuration} instance, may not be null. + * @param typeRegistry an existing/already-primed {@link TypeRegistry} instance + * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools + */ + public PostgresConnection( + PostgresConnectorConfig config, TypeRegistry typeRegistry, String connectionUsage) { + super( + addDefaultSettings(config.getJdbcConfig(), connectionUsage), + FACTORY, + PostgresConnection::validateServerVersion, + null, + "\"", + "\""); + if (Objects.isNull(typeRegistry)) { + this.typeRegistry = null; + this.defaultValueConverter = null; + } else { + this.typeRegistry = typeRegistry; + final PostgresValueConverter valueConverter = + PostgresValueConverter.of(config, this.getDatabaseCharset(), typeRegistry); + this.defaultValueConverter = + new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils()); + } + } + + /** + * Creates a Postgres connection using the supplied configuration. The connector is the regular + * one without datatype resolution capabilities. + * + * @param config {@link Configuration} instance, may not be null. + * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools + */ + public PostgresConnection(JdbcConfiguration config, String connectionUsage) { + this(config, null, connectionUsage); + } + + static JdbcConfiguration addDefaultSettings( + JdbcConfiguration configuration, String connectionUsage) { + // we require Postgres 9.4 as the minimum server version since that's where logical + // replication was first introduced + return JdbcConfiguration.adapt( + configuration + .edit() + .with("assumeMinServerVersion", "9.4") + .with("ApplicationName", connectionUsage) + .build()); + } + + /** + * Returns a JDBC connection string for the current configuration. + * + * @return a {@code String} where the variables in {@code urlPattern} are replaced with values + * from the configuration + */ + public String connectionString() { + return connectionString(URL_PATTERN); + } + + /** + * Prints out information about the REPLICA IDENTITY status of a table. This in turn determines + * how much information is available for UPDATE and DELETE operations for logical replication. + * + * @param tableId the identifier of the table + * @return the replica identity information; never null + * @throws SQLException if there is a problem obtaining the replica identity information for the + * given table + */ + public ServerInfo.ReplicaIdentity readReplicaIdentityInfo(TableId tableId) throws SQLException { + String statement = + "SELECT relreplident FROM pg_catalog.pg_class c " + + "LEFT JOIN pg_catalog.pg_namespace n ON c.relnamespace=n.oid " + + "WHERE n.nspname=? and c.relname=?"; + String schema = + tableId.schema() != null && tableId.schema().length() > 0 + ? tableId.schema() + : "public"; + StringBuilder replIdentity = new StringBuilder(); + prepareQuery( + statement, + stmt -> { + stmt.setString(1, schema); + stmt.setString(2, tableId.table()); + }, + rs -> { + if (rs.next()) { + replIdentity.append(rs.getString(1)); + } else { + LOGGER.warn( + "Cannot determine REPLICA IDENTITY information for table '{}'", + tableId); + } + }); + return ServerInfo.ReplicaIdentity.parseFromDB(replIdentity.toString()); + } + + /** + * Returns the current state of the replication slot + * + * @param slotName the name of the slot + * @param pluginName the name of the plugin used for the desired slot + * @return the {@link SlotState} or null, if no slot state is found + * @throws SQLException + */ + public SlotState getReplicationSlotState(String slotName, String pluginName) + throws SQLException { + ServerInfo.ReplicationSlot slot; + try { + slot = readReplicationSlotInfo(slotName, pluginName); + if (slot.equals(ServerInfo.ReplicationSlot.INVALID)) { + return null; + } else { + return slot.asSlotState(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ConnectException( + "Interrupted while waiting for valid replication slot info", e); + } + } + + /** + * Fetches the state of a replication stage given a slot name and plugin name + * + * @param slotName the name of the slot + * @param pluginName the name of the plugin used for the desired slot + * @return the {@link ServerInfo.ReplicationSlot} object or a {@link + * ServerInfo.ReplicationSlot#INVALID} if the slot is not valid + * @throws SQLException is thrown by the underlying JDBC + */ + private ServerInfo.ReplicationSlot fetchReplicationSlotInfo(String slotName, String pluginName) + throws SQLException { + final String database = database(); + final ServerInfo.ReplicationSlot slot = + queryForSlot( + slotName, + database, + pluginName, + rs -> { + if (rs.next()) { + boolean active = rs.getBoolean("active"); + final Lsn confirmedFlushedLsn = + parseConfirmedFlushLsn(slotName, pluginName, database, rs); + if (confirmedFlushedLsn == null) { + return null; + } + Lsn restartLsn = + parseRestartLsn(slotName, pluginName, database, rs); + if (restartLsn == null) { + return null; + } + final Long xmin = rs.getLong("catalog_xmin"); + return new ServerInfo.ReplicationSlot( + active, confirmedFlushedLsn, restartLsn, xmin); + } else { + LOGGER.debug( + "No replication slot '{}' is present for plugin '{}' and database '{}'", + slotName, + pluginName, + database); + return ServerInfo.ReplicationSlot.INVALID; + } + }); + return slot; + } + + /** + * Fetches a replication slot, repeating the query until either the slot is created or until the + * max number of attempts has been reached + * + *

To fetch the slot without the retries, use the {@link + * PostgresConnection#fetchReplicationSlotInfo} call + * + * @param slotName the slot name + * @param pluginName the name of the plugin + * @return the {@link ServerInfo.ReplicationSlot} object or a {@link + * ServerInfo.ReplicationSlot#INVALID} if the slot is not valid + * @throws SQLException is thrown by the underyling jdbc driver + * @throws InterruptedException is thrown if we don't return an answer within the set number of + * retries + */ + @VisibleForTesting + ServerInfo.ReplicationSlot readReplicationSlotInfo(String slotName, String pluginName) + throws SQLException, InterruptedException { + final String database = database(); + final Metronome metronome = + Metronome.parker(PAUSE_BETWEEN_REPLICATION_SLOT_RETRIEVAL_ATTEMPTS, Clock.SYSTEM); + + for (int attempt = 1; attempt <= MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT; attempt++) { + final ServerInfo.ReplicationSlot slot = fetchReplicationSlotInfo(slotName, pluginName); + if (slot != null) { + LOGGER.info("Obtained valid replication slot {}", slot); + return slot; + } + LOGGER.warn( + "Cannot obtain valid replication slot '{}' for plugin '{}' and database '{}' [during attempt {} out of {}, concurrent tx probably blocks taking snapshot.", + slotName, + pluginName, + database, + attempt, + MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT); + metronome.pause(); + } + + throw new ConnectException( + "Unable to obtain valid replication slot. " + + "Make sure there are no long-running transactions running in parallel as they may hinder the allocation of the replication slot when starting this connector"); + } + + protected ServerInfo.ReplicationSlot queryForSlot( + String slotName, + String database, + String pluginName, + ResultSetMapper map) + throws SQLException { + return prepareQueryAndMap( + "select * from pg_replication_slots where slot_name = ? and database = ? and plugin = ?", + statement -> { + statement.setString(1, slotName); + statement.setString(2, database); + statement.setString(3, pluginName); + }, + map); + } + + /** + * Obtains the LSN to resume streaming from. On PG 9.5 there is no confirmed_flushed_lsn yet, so + * restart_lsn will be read instead. This may result in more records to be re-read after a + * restart. + */ + private Lsn parseConfirmedFlushLsn( + String slotName, String pluginName, String database, ResultSet rs) { + Lsn confirmedFlushedLsn = null; + + try { + confirmedFlushedLsn = + tryParseLsn(slotName, pluginName, database, rs, "confirmed_flush_lsn"); + } catch (SQLException e) { + LOGGER.info("unable to find confirmed_flushed_lsn, falling back to restart_lsn"); + try { + confirmedFlushedLsn = + tryParseLsn(slotName, pluginName, database, rs, "restart_lsn"); + } catch (SQLException e2) { + throw new ConnectException( + "Neither confirmed_flush_lsn nor restart_lsn could be found"); + } + } + + return confirmedFlushedLsn; + } + + private Lsn parseRestartLsn(String slotName, String pluginName, String database, ResultSet rs) { + Lsn restartLsn = null; + try { + restartLsn = tryParseLsn(slotName, pluginName, database, rs, "restart_lsn"); + } catch (SQLException e) { + throw new ConnectException("restart_lsn could be found"); + } + + return restartLsn; + } + + private Lsn tryParseLsn( + String slotName, String pluginName, String database, ResultSet rs, String column) + throws ConnectException, SQLException { + Lsn lsn = null; + + String lsnStr = rs.getString(column); + if (lsnStr == null) { + return null; + } + try { + lsn = Lsn.valueOf(lsnStr); + } catch (Exception e) { + throw new ConnectException( + "Value " + + column + + " in the pg_replication_slots table for slot = '" + + slotName + + "', plugin = '" + + pluginName + + "', database = '" + + database + + "' is not valid. This is an abnormal situation and the database status should be checked."); + } + if (!lsn.isValid()) { + throw new ConnectException("Invalid LSN returned from database"); + } + return lsn; + } + + /** + * Drops a replication slot that was created on the DB + * + * @param slotName the name of the replication slot, may not be null + * @return {@code true} if the slot was dropped, {@code false} otherwise + */ + public boolean dropReplicationSlot(String slotName) { + final int ATTEMPTS = 3; + for (int i = 0; i < ATTEMPTS; i++) { + try { + execute("select pg_drop_replication_slot('" + slotName + "')"); + return true; + } catch (SQLException e) { + // slot is active + if (PSQLState.OBJECT_IN_USE.getState().equals(e.getSQLState())) { + if (i < ATTEMPTS - 1) { + LOGGER.debug( + "Cannot drop replication slot '{}' because it's still in use", + slotName); + } else { + LOGGER.warn( + "Cannot drop replication slot '{}' because it's still in use", + slotName); + return false; + } + } else if (PSQLState.UNDEFINED_OBJECT.getState().equals(e.getSQLState())) { + LOGGER.debug("Replication slot {} has already been dropped", slotName); + return false; + } else { + LOGGER.error("Unexpected error while attempting to drop replication slot", e); + return false; + } + } + try { + Metronome.parker(Duration.ofSeconds(1), Clock.system()).pause(); + } catch (InterruptedException e) { + } + } + return false; + } + + /** + * Drops the debezium publication that was created. + * + * @param publicationName the publication name, may not be null + * @return {@code true} if the publication was dropped, {@code false} otherwise + */ + public boolean dropPublication(String publicationName) { + try { + LOGGER.debug("Dropping publication '{}'", publicationName); + execute("DROP PUBLICATION " + publicationName); + return true; + } catch (SQLException e) { + if (PSQLState.UNDEFINED_OBJECT.getState().equals(e.getSQLState())) { + LOGGER.debug("Publication {} has already been dropped", publicationName); + } else { + LOGGER.error("Unexpected error while attempting to drop publication", e); + } + return false; + } + } + + @Override + public synchronized void close() { + try { + super.close(); + } catch (SQLException e) { + LOGGER.error("Unexpected error while closing Postgres connection", e); + } + } + + /** + * Returns the PG id of the current active transaction + * + * @return a PG transaction identifier, or null if no tx is active + * @throws SQLException if anything fails. + */ + public Long currentTransactionId() throws SQLException { + AtomicLong txId = new AtomicLong(0); + query( + "select (case pg_is_in_recovery() when 't' then 0 else txid_current() end) AS pg_current_txid", + rs -> { + if (rs.next()) { + txId.compareAndSet(0, rs.getLong(1)); + } + }); + long value = txId.get(); + return value > 0 ? value : null; + } + + /** + * Returns the current position in the server tx log. + * + * @return a long value, never negative + * @throws SQLException if anything unexpected fails. + */ + public long currentXLogLocation() throws SQLException { + AtomicLong result = new AtomicLong(0); + int majorVersion = connection().getMetaData().getDatabaseMajorVersion(); + query( + majorVersion >= 10 + ? "select (case pg_is_in_recovery() when 't' then pg_last_wal_receive_lsn() else pg_current_wal_lsn() end) AS pg_current_wal_lsn" + : "select * from pg_current_xlog_location()", + rs -> { + if (!rs.next()) { + throw new IllegalStateException( + "there should always be a valid xlog position"); + } + result.compareAndSet(0, LogSequenceNumber.valueOf(rs.getString(1)).asLong()); + }); + return result.get(); + } + + /** + * Returns information about the PG server to which this instance is connected. + * + * @return a {@link ServerInfo} instance, never {@code null} + * @throws SQLException if anything fails + */ + public ServerInfo serverInfo() throws SQLException { + ServerInfo serverInfo = new ServerInfo(); + query( + "SELECT version(), current_user, current_database()", + rs -> { + if (rs.next()) { + serverInfo + .withServer(rs.getString(1)) + .withUsername(rs.getString(2)) + .withDatabase(rs.getString(3)); + } + }); + String username = serverInfo.username(); + if (username != null) { + query( + "SELECT oid, rolname, rolsuper, rolinherit, rolcreaterole, rolcreatedb, rolcanlogin, rolreplication FROM pg_roles " + + "WHERE pg_has_role('" + + username + + "', oid, 'member')", + rs -> { + while (rs.next()) { + String roleInfo = + "superuser: " + + rs.getBoolean(3) + + ", replication: " + + rs.getBoolean(8) + + ", inherit: " + + rs.getBoolean(4) + + ", create role: " + + rs.getBoolean(5) + + ", create db: " + + rs.getBoolean(6) + + ", can log in: " + + rs.getBoolean(7); + String roleName = rs.getString(2); + serverInfo.addRole(roleName, roleInfo); + } + }); + } + return serverInfo; + } + + public Charset getDatabaseCharset() { + try { + return Charset.forName(((BaseConnection) connection()).getEncoding().name()); + } catch (SQLException e) { + throw new DebeziumException("Couldn't obtain encoding for database " + database(), e); + } + } + + public TimestampUtils getTimestampUtils() { + try { + return ((PgConnection) this.connection()).getTimestampUtils(); + } catch (SQLException e) { + throw new DebeziumException( + "Couldn't get timestamp utils from underlying connection", e); + } + } + + private static void validateServerVersion(Statement statement) throws SQLException {} + + @Override + public String quotedColumnIdString(String columnName) { + if (columnName.contains("\"")) { + columnName = columnName.replaceAll("\"", "\"\""); + } + + return super.quotedColumnIdString(columnName); + } + + @Override + protected int resolveNativeType(String typeName) { + return getTypeRegistry().get(typeName).getRootType().getOid(); + } + + @Override + protected int resolveJdbcType(int metadataJdbcType, int nativeType) { + // Special care needs to be taken for columns that use user-defined domain type data types + // where resolution of the column's JDBC type needs to be that of the root type instead of + // the actual column to properly influence schema building and value conversion. + return getTypeRegistry().get(nativeType).getRootType().getJdbcId(); + } + + @Override + protected Optional readTableColumn( + ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnFilter) + throws SQLException { + return doReadTableColumn(columnMetadata, tableId, columnFilter); + } + + public Optional readColumnForDecoder( + ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnNameFilter) + throws SQLException { + return doReadTableColumn(columnMetadata, tableId, columnNameFilter) + .map(ColumnEditor::create); + } + + private Optional doReadTableColumn( + ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnFilter) + throws SQLException { + final String columnName = columnMetadata.getString(4); + if (columnFilter == null + || columnFilter.matches( + tableId.catalog(), tableId.schema(), tableId.table(), columnName)) { + final ColumnEditor column = Column.editor().name(columnName); + column.type(columnMetadata.getString(6)); + + // first source the length/scale from the column metadata provided by the driver + // this may be overridden below if the column type is a user-defined domain type + column.length(columnMetadata.getInt(7)); + if (columnMetadata.getObject(9) != null) { + column.scale(columnMetadata.getInt(9)); + } + + column.optional(isNullable(columnMetadata.getInt(11))); + column.position(columnMetadata.getInt(17)); + column.autoIncremented("YES".equalsIgnoreCase(columnMetadata.getString(23))); + + String autogenerated = null; + try { + autogenerated = columnMetadata.getString(24); + } catch (SQLException e) { + // ignore, some drivers don't have this index - e.g. Postgres + } + column.generated("YES".equalsIgnoreCase(autogenerated)); + + // Lookup the column type from the TypeRegistry + // For all types, we need to set the Native and Jdbc types by using the root-type + final PostgresType nativeType = getTypeRegistry().get(column.typeName()); + column.nativeType(nativeType.getRootType().getOid()); + column.jdbcType(nativeType.getRootType().getJdbcId()); + + // For domain types, the postgres driver is unable to traverse a nested unbounded + // hierarchy of types and report the right length/scale of a given type. We use + // the TypeRegistry to accomplish this since it is capable of traversing the type + // hierarchy upward to resolve length/scale regardless of hierarchy depth. + if (TypeRegistry.DOMAIN_TYPE == nativeType.getJdbcId()) { + column.length(nativeType.getDefaultLength()); + column.scale(nativeType.getDefaultScale()); + } + + final String defaultValueExpression = columnMetadata.getString(13); + if (defaultValueExpression != null + && getDefaultValueConverter().supportConversion(column.typeName())) { + column.defaultValueExpression(defaultValueExpression); + } + + return Optional.of(column); + } + + return Optional.empty(); + } + + public PostgresDefaultValueConverter getDefaultValueConverter() { + Objects.requireNonNull( + defaultValueConverter, "Connection does not provide default value converter"); + return defaultValueConverter; + } + + public TypeRegistry getTypeRegistry() { + Objects.requireNonNull(typeRegistry, "Connection does not provide type registry"); + return typeRegistry; + } + + @Override + public > Object getColumnValue( + ResultSet rs, int columnIndex, Column column, Table table, T schema) + throws SQLException { + try { + final ResultSetMetaData metaData = rs.getMetaData(); + final String columnTypeName = metaData.getColumnTypeName(columnIndex); + final PostgresType type = + ((PostgresSchema) schema).getTypeRegistry().get(columnTypeName); + + LOGGER.trace("Type of incoming data is: {}", type.getOid()); + LOGGER.trace("ColumnTypeName is: {}", columnTypeName); + LOGGER.trace("Type is: {}", type); + + if (type.isArrayType()) { + return rs.getArray(columnIndex); + } + + switch (type.getOid()) { + case PgOid.MONEY: + // TODO author=Horia Chiorean date=14/11/2016 description=workaround for + // https://github.com/pgjdbc/pgjdbc/issues/100 + final String sMoney = rs.getString(columnIndex); + if (sMoney == null) { + return sMoney; + } + if (sMoney.startsWith("-")) { + // PGmoney expects negative values to be provided in the format of + // "($XXXXX.YY)" + final String negativeMoney = "(" + sMoney.substring(1) + ")"; + return new PGmoney(negativeMoney).val; + } + return new PGmoney(sMoney).val; + case PgOid.BIT: + return rs.getString(columnIndex); + case PgOid.NUMERIC: + final String s = rs.getString(columnIndex); + if (s == null) { + return s; + } + + Optional value = PostgresValueConverter.toSpecialValue(s); + return value.isPresent() + ? value.get() + : new SpecialValueDecimal(rs.getBigDecimal(columnIndex)); + case PgOid.TIME: + // To handle time 24:00:00 supported by TIME columns, read the column as a + // string. + case PgOid.TIMETZ: + // In order to guarantee that we resolve TIMETZ columns with proper microsecond + // precision, + // read the column as a string instead and then re-parse inside the converter. + return rs.getString(columnIndex); + default: + Object x = rs.getObject(columnIndex); + if (x != null) { + LOGGER.trace( + "rs getobject returns class: {}; rs getObject value is: {}", + x.getClass(), + x); + } + return x; + } + } catch (SQLException e) { + // not a known type + return super.getColumnValue(rs, columnIndex, column, table, schema); + } + } + + @Override + protected String[] supportedTableTypes() { + return new String[] {"VIEW", "MATERIALIZED VIEW", "TABLE", "PARTITIONED TABLE"}; + } + + @Override + protected boolean isTableType(String tableType) { + return "TABLE".equals(tableType) || "PARTITIONED TABLE".equals(tableType); + } + + /** + * Retrieves all {@code TableId}s in a given database catalog, including partitioned tables. + * + * @param catalogName the catalog/database name + * @return set of all table ids for existing table objects + * @throws SQLException if a database exception occurred + */ + public Set getAllTableIds(String catalogName) throws SQLException { + return readTableNames(catalogName, null, null, new String[] {"TABLE", "PARTITIONED TABLE"}); + } + + @FunctionalInterface + public interface PostgresValueConverterBuilder { + PostgresValueConverter build(TypeRegistry registry); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java new file mode 100644 index 00000000000..c69a63c7136 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java @@ -0,0 +1,928 @@ +/* + * 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 io.debezium.connector.postgresql.connection; + +import org.apache.kafka.connect.errors.ConnectException; + +import org.postgresql.core.BaseConnection; +import org.postgresql.core.ServerVersion; +import org.postgresql.replication.PGReplicationStream; +import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder; +import org.postgresql.util.PSQLException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.DebeziumException; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresSchema; +import io.debezium.connector.postgresql.TypeRegistry; +import io.debezium.connector.postgresql.spi.SlotCreationResult; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.jdbc.JdbcConnectionException; +import io.debezium.relational.RelationalTableFilters; +import io.debezium.relational.TableId; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; + +import java.nio.ByteBuffer; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLWarning; +import java.sql.Statement; +import java.time.Duration; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.lang.Math.toIntExact; + +/** + * Copied from Debezium 1.9.8.Final. Implementation of a {@link ReplicationConnection} for + * Postgresql. Note that replication connections in PG cannot execute regular statements but only a + * limited number of replication-related commands. + * + *

Line 179 : Modify the method named initPublication which we use the regular - i.e. not a + * replication - connection to avoid the I/O error + * + *

Line 440: Modify the method named createReplicationSlot which add logical that create the slot + * if it doesn't exist + */ +public class PostgresReplicationConnection extends JdbcConnection implements ReplicationConnection { + + private static Logger LOGGER = LoggerFactory.getLogger(PostgresReplicationConnection.class); + + private final String slotName; + private final String publicationName; + private final RelationalTableFilters tableFilter; + private final PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode; + private final PostgresConnectorConfig.LogicalDecoder plugin; + private final boolean dropSlotOnClose; + private final PostgresConnectorConfig connectorConfig; + private final Duration statusUpdateInterval; + private final MessageDecoder messageDecoder; + private final PostgresConnection jdbcConnection; + private final TypeRegistry typeRegistry; + private final Properties streamParams; + + private Lsn defaultStartingPos; + private SlotCreationResult slotCreationInfo; + private boolean hasInitedSlot; + + /** + * Creates a new replication connection with the given params. + * + * @param config the JDBC configuration for the connection; may not be null + * @param slotName the name of the DB slot for logical replication; may not be null + * @param publicationName the name of the DB publication for logical replication; may not be + * null + * @param tableFilter the tables to watch of the DB publication for logical replication; may not + * be null + * @param publicationAutocreateMode the mode for publication autocreation; may not be null + * @param plugin decoder matching the server side plug-in used for streaming changes; may not be + * null + * @param dropSlotOnClose whether the replication slot should be dropped once the connection is + * closed + * @param statusUpdateInterval the interval at which the replication connection should + * periodically send status + * @param doSnapshot whether the connector is doing snapshot + * @param jdbcConnection general PostgreSQL JDBC connection + * @param typeRegistry registry with PostgreSQL types + * @param streamParams additional parameters to pass to the replication stream + * @param schema the schema; must not be null + *

updates to the server + */ + private PostgresReplicationConnection( + PostgresConnectorConfig config, + String slotName, + String publicationName, + RelationalTableFilters tableFilter, + PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode, + PostgresConnectorConfig.LogicalDecoder plugin, + boolean dropSlotOnClose, + boolean doSnapshot, + Duration statusUpdateInterval, + PostgresConnection jdbcConnection, + TypeRegistry typeRegistry, + Properties streamParams, + PostgresSchema schema) { + super( + addDefaultSettings(config.getJdbcConfig()), + PostgresConnection.FACTORY, + null, + null, + "\"", + "\""); + + this.connectorConfig = config; + this.slotName = slotName; + this.publicationName = publicationName; + this.tableFilter = tableFilter; + this.publicationAutocreateMode = publicationAutocreateMode; + this.plugin = plugin; + this.dropSlotOnClose = dropSlotOnClose; + this.statusUpdateInterval = statusUpdateInterval; + this.messageDecoder = + plugin.messageDecoder(new MessageDecoderContext(config, schema), jdbcConnection); + this.jdbcConnection = jdbcConnection; + this.typeRegistry = typeRegistry; + this.streamParams = streamParams; + this.slotCreationInfo = null; + this.hasInitedSlot = false; + } + + private static JdbcConfiguration addDefaultSettings(JdbcConfiguration configuration) { + // first copy the parent's default settings... + // then set some additional replication specific settings + return JdbcConfiguration.adapt( + PostgresConnection.addDefaultSettings( + configuration, PostgresConnection.CONNECTION_STREAMING) + .edit() + .with("replication", "database") + .with( + "preferQueryMode", + "simple") // replication protocol only supports simple query mode + .build()); + } + + private ServerInfo.ReplicationSlot getSlotInfo() throws SQLException, InterruptedException { + try (PostgresConnection connection = + new PostgresConnection( + connectorConfig.getJdbcConfig(), PostgresConnection.CONNECTION_SLOT_INFO)) { + return connection.readReplicationSlotInfo(slotName, plugin.getPostgresPluginName()); + } + } + + protected void initPublication() { + String tableFilterString = null; + if (PostgresConnectorConfig.LogicalDecoder.PGOUTPUT.equals(plugin)) { + LOGGER.info("Initializing PgOutput logical decoder publication"); + try { + PostgresConnection conn = jdbcConnection; + // Unless the autocommit is disabled the SELECT publication query will stay running + conn.setAutoCommit(false); + + String selectPublication = + String.format( + "SELECT COUNT(1) FROM pg_publication WHERE pubname = '%s'", + publicationName); + conn.query( + selectPublication, + rs -> { + if (rs.next()) { + Long count = rs.getLong(1); + // Close eagerly as the transaction might stay running + if (count == 0L) { + LOGGER.info( + "Creating new publication '{}' for plugin '{}'", + publicationName, + plugin); + switch (publicationAutocreateMode) { + case DISABLED: + throw new ConnectException( + "Publication autocreation is disabled, please create one and restart the connector."); + case ALL_TABLES: + String createPublicationStmt = + String.format( + "CREATE PUBLICATION %s FOR ALL TABLES;", + publicationName); + LOGGER.info( + "Creating Publication with statement '{}'", + createPublicationStmt); + // Publication doesn't exist, create it. + conn.executeWithoutCommitting(createPublicationStmt); + break; + case FILTERED: + createOrUpdatePublicationModeFilterted( + tableFilterString, conn, false); + break; + } + } else { + switch (publicationAutocreateMode) { + case FILTERED: + createOrUpdatePublicationModeFilterted( + tableFilterString, conn, true); + break; + default: + LOGGER.trace( + "A logical publication named '{}' for plugin '{}' and database '{}' is already active on the server " + + "and will be used by the plugin", + publicationName, + plugin, + database()); + } + } + } + }); + conn.commit(); + conn.setAutoCommit(true); + } catch (SQLException e) { + throw new JdbcConnectionException(e); + } + } + } + + private void createOrUpdatePublicationModeFilterted( + String tableFilterString, PostgresConnection conn, boolean isUpdate) { + String createOrUpdatePublicationStmt; + try { + Set tablesToCapture = determineCapturedTables(); + tableFilterString = + tablesToCapture.stream() + .map(TableId::toDoubleQuotedString) + .collect(Collectors.joining(", ")); + if (tableFilterString.isEmpty()) { + throw new DebeziumException( + String.format( + "No table filters found for filtered publication %s", + publicationName)); + } + createOrUpdatePublicationStmt = + isUpdate + ? String.format( + "ALTER PUBLICATION %s SET TABLE %s;", + publicationName, tableFilterString) + : String.format( + "CREATE PUBLICATION %s FOR TABLE %s;", + publicationName, tableFilterString); + LOGGER.info( + isUpdate + ? "Updating Publication with statement '{}'" + : "Creating Publication with statement '{}'", + createOrUpdatePublicationStmt); + conn.execute(createOrUpdatePublicationStmt); + } catch (Exception e) { + throw new ConnectException( + String.format( + "Unable to %s filtered publication %s for %s", + isUpdate ? "update" : "create", publicationName, tableFilterString), + e); + } + } + + private Set determineCapturedTables() throws Exception { + Set allTableIds = jdbcConnection.getAllTableIds(connectorConfig.databaseName()); + + Set capturedTables = new HashSet<>(); + + for (TableId tableId : allTableIds) { + if (tableFilter.dataCollectionFilter().isIncluded(tableId)) { + LOGGER.trace("Adding table {} to the list of captured tables", tableId); + capturedTables.add(tableId); + } else { + LOGGER.trace( + "Ignoring table {} as it's not included in the filter configuration", + tableId); + } + } + + return capturedTables.stream() + .sorted() + .collect(Collectors.toCollection(LinkedHashSet::new)); + } + + protected void initReplicationSlot() throws SQLException, InterruptedException { + ServerInfo.ReplicationSlot slotInfo = getSlotInfo(); + + boolean shouldCreateSlot = ServerInfo.ReplicationSlot.INVALID == slotInfo; + try { + // there's no info for this plugin and slot so create a new slot + if (shouldCreateSlot) { + this.createReplicationSlot(); + } + + // replication connection does not support parsing of SQL statements so we need to + // create + // the connection without executing on connect statements - see JDBC opt + // preferQueryMode=simple + pgConnection(); + final String identifySystemStatement = "IDENTIFY_SYSTEM"; + LOGGER.debug( + "running '{}' to validate replication connection", identifySystemStatement); + final Lsn xlogStart = + queryAndMap( + identifySystemStatement, + rs -> { + if (!rs.next()) { + throw new IllegalStateException( + "The DB connection is not a valid replication connection"); + } + String xlogpos = rs.getString("xlogpos"); + LOGGER.debug("received latest xlogpos '{}'", xlogpos); + return Lsn.valueOf(xlogpos); + }); + + if (slotCreationInfo != null) { + this.defaultStartingPos = slotCreationInfo.startLsn(); + } else if (shouldCreateSlot || !slotInfo.hasValidFlushedLsn()) { + // this is a new slot or we weren't able to read a valid flush LSN pos, so we always + // start from the xlog pos that was reported + this.defaultStartingPos = xlogStart; + } else { + Lsn latestFlushedLsn = slotInfo.latestFlushedLsn(); + this.defaultStartingPos = + latestFlushedLsn.compareTo(xlogStart) < 0 ? latestFlushedLsn : xlogStart; + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("found previous flushed LSN '{}'", latestFlushedLsn); + } + } + hasInitedSlot = true; + } catch (SQLException e) { + throw new JdbcConnectionException(e); + } + } + + // Temporary replication slots is a new feature of PostgreSQL 10 + private boolean useTemporarySlot() throws SQLException { + // Temporary replication slots cannot be used due to connection restart + // when finding WAL position + // return dropSlotOnClose && pgConnection().haveMinimumServerVersion(ServerVersion.v10); + return false; + } + + /** + * creating a replication connection and starting to stream involves a few steps: 1. we create + * the connection and ensure that a. the slot exists b. the slot isn't currently being used 2. + * we query to get our potential start position in the slot (lsn) 3. we try and start streaming, + * depending on our options (such as in wal2json) this may fail, which can result in the + * connection being killed and we need to start the process over if we are using a temporary + * slot 4. actually start the streamer + * + *

This method takes care of all of these and this method queries for a default starting + * position If you know where you are starting from you should call {@link #startStreaming(Lsn, + * WalPositionLocator)}, this method delegates to that method + * + * @return + * @throws SQLException + * @throws InterruptedException + */ + @Override + public ReplicationStream startStreaming(WalPositionLocator walPosition) + throws SQLException, InterruptedException { + return startStreaming(null, walPosition); + } + + @Override + public ReplicationStream startStreaming(Lsn offset, WalPositionLocator walPosition) + throws SQLException, InterruptedException { + initConnection(); + + connect(); + if (offset == null || !offset.isValid()) { + offset = defaultStartingPos; + } + Lsn lsn = offset; + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("starting streaming from LSN '{}'", lsn); + } + + final int maxRetries = connectorConfig.maxRetries(); + final Duration delay = connectorConfig.retryDelay(); + int tryCount = 0; + while (true) { + try { + return createReplicationStream(lsn, walPosition); + } catch (Exception e) { + String message = "Failed to start replication stream at " + lsn; + if (++tryCount > maxRetries) { + if (e.getMessage().matches(".*replication slot .* is active.*")) { + message += + "; when setting up multiple connectors for the same database host, please make sure to use a distinct replication slot name for each."; + } + throw new DebeziumException(message, e); + } else { + LOGGER.warn( + message + ", waiting for {} ms and retrying, attempt number {} over {}", + delay, + tryCount, + maxRetries); + final Metronome metronome = Metronome.sleeper(delay, Clock.SYSTEM); + metronome.pause(); + } + } + } + } + + @Override + public void initConnection() throws SQLException, InterruptedException { + // See https://www.postgresql.org/docs/current/logical-replication-quick-setup.html + // For pgoutput specifically, the publication must be created before the slot. + initPublication(); + if (!hasInitedSlot) { + initReplicationSlot(); + } + } + + @Override + public Optional createReplicationSlot() throws SQLException { + // note that some of these options are only supported in Postgres 9.4+, additionally + // the options are not yet exported by the jdbc api wrapper, therefore, we just do + // this ourselves but eventually this should be moved back to the jdbc API + // see https://github.com/pgjdbc/pgjdbc/issues/1305 + ServerInfo.ReplicationSlot slotInfo; + try { + slotInfo = getSlotInfo(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + boolean shouldCreateSlot = ServerInfo.ReplicationSlot.INVALID == slotInfo; + + if (shouldCreateSlot) { + LOGGER.debug("Creating new replication slot '{}' for plugin '{}'", slotName, plugin); + String tempPart = ""; + // Exported snapshots are supported in Postgres 9.4+ + boolean canExportSnapshot = pgConnection().haveMinimumServerVersion(ServerVersion.v9_4); + if ((dropSlotOnClose) && !canExportSnapshot) { + LOGGER.warn( + "A slot marked as temporary or with an exported snapshot was created, " + + "but not on a supported version of Postgres, ignoring!"); + } + if (useTemporarySlot()) { + tempPart = "TEMPORARY"; + } + + // See https://www.postgresql.org/docs/current/logical-replication-quick-setup.html + // For pgoutput specifically, the publication must be created prior to the slot. + initPublication(); + + try (Statement stmt = pgConnection().createStatement()) { + String createCommand = + String.format( + "CREATE_REPLICATION_SLOT \"%s\" %s LOGICAL %s", + slotName, tempPart, plugin.getPostgresPluginName()); + LOGGER.info("Creating replication slot with command {}", createCommand); + stmt.execute(createCommand); + // when we are in Postgres 9.4+, we can parse the slot creation info, + // otherwise, it returns nothing + if (canExportSnapshot) { + this.slotCreationInfo = parseSlotCreation(stmt.getResultSet()); + } + } + } + return Optional.ofNullable(slotCreationInfo); + } + + protected BaseConnection pgConnection() throws SQLException { + return (BaseConnection) connection(false); + } + + private SlotCreationResult parseSlotCreation(ResultSet rs) { + try { + if (rs.next()) { + String slotName = rs.getString("slot_name"); + String startPoint = rs.getString("consistent_point"); + String snapName = rs.getString("snapshot_name"); + String pluginName = rs.getString("output_plugin"); + + return new SlotCreationResult(slotName, startPoint, snapName, pluginName); + } else { + throw new ConnectException("No replication slot found"); + } + } catch (SQLException ex) { + throw new ConnectException("Unable to parse create_replication_slot response", ex); + } + } + + private ReplicationStream createReplicationStream( + final Lsn startLsn, WalPositionLocator walPosition) + throws SQLException, InterruptedException { + PGReplicationStream s; + + try { + try { + s = + startPgReplicationStream( + startLsn, + plugin.forceRds() + ? messageDecoder::optionsWithoutMetadata + : messageDecoder::optionsWithMetadata); + messageDecoder.setContainsMetadata(plugin.forceRds() ? false : true); + } catch (PSQLException e) { + LOGGER.debug( + "Could not register for streaming, retrying without optional options", e); + + // re-init the slot after a failed start of slot, as this + // may have closed the slot + if (useTemporarySlot()) { + initReplicationSlot(); + } + + s = + startPgReplicationStream( + startLsn, + plugin.forceRds() + ? messageDecoder::optionsWithoutMetadata + : messageDecoder::optionsWithMetadata); + messageDecoder.setContainsMetadata(plugin.forceRds() ? false : true); + } + } catch (PSQLException e) { + if (e.getMessage().matches("(?s)ERROR: option .* is unknown.*")) { + // It is possible we are connecting to an old wal2json plug-in + LOGGER.warn( + "Could not register for streaming with metadata in messages, falling back to messages without metadata"); + + // re-init the slot after a failed start of slot, as this + // may have closed the slot + if (useTemporarySlot()) { + initReplicationSlot(); + } + + s = startPgReplicationStream(startLsn, messageDecoder::optionsWithoutMetadata); + messageDecoder.setContainsMetadata(false); + } else if (e.getMessage() + .matches("(?s)ERROR: requested WAL segment .* has already been removed.*")) { + LOGGER.error("Cannot rewind to last processed WAL position", e); + throw new ConnectException( + "The offset to start reading from has been removed from the database write-ahead log. Create a new snapshot and consider setting of PostgreSQL parameter wal_keep_segments = 0."); + } else { + throw e; + } + } + + final PGReplicationStream stream = s; + + return new ReplicationStream() { + + private static final int CHECK_WARNINGS_AFTER_COUNT = 100; + private int warningCheckCounter = CHECK_WARNINGS_AFTER_COUNT; + private ExecutorService keepAliveExecutor = null; + private AtomicBoolean keepAliveRunning; + private final Metronome metronome = + Metronome.sleeper(statusUpdateInterval, Clock.SYSTEM); + + // make sure this is volatile since multiple threads may be interested in this value + private volatile Lsn lastReceivedLsn; + + @Override + public void read(ReplicationMessageProcessor processor) + throws SQLException, InterruptedException { + processWarnings(false); + ByteBuffer read = stream.read(); + final Lsn lastReceiveLsn = Lsn.valueOf(stream.getLastReceiveLSN()); + LOGGER.trace( + "Streaming requested from LSN {}, received LSN {}", + startLsn, + lastReceiveLsn); + if (messageDecoder.shouldMessageBeSkipped( + read, lastReceiveLsn, startLsn, walPosition)) { + return; + } + deserializeMessages(read, processor); + } + + @Override + public boolean readPending(ReplicationMessageProcessor processor) + throws SQLException, InterruptedException { + processWarnings(false); + ByteBuffer read = stream.readPending(); + final Lsn lastReceiveLsn = Lsn.valueOf(stream.getLastReceiveLSN()); + LOGGER.trace( + "Streaming requested from LSN {}, received LSN {}", + startLsn, + lastReceiveLsn); + + if (read == null) { + return false; + } + + if (messageDecoder.shouldMessageBeSkipped( + read, lastReceiveLsn, startLsn, walPosition)) { + return true; + } + + deserializeMessages(read, processor); + + return true; + } + + private void deserializeMessages( + ByteBuffer buffer, ReplicationMessageProcessor processor) + throws SQLException, InterruptedException { + lastReceivedLsn = Lsn.valueOf(stream.getLastReceiveLSN()); + LOGGER.trace("Received message at LSN {}", lastReceivedLsn); + messageDecoder.processMessage(buffer, processor, typeRegistry); + } + + @Override + public void close() throws SQLException { + processWarnings(true); + stream.close(); + } + + @Override + public void flushLsn(Lsn lsn) throws SQLException { + doFlushLsn(lsn); + } + + private void doFlushLsn(Lsn lsn) throws SQLException { + stream.setFlushedLSN(lsn.asLogSequenceNumber()); + stream.setAppliedLSN(lsn.asLogSequenceNumber()); + + stream.forceUpdateStatus(); + } + + @Override + public Lsn lastReceivedLsn() { + return lastReceivedLsn; + } + + @Override + public void startKeepAlive(ExecutorService service) { + if (keepAliveExecutor == null) { + keepAliveExecutor = service; + keepAliveRunning = new AtomicBoolean(true); + keepAliveExecutor.submit( + () -> { + while (keepAliveRunning.get()) { + try { + LOGGER.trace( + "Forcing status update with replication stream"); + stream.forceUpdateStatus(); + metronome.pause(); + } catch (Exception exp) { + throw new RuntimeException( + "received unexpected exception will perform keep alive", + exp); + } + } + }); + } + } + + @Override + public void stopKeepAlive() { + if (keepAliveExecutor != null) { + keepAliveRunning.set(false); + keepAliveExecutor.shutdownNow(); + keepAliveExecutor = null; + } + } + + private void processWarnings(final boolean forced) throws SQLException { + if (--warningCheckCounter == 0 || forced) { + warningCheckCounter = CHECK_WARNINGS_AFTER_COUNT; + for (SQLWarning w = connection().getWarnings(); + w != null; + w = w.getNextWarning()) { + LOGGER.debug( + "Server-side message: '{}', state = {}, code = {}", + w.getMessage(), + w.getSQLState(), + w.getErrorCode()); + } + connection().clearWarnings(); + } + } + + @Override + public Lsn startLsn() { + return startLsn; + } + }; + } + + private PGReplicationStream startPgReplicationStream( + final Lsn lsn, + BiFunction< + ChainedLogicalStreamBuilder, + Function, + ChainedLogicalStreamBuilder> + configurator) + throws SQLException { + assert lsn != null; + ChainedLogicalStreamBuilder streamBuilder = + pgConnection() + .getReplicationAPI() + .replicationStream() + .logical() + .withSlotName("\"" + slotName + "\"") + .withStartPosition(lsn.asLogSequenceNumber()) + .withSlotOptions(streamParams); + streamBuilder = configurator.apply(streamBuilder, this::hasMinimumVersion); + + if (statusUpdateInterval != null && statusUpdateInterval.toMillis() > 0) { + streamBuilder.withStatusInterval( + toIntExact(statusUpdateInterval.toMillis()), TimeUnit.MILLISECONDS); + } + + PGReplicationStream stream = streamBuilder.start(); + + // TODO DBZ-508 get rid of this + // Needed by tests when connections are opened and closed in a fast sequence + try { + Thread.sleep(10); + } catch (Exception e) { + } + stream.forceUpdateStatus(); + return stream; + } + + private Boolean hasMinimumVersion(int version) { + try { + return pgConnection().haveMinimumServerVersion(version); + } catch (SQLException e) { + throw new DebeziumException(e); + } + } + + @Override + public synchronized void close() { + close(true); + } + + public synchronized void close(boolean dropSlot) { + try { + LOGGER.debug("Closing message decoder"); + messageDecoder.close(); + } catch (Throwable e) { + LOGGER.error("Unexpected error while closing message decoder", e); + } + + try { + LOGGER.debug("Closing replication connection"); + super.close(); + } catch (Throwable e) { + LOGGER.error("Unexpected error while closing Postgres connection", e); + } + if (dropSlotOnClose && dropSlot) { + // we're dropping the replication slot via a regular - i.e. not a replication - + // connection + try (PostgresConnection connection = + new PostgresConnection( + connectorConfig.getJdbcConfig(), + PostgresConnection.CONNECTION_DROP_SLOT)) { + connection.dropReplicationSlot(slotName); + connection.dropPublication(publicationName); + } catch (Throwable e) { + LOGGER.error("Unexpected error while dropping replication slot", e); + } + } + } + + @Override + public void reconnect() throws SQLException { + close(false); + // Don't re-execute initial commands on reconnection + connection(false); + } + + protected static class ReplicationConnectionBuilder implements Builder { + + private final PostgresConnectorConfig config; + private String slotName = DEFAULT_SLOT_NAME; + private String publicationName = DEFAULT_PUBLICATION_NAME; + private RelationalTableFilters tableFilter; + private PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode = + PostgresConnectorConfig.AutoCreateMode.ALL_TABLES; + private PostgresConnectorConfig.LogicalDecoder plugin = + PostgresConnectorConfig.LogicalDecoder.DECODERBUFS; + private boolean dropSlotOnClose = DEFAULT_DROP_SLOT_ON_CLOSE; + private Duration statusUpdateIntervalVal; + private boolean doSnapshot; + private TypeRegistry typeRegistry; + private PostgresSchema schema; + private Properties slotStreamParams = new Properties(); + private PostgresConnection jdbcConnection; + + protected ReplicationConnectionBuilder(PostgresConnectorConfig config) { + assert config != null; + this.config = config; + } + + @Override + public ReplicationConnectionBuilder withSlot(final String slotName) { + assert slotName != null; + this.slotName = slotName; + return this; + } + + @Override + public Builder withPublication(String publicationName) { + assert publicationName != null; + this.publicationName = publicationName; + return this; + } + + @Override + public Builder withTableFilter(RelationalTableFilters tableFilter) { + assert tableFilter != null; + this.tableFilter = tableFilter; + return this; + } + + @Override + public Builder withPublicationAutocreateMode( + PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode) { + assert publicationName != null; + this.publicationAutocreateMode = publicationAutocreateMode; + return this; + } + + @Override + public ReplicationConnectionBuilder withPlugin( + final PostgresConnectorConfig.LogicalDecoder plugin) { + assert plugin != null; + this.plugin = plugin; + return this; + } + + @Override + public ReplicationConnectionBuilder dropSlotOnClose(final boolean dropSlotOnClose) { + this.dropSlotOnClose = dropSlotOnClose; + return this; + } + + @Override + public ReplicationConnectionBuilder streamParams(final String slotStreamParams) { + if (slotStreamParams != null && !slotStreamParams.isEmpty()) { + this.slotStreamParams = new Properties(); + String[] paramsWithValues = slotStreamParams.split(";"); + for (String paramsWithValue : paramsWithValues) { + String[] paramAndValue = paramsWithValue.split("="); + if (paramAndValue.length == 2) { + this.slotStreamParams.setProperty(paramAndValue[0], paramAndValue[1]); + } else { + LOGGER.warn( + "The following STREAM_PARAMS value is invalid: {}", + paramsWithValue); + } + } + } + return this; + } + + @Override + public ReplicationConnectionBuilder statusUpdateInterval( + final Duration statusUpdateInterval) { + this.statusUpdateIntervalVal = statusUpdateInterval; + return this; + } + + @Override + public Builder doSnapshot(boolean doSnapshot) { + this.doSnapshot = doSnapshot; + return this; + } + + @Override + public Builder jdbcMetadataConnection(PostgresConnection jdbcConnection) { + this.jdbcConnection = jdbcConnection; + return this; + } + + @Override + public ReplicationConnection build() { + assert plugin != null : "Decoding plugin name is not set"; + return new PostgresReplicationConnection( + config, + slotName, + publicationName, + tableFilter, + publicationAutocreateMode, + plugin, + dropSlotOnClose, + doSnapshot, + statusUpdateIntervalVal, + jdbcConnection, + typeRegistry, + slotStreamParams, + schema); + } + + @Override + public Builder withTypeRegistry(TypeRegistry typeRegistry) { + this.typeRegistry = typeRegistry; + return this; + } + + @Override + public Builder withSchema(PostgresSchema schema) { + this.schema = schema; + return this; + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java new file mode 100644 index 00000000000..e9f552db6c0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java @@ -0,0 +1,110 @@ +/* + * 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.cdc.opengauss; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.CatalogOptions; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceTableConfig; +import org.apache.seatunnel.connectors.cdc.base.option.JdbcSourceOptions; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.utils.CatalogTableUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option.PostgresOptions; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresIncrementalSource; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresSourceOptions; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; +import java.util.List; +import java.util.Optional; + +@AutoService(Factory.class) +public class OpengaussIncrementalSourceFactory implements TableSourceFactory { + private static final String IDENTIFIER = "Opengauss-CDC"; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public OptionRule optionRule() { + return JdbcSourceOptions.getBaseRule() + .required( + JdbcSourceOptions.USERNAME, + JdbcSourceOptions.PASSWORD, + JdbcCatalogOptions.BASE_URL) + .exclusive(CatalogOptions.TABLE_NAMES, CatalogOptions.TABLE_PATTERN) + .optional( + JdbcSourceOptions.DATABASE_NAMES, + JdbcSourceOptions.SERVER_TIME_ZONE, + JdbcSourceOptions.CONNECT_TIMEOUT_MS, + JdbcSourceOptions.CONNECT_MAX_RETRIES, + JdbcSourceOptions.CONNECTION_POOL_SIZE, + PostgresOptions.DECODING_PLUGIN_NAME, + PostgresOptions.SLOT_NAME, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND, + JdbcSourceOptions.SAMPLE_SHARDING_THRESHOLD, + JdbcSourceOptions.TABLE_NAMES_CONFIG) + .optional(PostgresSourceOptions.STARTUP_MODE, PostgresSourceOptions.STOP_MODE) + .conditional( + PostgresSourceOptions.STARTUP_MODE, + StartupMode.INITIAL, + JdbcSourceOptions.EXACTLY_ONCE) + .build(); + } + + @Override + public Class getSourceClass() { + return PostgresIncrementalSource.class; + } + + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> { + List catalogTables = + CatalogTableUtil.getCatalogTables( + "Postgres", context.getOptions(), context.getClassLoader()); + Optional> tableConfigs = + context.getOptions().getOptional(JdbcSourceOptions.TABLE_NAMES_CONFIG); + if (tableConfigs.isPresent()) { + catalogTables = + CatalogTableUtils.mergeCatalogTableConfig( + catalogTables, tableConfigs.get(), s -> TablePath.of(s, true)); + } + SeaTunnelDataType dataType = + CatalogTableUtil.convertToMultipleRowType(catalogTables); + return (SeaTunnelSource) + new PostgresIncrementalSource<>(context.getOptions(), dataType, catalogTables); + }; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/pom.xml b/seatunnel-connectors-v2/connector-cdc/pom.xml index 44916d35caa..a422f6406d3 100644 --- a/seatunnel-connectors-v2/connector-cdc/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/pom.xml @@ -36,6 +36,7 @@ connector-cdc-mongodb connector-cdc-postgres connector-cdc-oracle + connector-cdc-opengauss diff --git a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java index 4c9e6f47605..d83e8b5c96b 100644 --- a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java +++ b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; -import org.apache.seatunnel.api.sink.event.WriterCloseEvent; 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; @@ -99,9 +98,7 @@ public void write(SeaTunnelRow element) { } @Override - public void close() { - context.getEventListener().onEvent(new WriterCloseEvent()); - } + public void close() {} private String fieldsInfo(SeaTunnelRowType seaTunnelRowType) { String[] fieldsInfo = new String[seaTunnelRowType.getTotalFields()]; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java index bbf594eb10b..32a86dd75d6 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java @@ -110,8 +110,7 @@ public String getDefaultDatabase() throws CatalogException { public boolean databaseExists(String databaseName) throws CatalogException { // check if the index exist try { - List indexDocsCount = esRestClient.getIndexDocsCount(databaseName); - return true; + return esRestClient.checkIndexExist(databaseName); } catch (Exception e) { log.error( String.format( diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java index f80f20f6736..b54541bf93b 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java @@ -58,6 +58,7 @@ import javax.net.ssl.SSLContext; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -76,7 +77,7 @@ import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.OBJECT; @Slf4j -public class EsRestClient { +public class EsRestClient implements Closeable { private static final int CONNECTION_REQUEST_TIMEOUT = 10 * 1000; @@ -258,6 +259,7 @@ public ElasticsearchClusterInfo getClusterInfo() { } } + @Override public void close() { try { restClient.close(); @@ -370,6 +372,30 @@ private ScrollResult getDocsFromScrollResponse(ObjectNode responseJson) { return scrollResult; } + /** + * Instead of the getIndexDocsCount method to determine if the index exists, + * + *

+ * + *

getIndexDocsCount throws an exception if the index does not exist + * + *

+ * + * @param index index + * @return true or false + */ + public boolean checkIndexExist(String index) { + Request request = new Request("HEAD", "/" + index); + try { + Response response = restClient.performRequest(request); + int statusCode = response.getStatusLine().getStatusCode(); + return statusCode == 200; + } catch (Exception ex) { + throw new ElasticsearchConnectorException( + ElasticsearchConnectorErrorCode.CHECK_INDEX_FAILED, ex); + } + } + public List getIndexDocsCount(String index) { String endpoint = String.format("/_cat/indices/%s?h=index,docsCount&format=json", index); Request request = new Request("GET", endpoint); diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java index c63cd375952..ffeb69d67f2 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java @@ -17,15 +17,31 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.config; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; + import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.table.catalog.CatalogTable; + +import lombok.Getter; +import lombok.Setter; +import java.io.Serializable; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -public class SourceConfig { +@Getter +@Setter +public class SourceConfig implements Serializable { + + public static final Option>> INDEX_LIST = + Options.key("index_list") + .type(new TypeReference>>() {}) + .noDefaultValue() + .withDescription("index_list for multiTable sync"); public static final Option INDEX = Options.key("index") @@ -61,11 +77,30 @@ public class SourceConfig { .withDescription( "Maximum number of hits to be returned with each Elasticsearch scroll request"); - public static final Option QUERY = + public static final Option> QUERY = Options.key("query") - .objectType(Map.class) + .type(new TypeReference>() {}) .defaultValue( Collections.singletonMap("match_all", new HashMap())) .withDescription( "Elasticsearch query language. You can control the range of data read"); + + private String index; + private List source; + private Map query; + private String scrollTime; + private int scrollSize; + + private CatalogTable catalogTable; + + public SourceConfig clone() { + SourceConfig sourceConfig = new SourceConfig(); + sourceConfig.setIndex(index); + sourceConfig.setSource(new ArrayList<>(source)); + sourceConfig.setQuery(new HashMap<>(query)); + sourceConfig.setScrollTime(scrollTime); + sourceConfig.setScrollSize(scrollSize); + sourceConfig.setCatalogTable(catalogTable); + return sourceConfig; + } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java index fe182868d4d..8ffbb7f4b9d 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java @@ -29,7 +29,11 @@ public enum ElasticsearchConnectorErrorCode implements SeaTunnelErrorCode { DROP_INDEX_FAILED("ELASTICSEARCH-06", "Drop elasticsearch index failed"), CREATE_INDEX_FAILED("ELASTICSEARCH-07", "Create elasticsearch index failed"), ES_FIELD_TYPE_NOT_SUPPORT("ELASTICSEARCH-08", "Not support the elasticsearch field type"), - CLEAR_INDEX_DATA_FAILED("ELASTICSEARCH-09", "Clear elasticsearch index data failed"); + CLEAR_INDEX_DATA_FAILED("ELASTICSEARCH-09", "Clear elasticsearch index data failed"), + CHECK_INDEX_FAILED("ELASTICSEARCH-10", "Failed to check whether the index exists"), + SOURCE_CONFIG_ERROR_01( + "ELASTICSEARCH-11", + "'index' or 'index_list' must be configured, with at least one being required."); ; private final String code; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java index a409a4ae886..fd176f2f034 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java @@ -137,7 +137,9 @@ SeaTunnelRow convert(ElasticsearchRecord rowRecord) { fieldName, value, seaTunnelDataType, JsonUtils.toJsonString(rowRecord)), ex); } - return new SeaTunnelRow(seaTunnelFields); + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(seaTunnelFields); + seaTunnelRow.setTableId(rowRecord.getTableId()); + return seaTunnelRow; } Object convertValue(SeaTunnelDataType fieldType, String fieldValue) diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java index 3e5eb10b582..57c9dcb084f 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java @@ -30,4 +30,6 @@ public class ElasticsearchRecord { private Map doc; private List source; + + private String tableId; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java index 7b153f0be3a..a22ca179569 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java @@ -40,6 +40,8 @@ import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ElasticsearchConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ElasticsearchConnectorException; import org.apache.commons.collections4.CollectionUtils; @@ -50,6 +52,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; @Slf4j public class ElasticsearchSource @@ -58,30 +61,60 @@ public class ElasticsearchSource SupportParallelism, SupportColumnProjection { - private final ReadonlyConfig config; + private final List sourceConfigList; + private final ReadonlyConfig connectionConfig; - private CatalogTable catalogTable; + public ElasticsearchSource(ReadonlyConfig config) { + this.connectionConfig = config; + boolean multiSource = config.getOptional(SourceConfig.INDEX_LIST).isPresent(); + boolean singleSource = config.getOptional(SourceConfig.INDEX).isPresent(); + if (multiSource && singleSource) { + log.warn( + "Elasticsearch Source config warn: when both 'index' and 'index_list' are present in the configuration, only the 'index_list' configuration will take effect"); + } + if (!multiSource && !singleSource) { + throw new ElasticsearchConnectorException( + ElasticsearchConnectorErrorCode.SOURCE_CONFIG_ERROR_01, + ElasticsearchConnectorErrorCode.SOURCE_CONFIG_ERROR_01.getDescription()); + } + if (multiSource) { + this.sourceConfigList = createMultiSource(config); + } else { + this.sourceConfigList = Collections.singletonList(parseOneIndexQueryConfig(config)); + } + } + + private List createMultiSource(ReadonlyConfig config) { + List> configMaps = config.get(SourceConfig.INDEX_LIST); + List configList = + configMaps.stream().map(ReadonlyConfig::fromMap).collect(Collectors.toList()); + List sourceConfigList = new ArrayList<>(configList.size()); + for (ReadonlyConfig readonlyConfig : configList) { + SourceConfig sourceConfig = parseOneIndexQueryConfig(readonlyConfig); + sourceConfigList.add(sourceConfig); + } + return sourceConfigList; + } - private List source; + private SourceConfig parseOneIndexQueryConfig(ReadonlyConfig readonlyConfig) { - private Map arrayColumn; + Map query = readonlyConfig.get(SourceConfig.QUERY); + String index = readonlyConfig.get(SourceConfig.INDEX); - public ElasticsearchSource(ReadonlyConfig config) { - this.config = config; - if (config.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { + CatalogTable catalogTable; + List source; + Map arrayColumn; + + if (readonlyConfig.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { // todo: We need to remove the schema in ES. log.warn( - "The schema config in ElasticSearch sink is deprecated, please use source config instead!"); - catalogTable = CatalogTableUtil.buildWithConfig(config); + "The schema config in ElasticSearch source/sink is deprecated, please use source config instead!"); + catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); source = Arrays.asList(catalogTable.getSeaTunnelRowType().getFieldNames()); } else { - source = config.get(SourceConfig.SOURCE); - arrayColumn = config.get(SourceConfig.ARRAY_COLUMN); - EsRestClient esRestClient = EsRestClient.createInstance(config); - Map> esFieldType = - esRestClient.getFieldTypeMapping(config.get(SourceConfig.INDEX), source); - esRestClient.close(); - + source = readonlyConfig.get(SourceConfig.SOURCE); + arrayColumn = readonlyConfig.get(SourceConfig.ARRAY_COLUMN); + Map> esFieldType = getFieldTypeMapping(index, source); if (CollectionUtils.isEmpty(source)) { source = new ArrayList<>(esFieldType.keySet()); } @@ -90,26 +123,48 @@ public ElasticsearchSource(ReadonlyConfig config) { for (int i = 0; i < source.size(); i++) { String key = source.get(i); + String sourceType = esFieldType.get(key).getDataType(); if (arrayColumn.containsKey(key)) { String value = arrayColumn.get(key); SeaTunnelDataType dataType = SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType(key, value); - builder.column(PhysicalColumn.of(key, dataType, 0, true, null, null)); + builder.column( + PhysicalColumn.of( + key, dataType, 0L, true, null, null, sourceType, null)); continue; } builder.column( - PhysicalColumn.of(source.get(i), fieldTypes[i], 0, true, null, null)); + PhysicalColumn.of( + source.get(i), + fieldTypes[i], + 0L, + true, + null, + null, + sourceType, + null)); } catalogTable = CatalogTable.of( - TableIdentifier.of( - "elasticsearch", null, config.get(SourceConfig.INDEX)), + TableIdentifier.of("elasticsearch", null, index), builder.build(), Collections.emptyMap(), Collections.emptyList(), ""); } + + String scrollTime = readonlyConfig.get(SourceConfig.SCROLL_TIME); + int scrollSize = readonlyConfig.get(SourceConfig.SCROLL_SIZE); + SourceConfig sourceConfig = new SourceConfig(); + sourceConfig.setSource(source); + sourceConfig.setCatalogTable(catalogTable); + sourceConfig.setQuery(query); + sourceConfig.setScrollTime(scrollTime); + sourceConfig.setScrollSize(scrollSize); + sourceConfig.setIndex(index); + sourceConfig.setCatalogTable(catalogTable); + return sourceConfig; } @Override @@ -124,21 +179,23 @@ public Boundedness getBoundedness() { @Override public List getProducedCatalogTables() { - return Collections.singletonList(catalogTable); + return sourceConfigList.stream() + .map(SourceConfig::getCatalogTable) + .collect(Collectors.toList()); } @Override public SourceReader createReader( SourceReader.Context readerContext) { - return new ElasticsearchSourceReader( - readerContext, config, catalogTable.getSeaTunnelRowType()); + return new ElasticsearchSourceReader(readerContext, connectionConfig); } @Override public SourceSplitEnumerator createEnumerator( SourceSplitEnumerator.Context enumeratorContext) { - return new ElasticsearchSourceSplitEnumerator(enumeratorContext, config, source); + return new ElasticsearchSourceSplitEnumerator( + enumeratorContext, connectionConfig, sourceConfigList); } @Override @@ -147,7 +204,7 @@ public SourceReader createReader( SourceSplitEnumerator.Context enumeratorContext, ElasticsearchSourceState sourceState) { return new ElasticsearchSourceSplitEnumerator( - enumeratorContext, sourceState, config, source); + enumeratorContext, sourceState, connectionConfig, sourceConfigList); } @VisibleForTesting @@ -162,4 +219,13 @@ public static SeaTunnelDataType[] getSeaTunnelDataType( } return fieldTypes; } + + private Map> getFieldTypeMapping( + String index, List source) { + // EsRestClient#getFieldTypeMapping may throw runtime exception + // so here we use try-resources-finally to close the resource + try (EsRestClient esRestClient = EsRestClient.createInstance(connectionConfig)) { + return esRestClient.getFieldTypeMapping(index, source); + } + } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java index 6ff08b7d069..8f41256e37c 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java @@ -20,7 +20,6 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; @@ -40,10 +39,10 @@ import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.EsClusterConnectionConfig.TLS_VERIFY_HOSTNAME; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.EsClusterConnectionConfig.USERNAME; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.INDEX; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.INDEX_LIST; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.QUERY; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.SCROLL_SIZE; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.SCROLL_TIME; -import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.SOURCE; @AutoService(Factory.class) public class ElasticsearchSourceFactory implements TableSourceFactory { @@ -55,8 +54,10 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(HOSTS, INDEX) + .required(HOSTS) .optional( + INDEX, + INDEX_LIST, USERNAME, PASSWORD, SCROLL_TIME, @@ -68,7 +69,6 @@ public OptionRule optionRule() { TLS_KEY_STORE_PASSWORD, TLS_TRUST_STORE_PATH, TLS_TRUST_STORE_PASSWORD) - .exclusive(SOURCE, TableSchemaOptions.SCHEMA) .build(); } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java index 7d2398816a1..a58c2c622d8 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java @@ -23,8 +23,8 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.DefaultSeaTunnelRowDeserializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.ElasticsearchRecord; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.SeaTunnelRowDeserializer; @@ -44,27 +44,23 @@ public class ElasticsearchSourceReader SourceReader.Context context; - private final ReadonlyConfig config; + private final ReadonlyConfig connConfig; private EsRestClient esRestClient; - private final SeaTunnelRowDeserializer deserializer; - Deque splits = new LinkedList<>(); boolean noMoreSplit; private final long pollNextWaitTime = 1000L; - public ElasticsearchSourceReader( - SourceReader.Context context, ReadonlyConfig config, SeaTunnelRowType rowTypeInfo) { + public ElasticsearchSourceReader(SourceReader.Context context, ReadonlyConfig connConfig) { this.context = context; - this.config = config; - this.deserializer = new DefaultSeaTunnelRowDeserializer(rowTypeInfo); + this.connConfig = connConfig; } @Override public void open() { - esRestClient = EsRestClient.createInstance(this.config); + esRestClient = EsRestClient.createInstance(this.connConfig); } @Override @@ -77,7 +73,10 @@ public void pollNext(Collector output) throws Exception { synchronized (output.getCheckpointLock()) { ElasticsearchSourceSplit split = splits.poll(); if (split != null) { - SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo(); + SeaTunnelRowType seaTunnelRowType = split.getSeaTunnelRowType(); + SeaTunnelRowDeserializer deserializer = + new DefaultSeaTunnelRowDeserializer(seaTunnelRowType); + SourceConfig sourceIndexInfo = split.getSourceConfig(); ScrollResult scrollResult = esRestClient.searchByScroll( sourceIndexInfo.getIndex(), @@ -85,12 +84,12 @@ public void pollNext(Collector output) throws Exception { sourceIndexInfo.getQuery(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize()); - outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output); + outputFromScrollResult(scrollResult, sourceIndexInfo, output, deserializer); while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) { scrollResult = esRestClient.searchWithScrollId( scrollResult.getScrollId(), sourceIndexInfo.getScrollTime()); - outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output); + outputFromScrollResult(scrollResult, sourceIndexInfo, output, deserializer); } } else if (noMoreSplit) { // signal to the source that we have reached the end of the data. @@ -103,10 +102,15 @@ public void pollNext(Collector output) throws Exception { } private void outputFromScrollResult( - ScrollResult scrollResult, List source, Collector output) { + ScrollResult scrollResult, + SourceConfig sourceConfig, + Collector output, + SeaTunnelRowDeserializer deserializer) { + List source = sourceConfig.getSource(); + String tableId = sourceConfig.getCatalogTable().getTablePath().toString(); for (Map doc : scrollResult.getDocs()) { SeaTunnelRow seaTunnelRow = - deserializer.deserialize(new ElasticsearchRecord(doc, source)); + deserializer.deserialize(new ElasticsearchRecord(doc, source, tableId)); output.collect(seaTunnelRow); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java index f2ad78fa5c4..3c7d25b5b49 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java @@ -18,7 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.source; import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; import lombok.AllArgsConstructor; import lombok.Getter; @@ -32,7 +33,11 @@ public class ElasticsearchSourceSplit implements SourceSplit { private String splitId; - @Getter private SourceIndexInfo sourceIndexInfo; + @Getter private SourceConfig sourceConfig; + + public SeaTunnelRowType getSeaTunnelRowType() { + return sourceConfig.getCatalogTable().getSeaTunnelRowType(); + } @Override public String splitId() { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java index 107aaac322a..5e3356ebd65 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java @@ -19,11 +19,10 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ElasticsearchConnectorException; import lombok.extern.slf4j.Slf4j; @@ -45,7 +44,7 @@ public class ElasticsearchSourceSplitEnumerator private final SourceSplitEnumerator.Context context; - private final ReadonlyConfig config; + private final ReadonlyConfig connConfig; private EsRestClient esRestClient; @@ -53,36 +52,36 @@ public class ElasticsearchSourceSplitEnumerator private Map> pendingSplit; - private final List source; + private final List sourceConfigs; private volatile boolean shouldEnumerate; public ElasticsearchSourceSplitEnumerator( SourceSplitEnumerator.Context context, - ReadonlyConfig config, - List source) { - this(context, null, config, source); + ReadonlyConfig connConfig, + List sourceConfigs) { + this(context, null, connConfig, sourceConfigs); } public ElasticsearchSourceSplitEnumerator( SourceSplitEnumerator.Context context, ElasticsearchSourceState sourceState, - ReadonlyConfig config, - List source) { + ReadonlyConfig connConfig, + List sourceConfigs) { this.context = context; - this.config = config; + this.connConfig = connConfig; this.pendingSplit = new HashMap<>(); this.shouldEnumerate = sourceState == null; if (sourceState != null) { this.shouldEnumerate = sourceState.isShouldEnumerate(); this.pendingSplit.putAll(sourceState.getPendingSplit()); } - this.source = source; + this.sourceConfigs = sourceConfigs; } @Override public void open() { - esRestClient = EsRestClient.createInstance(config); + esRestClient = EsRestClient.createInstance(connConfig); } @Override @@ -140,26 +139,22 @@ private void assignSplit(Collection readers) { private List getElasticsearchSplit() { List splits = new ArrayList<>(); - String scrollTime = config.get(SourceConfig.SCROLL_TIME); - int scrollSize = config.get(SourceConfig.SCROLL_SIZE); - Map query = config.get(SourceConfig.QUERY); - List indexDocsCounts = - esRestClient.getIndexDocsCount(config.get(SourceConfig.INDEX)); - indexDocsCounts = - indexDocsCounts.stream() - .filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0) - .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)) - .collect(Collectors.toList()); - for (IndexDocsCount indexDocsCount : indexDocsCounts) { - splits.add( - new ElasticsearchSourceSplit( - String.valueOf(indexDocsCount.getIndex().hashCode()), - new SourceIndexInfo( - indexDocsCount.getIndex(), - source, - query, - scrollTime, - scrollSize))); + for (SourceConfig sourceConfig : sourceConfigs) { + + String index = sourceConfig.getIndex(); + List indexDocsCounts = esRestClient.getIndexDocsCount(index); + indexDocsCounts = + indexDocsCounts.stream() + .filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0) + .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)) + .collect(Collectors.toList()); + for (IndexDocsCount indexDocsCount : indexDocsCounts) { + SourceConfig cloneCfg = sourceConfig.clone(); + cloneCfg.setIndex(indexDocsCount.getIndex()); + splits.add( + new ElasticsearchSourceSplit( + String.valueOf(indexDocsCount.getIndex().hashCode()), cloneCfg)); + } } return splits; } @@ -185,7 +180,7 @@ public int currentUnassignedSplitSize() { @Override public void handleSplitRequest(int subtaskId) { throw new ElasticsearchConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + CommonErrorCode.OPERATION_NOT_SUPPORTED, "Unsupported handleSplitRequest: " + subtaskId); } diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java index 9ac392b6a7a..524d2310632 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java @@ -18,8 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.fake.source; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -34,9 +34,11 @@ import java.io.IOException; import java.lang.reflect.Array; +import java.math.BigDecimal; import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import java.util.function.Function; public class FakeDataGenerator { private final CatalogTable catalogTable; @@ -71,12 +73,11 @@ private SeaTunnelRow convertRow(FakeConfig.RowData rowData) { } private SeaTunnelRow randomRow() { - SeaTunnelRowType rowType = catalogTable.getSeaTunnelRowType(); - String[] fieldNames = rowType.getFieldNames(); - SeaTunnelDataType[] fieldTypes = rowType.getFieldTypes(); - List randomRow = new ArrayList<>(fieldNames.length); - for (SeaTunnelDataType fieldType : fieldTypes) { - randomRow.add(randomColumnValue(fieldType)); + // Generate random data according to the data type and data colum of the table + List physicalColumns = catalogTable.getTableSchema().getColumns(); + List randomRow = new ArrayList<>(physicalColumns.size()); + for (Column column : physicalColumns) { + randomRow.add(randomColumnValue(column)); } SeaTunnelRow seaTunnelRow = new SeaTunnelRow(randomRow.toArray()); seaTunnelRow.setTableId(tableId); @@ -103,7 +104,8 @@ public List generateFakedRows(int rowNum) { } @SuppressWarnings("magicnumber") - private Object randomColumnValue(SeaTunnelDataType fieldType) { + private Object randomColumnValue(Column column) { + SeaTunnelDataType fieldType = column.getDataType(); switch (fieldType.getSqlType()) { case ARRAY: ArrayType arrayType = (ArrayType) fieldType; @@ -111,7 +113,7 @@ private Object randomColumnValue(SeaTunnelDataType fieldType) { int length = fakeConfig.getArraySize(); Object array = Array.newInstance(elementType.getTypeClass(), length); for (int i = 0; i < length; i++) { - Object value = randomColumnValue(elementType); + Object value = randomColumnValue(column.copy(elementType)); Array.set(array, i, value); } return array; @@ -122,59 +124,57 @@ private Object randomColumnValue(SeaTunnelDataType fieldType) { HashMap objectMap = new HashMap<>(); int mapSize = fakeConfig.getMapSize(); for (int i = 0; i < mapSize; i++) { - Object key = randomColumnValue(keyType); - Object value = randomColumnValue(valueType); + Object key = randomColumnValue(column.copy(keyType)); + Object value = randomColumnValue(column.copy(valueType)); objectMap.put(key, value); } return objectMap; case STRING: - return fakeDataRandomUtils.randomString(); + return value(column, String::toString, fakeDataRandomUtils::randomString); case BOOLEAN: - return fakeDataRandomUtils.randomBoolean(); + return value(column, Boolean::parseBoolean, fakeDataRandomUtils::randomBoolean); case TINYINT: - return fakeDataRandomUtils.randomTinyint(); + return value(column, Byte::parseByte, fakeDataRandomUtils::randomTinyint); case SMALLINT: - return fakeDataRandomUtils.randomSmallint(); + return value(column, Short::parseShort, fakeDataRandomUtils::randomSmallint); case INT: - return fakeDataRandomUtils.randomInt(); + return value(column, Integer::parseInt, fakeDataRandomUtils::randomInt); case BIGINT: - return fakeDataRandomUtils.randomBigint(); + return value(column, Long::parseLong, fakeDataRandomUtils::randomBigint); case FLOAT: - return fakeDataRandomUtils.randomFloat(); + return value(column, Float::parseFloat, fakeDataRandomUtils::randomFloat); case DOUBLE: - return fakeDataRandomUtils.randomDouble(); + return value(column, Double::parseDouble, fakeDataRandomUtils::randomDouble); case DECIMAL: - DecimalType decimalType = (DecimalType) fieldType; - return fakeDataRandomUtils.randomBigDecimal( - decimalType.getPrecision(), decimalType.getScale()); + return value(column, BigDecimal::new, fakeDataRandomUtils::randomBigDecimal); case NULL: return null; case BYTES: - return fakeDataRandomUtils.randomBytes(); + return value(column, String::getBytes, fakeDataRandomUtils::randomBytes); case DATE: - return fakeDataRandomUtils.randomLocalDate(); + return value(column, String::toString, fakeDataRandomUtils::randomLocalDate); case TIME: - return fakeDataRandomUtils.randomLocalTime(); + return value(column, String::toString, fakeDataRandomUtils::randomLocalTime); case TIMESTAMP: - return fakeDataRandomUtils.randomLocalDateTime(); + return value(column, String::toString, fakeDataRandomUtils::randomLocalDateTime); case ROW: SeaTunnelDataType[] fieldTypes = ((SeaTunnelRowType) fieldType).getFieldTypes(); Object[] objects = new Object[fieldTypes.length]; for (int i = 0; i < fieldTypes.length; i++) { - Object object = randomColumnValue(fieldTypes[i]); + Object object = randomColumnValue(column.copy(fieldTypes[i])); objects[i] = object; } return new SeaTunnelRow(objects); case BINARY_VECTOR: - return fakeDataRandomUtils.randomBinaryVector(); + return fakeDataRandomUtils.randomBinaryVector(column); case FLOAT_VECTOR: - return fakeDataRandomUtils.randomFloatVector(); + return fakeDataRandomUtils.randomFloatVector(column); case FLOAT16_VECTOR: - return fakeDataRandomUtils.randomFloat16Vector(); + return fakeDataRandomUtils.randomFloat16Vector(column); case BFLOAT16_VECTOR: - return fakeDataRandomUtils.randomBFloat16Vector(); + return fakeDataRandomUtils.randomBFloat16Vector(column); case SPARSE_FLOAT_VECTOR: - return fakeDataRandomUtils.randomSparseFloatVector(); + return fakeDataRandomUtils.randomSparseFloatVector(column); default: // never got in there throw new FakeConnectorException( @@ -182,4 +182,12 @@ private Object randomColumnValue(SeaTunnelDataType fieldType) { "SeaTunnel Fake source connector not support this data type"); } } + + private static T value( + Column column, Function convert, Function generate) { + if (column.getDefaultValue() != null) { + return convert.apply(column.getDefaultValue().toString()); + } + return generate.apply(column); + } } diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java index 95758cb971e..063ece63d2e 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java @@ -20,8 +20,6 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.source.event.ReaderCloseEvent; -import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.fake.config.FakeConfig; import org.apache.seatunnel.connectors.seatunnel.fake.config.MultipleTableFakeSourceConfig; @@ -73,14 +71,10 @@ public FakeSourceReader( } @Override - public void open() { - context.getEventListener().onEvent(new ReaderOpenEvent()); - } + public void open() {} @Override - public void close() { - context.getEventListener().onEvent(new ReaderCloseEvent()); - } + public void close() {} @Override @SuppressWarnings("MagicNumber") diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java index ecd6d509149..311e8183766 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java @@ -18,8 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.fake.source; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; -import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; import org.apache.seatunnel.connectors.seatunnel.fake.config.FakeConfig; import org.apache.seatunnel.connectors.seatunnel.fake.config.MultipleTableFakeSourceConfig; import org.apache.seatunnel.connectors.seatunnel.fake.state.FakeSourceState; @@ -58,9 +56,7 @@ public FakeSourceSplitEnumerator( } @Override - public void open() { - enumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); - } + public void open() {} @Override public void run() throws Exception { @@ -69,9 +65,7 @@ public void run() throws Exception { } @Override - public void close() throws IOException { - enumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); - } + public void close() throws IOException {} @Override public void addSplitsBack(List splits, int subtaskId) { diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java index 8a8a14dc70b..c4a038ff1a1 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.fake.utils; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.common.utils.BufferUtils; import org.apache.seatunnel.connectors.seatunnel.fake.config.FakeConfig; @@ -25,6 +27,7 @@ import org.apache.commons.lang3.RandomUtils; import java.math.BigDecimal; +import java.math.RoundingMode; import java.nio.ByteBuffer; import java.time.LocalDate; import java.time.LocalDateTime; @@ -45,30 +48,34 @@ private static T randomFromList(List list) { return list.get(index); } - public Boolean randomBoolean() { + public Boolean randomBoolean(Column column) { return RandomUtils.nextInt(0, 2) == 1; } - public BigDecimal randomBigDecimal(int precision, int scale) { + public BigDecimal randomBigDecimal(Column column) { + DecimalType dataType = (DecimalType) column.getDataType(); return new BigDecimal( - RandomStringUtils.randomNumeric(precision - scale) + RandomStringUtils.randomNumeric(dataType.getPrecision() - dataType.getScale()) + "." - + RandomStringUtils.randomNumeric(scale)); + + RandomStringUtils.randomNumeric(dataType.getScale())); } - public byte[] randomBytes() { + public byte[] randomBytes(Column column) { return RandomStringUtils.randomAlphabetic(fakeConfig.getBytesLength()).getBytes(); } - public String randomString() { + public String randomString(Column column) { List stringTemplate = fakeConfig.getStringTemplate(); if (!CollectionUtils.isEmpty(stringTemplate)) { return randomFromList(stringTemplate); } - return RandomStringUtils.randomAlphabetic(fakeConfig.getStringLength()); + return RandomStringUtils.randomAlphabetic( + column.getColumnLength() != null + ? column.getColumnLength().intValue() + : fakeConfig.getStringLength()); } - public Byte randomTinyint() { + public Byte randomTinyint(Column column) { List tinyintTemplate = fakeConfig.getTinyintTemplate(); if (!CollectionUtils.isEmpty(tinyintTemplate)) { return randomFromList(tinyintTemplate).byteValue(); @@ -76,7 +83,7 @@ public Byte randomTinyint() { return (byte) RandomUtils.nextInt(fakeConfig.getTinyintMin(), fakeConfig.getTinyintMax()); } - public Short randomSmallint() { + public Short randomSmallint(Column column) { List smallintTemplate = fakeConfig.getSmallintTemplate(); if (!CollectionUtils.isEmpty(smallintTemplate)) { return randomFromList(smallintTemplate).shortValue(); @@ -85,7 +92,7 @@ public Short randomSmallint() { RandomUtils.nextInt(fakeConfig.getSmallintMin(), fakeConfig.getSmallintMax()); } - public Integer randomInt() { + public Integer randomInt(Column column) { List intTemplate = fakeConfig.getIntTemplate(); if (!CollectionUtils.isEmpty(intTemplate)) { return randomFromList(intTemplate); @@ -93,7 +100,7 @@ public Integer randomInt() { return RandomUtils.nextInt(fakeConfig.getIntMin(), fakeConfig.getIntMax()); } - public Long randomBigint() { + public Long randomBigint(Column column) { List bigTemplate = fakeConfig.getBigTemplate(); if (!CollectionUtils.isEmpty(bigTemplate)) { return randomFromList(bigTemplate); @@ -101,32 +108,39 @@ public Long randomBigint() { return RandomUtils.nextLong(fakeConfig.getBigintMin(), fakeConfig.getBigintMax()); } - public Float randomFloat() { + public Float randomFloat(Column column) { List floatTemplate = fakeConfig.getFloatTemplate(); if (!CollectionUtils.isEmpty(floatTemplate)) { return randomFromList(floatTemplate).floatValue(); } - return RandomUtils.nextFloat( - (float) fakeConfig.getFloatMin(), (float) fakeConfig.getFloatMax()); + float v = + RandomUtils.nextFloat( + (float) fakeConfig.getFloatMin(), (float) fakeConfig.getFloatMax()); + return column.getScale() == null + ? v + : new BigDecimal(v).setScale(column.getScale(), RoundingMode.HALF_UP).floatValue(); } - public Double randomDouble() { + public Double randomDouble(Column column) { List doubleTemplate = fakeConfig.getDoubleTemplate(); if (!CollectionUtils.isEmpty(doubleTemplate)) { return randomFromList(doubleTemplate); } - return RandomUtils.nextDouble(fakeConfig.getDoubleMin(), fakeConfig.getDoubleMax()); + double v = RandomUtils.nextDouble(fakeConfig.getDoubleMin(), fakeConfig.getDoubleMax()); + return column.getScale() == null + ? v + : new BigDecimal(v).setScale(column.getScale(), RoundingMode.HALF_UP).floatValue(); } - public LocalDate randomLocalDate() { - return randomLocalDateTime().toLocalDate(); + public LocalDate randomLocalDate(Column column) { + return randomLocalDateTime(column).toLocalDate(); } - public LocalTime randomLocalTime() { - return randomLocalDateTime().toLocalTime(); + public LocalTime randomLocalTime(Column column) { + return randomLocalDateTime(column).toLocalTime(); } - public LocalDateTime randomLocalDateTime() { + public LocalDateTime randomLocalDateTime(Column column) { int year; int month; int day; @@ -172,15 +186,20 @@ public LocalDateTime randomLocalDateTime() { return LocalDateTime.of(year, month, day, hour, minute, second); } - public ByteBuffer randomBinaryVector() { - int byteCount = fakeConfig.getBinaryVectorDimension() / 8; + public ByteBuffer randomBinaryVector(Column column) { + int byteCount = + (column.getScale() != null) + ? column.getScale() / 8 + : fakeConfig.getBinaryVectorDimension() / 8; // binary vector doesn't care endian since each byte is independent return ByteBuffer.wrap(RandomUtils.nextBytes(byteCount)); } - public ByteBuffer randomFloatVector() { - Float[] floatVector = new Float[fakeConfig.getVectorDimension()]; - for (int i = 0; i < fakeConfig.getVectorDimension(); i++) { + public ByteBuffer randomFloatVector(Column column) { + int count = + (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); + Float[] floatVector = new Float[count]; + for (int i = 0; i < count; i++) { floatVector[i] = RandomUtils.nextFloat( fakeConfig.getVectorFloatMin(), fakeConfig.getVectorFloatMax()); @@ -188,9 +207,11 @@ public ByteBuffer randomFloatVector() { return BufferUtils.toByteBuffer(floatVector); } - public ByteBuffer randomFloat16Vector() { - Short[] float16Vector = new Short[fakeConfig.getVectorDimension()]; - for (int i = 0; i < fakeConfig.getVectorDimension(); i++) { + public ByteBuffer randomFloat16Vector(Column column) { + int count = + (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); + Short[] float16Vector = new Short[count]; + for (int i = 0; i < count; i++) { float value = RandomUtils.nextFloat( fakeConfig.getVectorFloatMin(), fakeConfig.getVectorFloatMax()); @@ -199,9 +220,11 @@ public ByteBuffer randomFloat16Vector() { return BufferUtils.toByteBuffer(float16Vector); } - public ByteBuffer randomBFloat16Vector() { - Short[] bfloat16Vector = new Short[fakeConfig.getVectorDimension()]; - for (int i = 0; i < fakeConfig.getVectorDimension(); i++) { + public ByteBuffer randomBFloat16Vector(Column column) { + int count = + (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); + Short[] bfloat16Vector = new Short[count]; + for (int i = 0; i < count; i++) { float value = RandomUtils.nextFloat( fakeConfig.getVectorFloatMin(), fakeConfig.getVectorFloatMax()); @@ -210,10 +233,10 @@ public ByteBuffer randomBFloat16Vector() { return BufferUtils.toByteBuffer(bfloat16Vector); } - public Map randomSparseFloatVector() { + public Map randomSparseFloatVector(Column column) { Map sparseVector = new HashMap<>(); - - Integer nonZeroElements = fakeConfig.getVectorDimension(); + int nonZeroElements = + (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); while (nonZeroElements > 0) { Integer index = RandomUtils.nextInt(); Float value = diff --git a/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java b/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java index c1cd826cb0a..e33883f554e 100644 --- a/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java +++ b/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -38,6 +39,7 @@ import java.io.FileNotFoundException; import java.net.URISyntaxException; import java.net.URL; +import java.nio.ByteBuffer; import java.nio.file.Paths; import java.util.Arrays; import java.util.List; @@ -141,6 +143,59 @@ public void testVectorParse(String conf) throws FileNotFoundException, URISyntax Assertions.assertNotNull(seaTunnelRows); } + @ParameterizedTest + @ValueSource(strings = {"fake-data.column.conf"}) + public void testColumnDataParse(String conf) throws FileNotFoundException, URISyntaxException { + ReadonlyConfig testConfig = getTestConfigFile(conf); + FakeConfig fakeConfig = FakeConfig.buildWithConfig(testConfig); + FakeDataGenerator fakeDataGenerator = new FakeDataGenerator(fakeConfig); + List seaTunnelRows = + fakeDataGenerator.generateFakedRows(fakeConfig.getRowNum()); + seaTunnelRows.forEach( + seaTunnelRow -> { + Assertions.assertEquals( + seaTunnelRow.getField(0).toString(), "Andersen's Fairy Tales"); + Assertions.assertEquals(seaTunnelRow.getField(1).toString().length(), 100); + Assertions.assertEquals(seaTunnelRow.getField(2).toString(), "10.1"); + Assertions.assertNotNull(seaTunnelRow.getField(3).toString()); + Assertions.assertNotNull(seaTunnelRow.getField(4).toString()); + // VectorType.VECTOR_FLOAT_TYPE + Assertions.assertEquals( + 8, ((ByteBuffer) seaTunnelRow.getField(5)).capacity() / 4); + // VectorType.VECTOR_BINARY_TYPE + Assertions.assertEquals( + 16, ((ByteBuffer) seaTunnelRow.getField(6)).capacity() * 8); + // VectorType.VECTOR_FLOAT16_TYPE + Assertions.assertEquals( + 8, ((ByteBuffer) seaTunnelRow.getField(7)).capacity() / 2); + // VectorType.VECTOR_BFLOAT16_TYPE + Assertions.assertEquals( + 8, ((ByteBuffer) seaTunnelRow.getField(8)).capacity() / 2); + // VectorType.VECTOR_SPARSE_FLOAT_TYPE + Assertions.assertEquals(8, ((Map) seaTunnelRow.getField(9)).size()); + Assertions.assertEquals( + 268, + seaTunnelRow.getBytesSize( + new SeaTunnelRowType( + new String[] { + "field1", "field2", "field3", "field4", "field5", + "field6", "field7", "field8", "field9", "field10" + }, + new SeaTunnelDataType[] { + BasicType.STRING_TYPE, + BasicType.STRING_TYPE, + BasicType.FLOAT_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + VectorType.VECTOR_FLOAT_TYPE, + VectorType.VECTOR_BINARY_TYPE, + VectorType.VECTOR_FLOAT16_TYPE, + VectorType.VECTOR_BFLOAT16_TYPE, + VectorType.VECTOR_SPARSE_FLOAT_TYPE + }))); + }); + } + private ReadonlyConfig getTestConfigFile(String configFile) throws FileNotFoundException, URISyntaxException { if (!configFile.startsWith("/")) { diff --git a/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf b/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf index 96e82ee41c5..e3f0d7ee267 100644 --- a/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf +++ b/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf @@ -23,7 +23,7 @@ FakeSource { string.length = 10 schema = { fields { - c_map = "map>" + c_map = "map>" c_array = "array" c_string = string c_boolean = boolean diff --git a/seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf b/seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf new file mode 100644 index 00000000000..9a1515264e0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf @@ -0,0 +1,97 @@ +# +# 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. +# + + FakeSource { + row.num = 5 + vector.float.max=1 + vector.float.min=0 + float.max = 2 + float.min = 0 + double.max = 4 + double.min = 2 + + # low weight + string.length = 4 + vector.dimension= 4 + binary.vector.dimension=8 + # end + + schema = { + columns = [ + { + name = book_name + type = string + defaultValue = "Andersen's Fairy Tales" + comment = "book name" + }, + { + name = book_reader_testimonials + type = string + columnLength = 100 + comment = "book reader testimonials" + }, + { + name = book_price + type = float + defaultValue = 10.1 + comment = "book price" + }, + { + name = book_percentage_popularity + type = float + columnScale = 4 + comment = "book percentage popularity" + }, + { + name = book_distribution_law + type = double + columnScale = 2 + comment = "book distribution law" + }, + { + name = book_intro_1 + type = float_vector + columnScale =8 + comment = "vector" + }, + { + name = book_intro_2 + type = binary_vector + columnScale = 16 + comment = "vector" + }, + { + name = book_intro_3 + type = float16_vector + columnScale =8 + comment = "vector" + }, + { + name = book_intro_4 + type = bfloat16_vector + columnScale =8 + comment = "vector" + }, + { + name = book_intro_5 + type = sparse_float_vector + columnScale =8 + comment = "vector" + } + ] + } + } \ No newline at end of file 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 d63c8ef8efd..66b4eb967b5 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 @@ -19,6 +19,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.common.config.TypesafeConfigUtils; import lombok.Builder; @@ -82,10 +83,9 @@ public class HbaseParameters implements Serializable { @Builder.Default private HbaseConfig.EnCoding enCoding = ENCODING.defaultValue(); - public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { + public static HbaseParameters buildWithConfig(ReadonlyConfig config) { HbaseParametersBuilder builder = HbaseParameters.builder(); - builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); - String table = pluginConfig.getString(TABLE.key()); + String table = config.get(TABLE); int colonIndex = table.indexOf(':'); if (colonIndex != -1) { String namespace = table.substring(0, colonIndex); @@ -96,38 +96,21 @@ public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { builder.namespace("default"); } - builder.rowkeyColumns(pluginConfig.getStringList(ROWKEY_COLUMNS.key())); - builder.familyNames( - TypesafeConfigUtils.configToMap(pluginConfig.getConfig(FAMILY_NAME.key()))); - - // optional parameters - if (pluginConfig.hasPath(HBASE_TTL_CONFIG.key())) { - builder.ttl(pluginConfig.getLong(HBASE_TTL_CONFIG.key())); - } - if (pluginConfig.hasPath(ROWKEY_DELIMITER.key())) { - builder.rowkeyDelimiter(pluginConfig.getString(ROWKEY_DELIMITER.key())); - } - if (pluginConfig.hasPath(VERSION_COLUMN.key())) { - builder.versionColumn(pluginConfig.getString(VERSION_COLUMN.key())); - } - if (pluginConfig.hasPath(NULL_MODE.key())) { - String nullMode = pluginConfig.getString(NULL_MODE.key()); - builder.nullMode(HbaseConfig.NullMode.valueOf(nullMode.toUpperCase())); - } - if (pluginConfig.hasPath(WAL_WRITE.key())) { - builder.walWrite(pluginConfig.getBoolean(WAL_WRITE.key())); - } - if (pluginConfig.hasPath(WRITE_BUFFER_SIZE.key())) { - builder.writeBufferSize(pluginConfig.getInt(WRITE_BUFFER_SIZE.key())); - } - if (pluginConfig.hasPath(ENCODING.key())) { - String encoding = pluginConfig.getString(ENCODING.key()); - builder.enCoding(HbaseConfig.EnCoding.valueOf(encoding.toUpperCase())); - } - if (pluginConfig.hasPath(HBASE_EXTRA_CONFIG.key())) { - Config extraConfig = pluginConfig.getConfig(HBASE_EXTRA_CONFIG.key()); - builder.hbaseExtraConfig(TypesafeConfigUtils.configToMap(extraConfig)); - } + // required parameters + builder.zookeeperQuorum(config.get(ZOOKEEPER_QUORUM)); + builder.rowkeyColumns(config.get(ROWKEY_COLUMNS)); + builder.familyNames(config.get(FAMILY_NAME)); + + builder.rowkeyDelimiter(config.get(ROWKEY_DELIMITER)); + builder.versionColumn(config.get(VERSION_COLUMN)); + String nullMode = String.valueOf(config.get(NULL_MODE)); + builder.nullMode(HbaseConfig.NullMode.valueOf(nullMode.toUpperCase())); + builder.walWrite(config.get(WAL_WRITE)); + builder.writeBufferSize(config.get(WRITE_BUFFER_SIZE)); + String encoding = String.valueOf(config.get(ENCODING)); + builder.enCoding(HbaseConfig.EnCoding.valueOf(encoding.toUpperCase())); + builder.hbaseExtraConfig(config.get(HBASE_EXTRA_CONFIG)); + builder.ttl(config.get(HBASE_TTL_CONFIG)); 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 14f8ec8c7ef..6f350e60e77 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 @@ -23,6 +23,8 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SchemaSaveMode; import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.shade.com.typesafe.config.Config; + import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSink; import org.apache.seatunnel.api.sink.SupportSaveMode; @@ -30,9 +32,12 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseAggregatedCommitInfo; @@ -65,7 +70,7 @@ public class HbaseSink private int versionColumnIndex = -1; public HbaseSink(ReadonlyConfig config, CatalogTable catalogTable) { - this.hbaseParameters = HbaseParameters.buildWithSinkConfig(config.toConfig()); + this.hbaseParameters = HbaseParameters.buildWithConfig(config); this.config = config; this.catalogTable = catalogTable; this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java index 9fab21954a7..f7620190b23 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java @@ -21,10 +21,15 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.connector.TableSink; +import org.apache.seatunnel.api.sink.SinkCommonOptions; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import com.google.auto.service.AutoService; @@ -45,6 +50,8 @@ @AutoService(Factory.class) public class HbaseSinkFactory implements TableSinkFactory { + public static final String IDENTIFIER = "Hbase"; + @Override public String factoryIdentifier() { return HbaseIdentifier.IDENTIFIER_NAME; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java index a41a2386297..ac2b8414a1b 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -58,7 +59,7 @@ public class HbaseSinkWriter private int versionColumnIndex; - private String writeAllColumnFamily; + private String defaultFamilyName = "value"; public HbaseSinkWriter( SeaTunnelRowType seaTunnelRowType, @@ -71,7 +72,8 @@ public HbaseSinkWriter( this.versionColumnIndex = versionColumnIndex; if (hbaseParameters.getFamilyNames().size() == 1) { - this.writeAllColumnFamily = hbaseParameters.getFamilyNames().get(ALL_COLUMNS); + defaultFamilyName = + hbaseParameters.getFamilyNames().getOrDefault(ALL_COLUMNS, defaultFamilyName); } this.hbaseClient = HbaseClient.createInstance(hbaseParameters); @@ -121,11 +123,7 @@ private Put convertRowToPut(SeaTunnelRow row) { String fieldName = seaTunnelRowType.getFieldName(writeColumnIndex); Map configurationFamilyNames = hbaseParameters.getFamilyNames(); String familyName = - configurationFamilyNames.getOrDefault(fieldName, writeAllColumnFamily); - if (!configurationFamilyNames.containsKey(ALL_COLUMNS) - && !configurationFamilyNames.containsKey(fieldName)) { - continue; - } + hbaseParameters.getFamilyNames().getOrDefault(fieldName, defaultFamilyName); byte[] bytes = convertColumnToBytes(row, writeColumnIndex); if (bytes != null) { put.addColumn(Bytes.toBytes(familyName), Bytes.toBytes(fieldName), bytes); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java index 0f4610783af..bbb590502cf 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java @@ -50,25 +50,20 @@ public class IcebergCatalogLoader implements Serializable { private static final long serialVersionUID = -6003040601422350869L; private static final List HADOOP_CONF_FILES = ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); - private CommonConfig config; + private final CommonConfig config; public IcebergCatalogLoader(CommonConfig config) { this.config = config; } public Catalog loadCatalog() { - // When using the seatunel engine, set the current class loader to prevent loading failures + // When using the SeaTunnel engine, set the current class loader to prevent loading failures Thread.currentThread().setContextClassLoader(IcebergCatalogLoader.class.getClassLoader()); return CatalogUtil.buildIcebergCatalog( config.getCatalogName(), config.getCatalogProps(), loadHadoopConfig(config)); } - /** - * Loading Hadoop configuration through reflection - * - * @param config - * @return - */ + /** Loading Hadoop configuration through reflection */ public Object loadHadoopConfig(CommonConfig config) { Class configClass = DynClasses.builder() diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java index 520f9bdbac9..fc28001b2ca 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java @@ -58,9 +58,9 @@ @Slf4j public class IcebergCatalog implements Catalog { - private String catalogName; - private ReadonlyConfig readonlyConfig; - private IcebergCatalogLoader icebergCatalogLoader; + private final String catalogName; + private final ReadonlyConfig readonlyConfig; + private final IcebergCatalogLoader icebergCatalogLoader; private org.apache.iceberg.catalog.Catalog catalog; public IcebergCatalog(String catalogName, ReadonlyConfig readonlyConfig) { @@ -224,22 +224,21 @@ public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) public CatalogTable toCatalogTable(Table icebergTable, TablePath tablePath) { List columns = icebergTable.schema().columns(); TableSchema.Builder builder = TableSchema.builder(); - columns.stream() - .forEach( - nestedField -> { - String name = nestedField.name(); - SeaTunnelDataType seaTunnelType = - SchemaUtils.toSeaTunnelType(name, nestedField.type()); - PhysicalColumn physicalColumn = - PhysicalColumn.of( - name, - seaTunnelType, - (Long) null, - true, - null, - nestedField.doc()); - builder.column(physicalColumn); - }); + columns.forEach( + nestedField -> { + String name = nestedField.name(); + SeaTunnelDataType seaTunnelType = + SchemaUtils.toSeaTunnelType(name, nestedField.type()); + PhysicalColumn physicalColumn = + PhysicalColumn.of( + name, + seaTunnelType, + (Long) null, + true, + null, + nestedField.doc()); + builder.column(physicalColumn); + }); List partitionKeys = icebergTable.spec().fields().stream() diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java index 8c699b34402..f46928456fb 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java @@ -92,17 +92,17 @@ private NameMapping createNameMapping(Table table) { return nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; } - public Record convert(Object row, SeaTunnelDataType rowType) { + public Record convert(Object row, SeaTunnelDataType rowType) { return convertStructValue(row, rowType, tableSchema.asStruct(), -1, null); } - public Record convert(Object row, SeaTunnelDataType rowType, SchemaChangeWrapper wrapper) { + public Record convert(Object row, SeaTunnelDataType rowType, SchemaChangeWrapper wrapper) { return convertStructValue(row, rowType, tableSchema.asStruct(), -1, wrapper); } protected GenericRecord convertStructValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.StructType schema, int parentFieldId, SchemaChangeWrapper wrapper) { @@ -120,15 +120,7 @@ protected GenericRecord convertStructValue( } } - /** - * Convert RowType - * - * @param row - * @param fromType - * @param schema - * @param structFieldId - * @return - */ + /** Convert RowType */ private GenericRecord convertToStruct( SeaTunnelRow row, SeaTunnelRowType fromType, @@ -179,7 +171,7 @@ private GenericRecord convertToStruct( public Object convertValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Type type, int fieldId, SchemaChangeWrapper wrapper) { @@ -252,7 +244,7 @@ private Map createStructNameMap(Types.StructType sche protected List convertListValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.ListType type, SchemaChangeWrapper wrapper) { Preconditions.checkArgument(value.getClass().isArray()); @@ -269,7 +261,7 @@ protected List convertListValue( protected Map convertMapValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.MapType type, SchemaChangeWrapper wrapper) { Preconditions.checkArgument(value instanceof Map); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java index 65bccbdb893..a1d43d6acfd 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java @@ -61,9 +61,9 @@ public class IcebergSink SupportSaveMode, SupportMultiTableSink { private static String PLUGIN_NAME = "Iceberg"; - private SinkConfig config; - private ReadonlyConfig readonlyConfig; - private CatalogTable catalogTable; + private final SinkConfig config; + private final ReadonlyConfig readonlyConfig; + private final CatalogTable catalogTable; public IcebergSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) { this.readonlyConfig = pluginConfig; diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java index aed6522ca87..3a5e22b93b4 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java @@ -54,13 +54,12 @@ public class IcebergSinkWriter implements SinkWriter, SupportMultiTableSinkWriter { private SeaTunnelRowType rowType; - private SinkConfig config; - private IcebergTableLoader icebergTableLoader; + private final SinkConfig config; + private final IcebergTableLoader icebergTableLoader; private RecordWriter writer; - private IcebergFilesCommitter filesCommitter; - private List results = Lists.newArrayList(); + private final IcebergFilesCommitter filesCommitter; + private final List results = Lists.newArrayList(); private String commitUser = UUID.randomUUID().toString(); - private long checkpointId; private final DataTypeChangeEventHandler dataTypeChangeEventHandler; @@ -77,7 +76,6 @@ public IcebergSinkWriter( tryCreateRecordWriter(); if (Objects.nonNull(states) && !states.isEmpty()) { this.commitUser = states.get(0).getCommitUser(); - this.checkpointId = states.get(0).getCheckpointId(); preCommit(states); } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java index 2be206ebb6e..06b48591df1 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java @@ -54,7 +54,7 @@ public class IcebergRecordWriter implements RecordWriter { private final List writerResults; private TaskWriter writer; private RowConverter recordConverter; - private IcebergWriterFactory writerFactory; + private final IcebergWriterFactory writerFactory; public IcebergRecordWriter(Table table, IcebergWriterFactory writerFactory, SinkConfig config) { this.config = config; @@ -122,12 +122,7 @@ private void changeColumn( } } - /** - * apply schema update - * - * @param updates - * @return - */ + /** apply schema update */ private void applySchemaUpdate(SchemaChangeWrapper updates) { // complete the current file flush(); @@ -169,7 +164,4 @@ private void flush() { table.spec().partitionType())); writer = null; } - - @Override - public void close() {} } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java index 67809088ef7..2ee7c3d6d74 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java @@ -40,9 +40,6 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import lombok.extern.slf4j.Slf4j; import java.util.List; @@ -58,7 +55,6 @@ @Slf4j public class IcebergWriterFactory { - private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); private final IcebergTableLoader tableLoader; private final SinkConfig config; diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java index 7a2fdf9d4ff..c56f3f2f00e 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java @@ -65,12 +65,12 @@ public class IcebergSource private static final long serialVersionUID = 4343414808223919870L; - private SourceConfig sourceConfig; - private Schema tableSchema; - private Schema projectedSchema; - private SeaTunnelRowType seaTunnelRowType; + private final SourceConfig sourceConfig; + private final Schema tableSchema; + private final Schema projectedSchema; + private final SeaTunnelRowType seaTunnelRowType; private JobContext jobContext; - private CatalogTable catalogTable; + private final CatalogTable catalogTable; public IcebergSource(ReadonlyConfig config, CatalogTable catalogTable) { this.sourceConfig = SourceConfig.loadConfig(config); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java index 6c99eb409c1..01343a119f6 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java @@ -40,7 +40,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.UpdateSchema; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; @@ -106,21 +105,11 @@ public static Table autoCreateTable( SinkConfig config = new SinkConfig(readonlyConfig); // build auto create table Map options = new HashMap<>(table.getOptions()); - options.put(TableProperties.FORMAT_VERSION, "2"); // override options.putAll(config.getAutoCreateProps()); return createTable(catalog, toIcebergTableIdentifier(tablePath), config, schema, options); } - /** - * For local test - * - * @param catalog - * @param tableIdentifier - * @param config - * @param rowType - * @return - */ public static Table autoCreateTable( Catalog catalog, TableIdentifier tableIdentifier, @@ -180,7 +169,7 @@ private static Table createTable( Optional pkId = structType.fields().stream() .filter(nestedField -> nestedField.name().equals(pk)) - .map(nestedField -> nestedField.fieldId()) + .map(Types.NestedField::fieldId) .findFirst(); if (!pkId.isPresent()) { throw new IllegalArgumentException( @@ -196,23 +185,14 @@ private static Table createTable( structType .fields() .forEach( - field -> { - fields.add( - identifierFieldIds.contains(field.fieldId()) - ? field.asRequired() - : field.asOptional()); - }); + field -> + fields.add( + identifierFieldIds.contains(field.fieldId()) + ? field.asRequired() + : field.asOptional())); return new Schema(fields, identifierFieldIds); } - public static TableIdentifier toIcebergTableIdentifierFromCatalogTable( - CatalogTable catalogTable) { - org.apache.seatunnel.api.table.catalog.TableIdentifier tableIdentifier = - catalogTable.getTableId(); - return TableIdentifier.of( - tableIdentifier.getDatabaseName(), tableIdentifier.getTableName()); - } - public static TableIdentifier toIcebergTableIdentifier(TablePath tablePath) { return TableIdentifier.of(tablePath.getDatabaseName(), tablePath.getTableName()); } @@ -221,12 +201,7 @@ public static TablePath toTablePath(TableIdentifier tableIdentifier) { return TablePath.of(tableIdentifier.namespace().toString(), tableIdentifier.name()); } - /** - * Commit table schema updates - * - * @param table - * @param wrapper - */ + /** Commit table schema updates */ private static void commitSchemaUpdates(Table table, SchemaChangeWrapper wrapper) { // get the latest schema in case another process updated it table.refresh(); @@ -249,7 +224,7 @@ private static void commitSchemaUpdates(Table table, SchemaChangeWrapper wrapper .collect(toList()); // Rename column name - List changeColumns = wrapper.changeColumns().stream().collect(toList()); + List changeColumns = new ArrayList<>(wrapper.changeColumns()); if (addColumns.isEmpty() && modifyColumns.isEmpty() @@ -294,7 +269,7 @@ public static SeaTunnelDataType toSeaTunnelType(String fieldName, Type type) return IcebergTypeMapper.mapping(fieldName, type); } - public static Type toIcebergType(SeaTunnelDataType rowType) { + public static Type toIcebergType(SeaTunnelDataType rowType) { return IcebergTypeMapper.toIcebergType(rowType); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java index c2f2405ee00..c412ca92186 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; public interface JdbcCatalogOptions { Option BASE_URL = @@ -59,7 +60,10 @@ public interface JdbcCatalogOptions { + "For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'."); OptionRule.Builder BASE_RULE = - OptionRule.builder().required(BASE_URL).required(USERNAME, PASSWORD).optional(SCHEMA); + OptionRule.builder() + .required(BASE_URL) + .required(USERNAME, PASSWORD) + .optional(SCHEMA, JdbcOptions.DECIMAL_TYPE_NARROWING); Option TABLE_PREFIX = Options.key("tablePrefix") 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 5aa6dcd874d..ccbbfb509dd 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 @@ -25,6 +25,7 @@ import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeMapper; @@ -71,13 +72,32 @@ public class OracleCatalog extends AbstractJdbcCatalog { + "ORDER BY \n" + " cols.column_id \n"; + private boolean decimalTypeNarrowing; + public OracleCatalog( String catalogName, String username, String pwd, JdbcUrlUtil.UrlInfo urlInfo, String defaultSchema) { + this( + catalogName, + username, + pwd, + urlInfo, + defaultSchema, + JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue()); + } + + public OracleCatalog( + String catalogName, + String username, + String pwd, + JdbcUrlUtil.UrlInfo urlInfo, + String defaultSchema, + boolean decimalTypeNarrowing) { super(catalogName, username, pwd, urlInfo, defaultSchema); + this.decimalTypeNarrowing = decimalTypeNarrowing; } @Override @@ -162,7 +182,7 @@ protected Column buildColumn(ResultSet resultSet) throws SQLException { .defaultValue(defaultValue) .comment(columnComment) .build(); - return OracleTypeConverter.INSTANCE.convert(typeDefine); + return new OracleTypeConverter(decimalTypeNarrowing).convert(typeDefine); } @Override @@ -183,7 +203,8 @@ private List listTables() { @Override public CatalogTable getTable(String sqlQuery) throws SQLException { Connection defaultConnection = getConnection(defaultUrl); - return CatalogUtils.getCatalogTable(defaultConnection, sqlQuery, new OracleTypeMapper()); + return CatalogUtils.getCatalogTable( + defaultConnection, sqlQuery, new OracleTypeMapper(decimalTypeNarrowing)); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java index 7c90c79347a..2b51d976212 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; import com.google.auto.service.AutoService; @@ -52,7 +53,8 @@ public Catalog createCatalog(String catalogName, ReadonlyConfig options) { options.get(JdbcCatalogOptions.USERNAME), options.get(JdbcCatalogOptions.PASSWORD), urlInfo, - options.get(JdbcCatalogOptions.SCHEMA)); + options.get(JdbcCatalogOptions.SCHEMA), + options.get(JdbcOptions.DECIMAL_TYPE_NARROWING)); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java index bb224c4624a..070ef670af1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java @@ -272,6 +272,15 @@ public static CatalogTable getCatalogTable( throws SQLException { TableSchema.Builder schemaBuilder = TableSchema.builder(); Map unsupported = new LinkedHashMap<>(); + String tableName = null; + String databaseName = null; + String schemaName = null; + try { + tableName = metadata.getTableName(1); + databaseName = metadata.getCatalogName(1); + schemaName = metadata.getSchemaName(1); + } catch (SQLException ignored) { + } for (int index = 1; index <= metadata.getColumnCount(); index++) { try { Column column = columnConverter.apply(metadata, index); @@ -289,8 +298,14 @@ public static CatalogTable getCatalogTable( throw CommonError.getCatalogTableWithUnsupportedType("UNKNOWN", sqlQuery, unsupported); } String catalogName = "jdbc_catalog"; + databaseName = StringUtils.isBlank(databaseName) ? null : databaseName; + schemaName = StringUtils.isBlank(schemaName) ? null : schemaName; + TablePath tablePath = + StringUtils.isBlank(tableName) + ? TablePath.DEFAULT + : TablePath.of(databaseName, schemaName, tableName); return CatalogTable.of( - TableIdentifier.of(catalogName, "default", "default", "default"), + TableIdentifier.of(catalogName, tablePath), schemaBuilder.build(), new HashMap<>(), new ArrayList<>(), @@ -307,11 +322,11 @@ public static CatalogTable getCatalogTable( } /** - * @deprecated instead by {@link #getCatalogTable(Connection, String, JdbcDialectTypeMapper)} * @param connection * @param sqlQuery * @return * @throws SQLException + * @deprecated instead by {@link #getCatalogTable(Connection, String, JdbcDialectTypeMapper)} */ @Deprecated public static CatalogTable getCatalogTable(Connection connection, String sqlQuery) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java index dc379bb38a5..053ab71a413 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java @@ -43,6 +43,8 @@ public class JdbcConnectionConfig implements Serializable { public String xaDataSourceClassName; + public boolean decimalTypeNarrowing = JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue(); + public int maxCommitAttempts = JdbcOptions.MAX_COMMIT_ATTEMPTS.defaultValue(); public int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); @@ -81,6 +83,8 @@ public static JdbcConnectionConfig of(ReadonlyConfig config) { config.getOptional(JdbcOptions.USER).ifPresent(builder::username); config.getOptional(JdbcOptions.PASSWORD).ifPresent(builder::password); config.getOptional(JdbcOptions.PROPERTIES).ifPresent(builder::properties); + config.getOptional(JdbcOptions.DECIMAL_TYPE_NARROWING) + .ifPresent(builder::decimalTypeNarrowing); return builder.build(); } @@ -108,6 +112,10 @@ public int getMaxRetries() { return maxRetries; } + public boolean isDecimalTypeNarrowing() { + return decimalTypeNarrowing; + } + public Optional getUsername() { return Optional.ofNullable(username); } @@ -153,6 +161,7 @@ public static final class Builder { private boolean autoCommit = JdbcOptions.AUTO_COMMIT.defaultValue(); private int batchSize = JdbcOptions.BATCH_SIZE.defaultValue(); private String xaDataSourceClassName; + private boolean decimalTypeNarrowing = JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue(); private int maxCommitAttempts = JdbcOptions.MAX_COMMIT_ATTEMPTS.defaultValue(); private int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); private Map properties; @@ -183,6 +192,11 @@ public Builder connectionCheckTimeoutSeconds(int connectionCheckTimeoutSeconds) return this; } + public Builder decimalTypeNarrowing(boolean decimalTypeNarrowing) { + this.decimalTypeNarrowing = decimalTypeNarrowing; + return this; + } + public Builder maxRetries(int maxRetries) { this.maxRetries = maxRetries; return this; @@ -267,6 +281,7 @@ public JdbcConnectionConfig build() { jdbcConnectionConfig.transactionTimeoutSec = this.transactionTimeoutSec; jdbcConnectionConfig.maxCommitAttempts = this.maxCommitAttempts; jdbcConnectionConfig.xaDataSourceClassName = this.xaDataSourceClassName; + jdbcConnectionConfig.decimalTypeNarrowing = this.decimalTypeNarrowing; jdbcConnectionConfig.useKerberos = this.useKerberos; jdbcConnectionConfig.kerberosPrincipal = this.kerberosPrincipal; jdbcConnectionConfig.kerberosKeytabPath = this.kerberosKeytabPath; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java index 7f0ec48f365..976650456b0 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java @@ -101,6 +101,13 @@ public interface JdbcOptions { .defaultValue(false) .withDescription("generate sql using the database table"); + Option DECIMAL_TYPE_NARROWING = + Options.key("decimal_type_narrowing") + .booleanType() + .defaultValue(true) + .withDescription( + "decimal type narrowing, if true, the decimal type will be narrowed to the int or long type if without loss of precision. Only support for Oracle at now."); + Option XA_DATA_SOURCE_CLASS_NAME = Options.key("xa_data_source_class_name") .stringType() diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java index 74c78013183..09cc92d70e8 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java @@ -42,6 +42,7 @@ public class JdbcSourceConfig implements Serializable { private double splitEvenDistributionFactorLowerBound; private int splitSampleShardingThreshold; private int splitInverseSamplingRate; + private boolean decimalTypeNarrowing; public static JdbcSourceConfig of(ReadonlyConfig config) { JdbcSourceConfig.Builder builder = JdbcSourceConfig.builder(); @@ -53,7 +54,7 @@ public static JdbcSourceConfig of(ReadonlyConfig config) { boolean isOldVersion = config.getOptional(JdbcOptions.QUERY).isPresent() && config.getOptional(JdbcOptions.PARTITION_COLUMN).isPresent(); - builder.useDynamicSplitter(isOldVersion ? false : true); + builder.useDynamicSplitter(!isOldVersion); builder.splitSize(config.get(JdbcSourceOptions.SPLIT_SIZE)); builder.splitEvenDistributionFactorUpperBound( @@ -64,6 +65,8 @@ public static JdbcSourceConfig of(ReadonlyConfig config) { config.get(JdbcSourceOptions.SPLIT_SAMPLE_SHARDING_THRESHOLD)); builder.splitInverseSamplingRate(config.get(JdbcSourceOptions.SPLIT_INVERSE_SAMPLING_RATE)); + builder.decimalTypeNarrowing(config.get(JdbcOptions.DECIMAL_TYPE_NARROWING)); + config.getOptional(JdbcSourceOptions.WHERE_CONDITION) .ifPresent( whereConditionClause -> { 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 d217a0b745a..f0af8513776 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 @@ -85,8 +85,6 @@ 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); } @@ -96,6 +94,9 @@ public static List of(ReadonlyConfig connectorConfig) { if (tableConfig.getPartitionNumber() == null) { tableConfig.setPartitionNumber(DEFAULT_PARTITION_NUMBER); } + tableConfig.setUseSelectCount( + connectorConfig.get(JdbcSourceOptions.USE_SELECT_COUNT)); + tableConfig.setSkipAnalyze(connectorConfig.get(JdbcSourceOptions.SKIP_ANALYZE)); }); if (tableList.size() > 1) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java index 97a37c0621b..8646d0d7e55 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java @@ -18,28 +18,27 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql; public enum MySqlVersion { - V_5_5, - V_5_6, - V_5_7, - V_8, - V_8_4; + V_5_5("5.5"), + V_5_6("5.6"), + V_5_7("5.7"), + V_8("8.0"), + V_8_1("8.1"), + V_8_2("8.2"), + V_8_3("8.3"), + V_8_4("8.4"); + + private final String versionPrefix; + + MySqlVersion(String versionPrefix) { + this.versionPrefix = versionPrefix; + } public static MySqlVersion parse(String version) { if (version != null) { - if (version.startsWith("5.5")) { - return V_5_5; - } - if (version.startsWith("5.6")) { - return V_5_6; - } - if (version.startsWith("5.7")) { - return V_5_7; - } - if (version.startsWith("8.0")) { - return V_8; - } - if (version.startsWith("8.4")) { - return V_8_4; + for (MySqlVersion mySqlVersion : values()) { + if (version.startsWith(mySqlVersion.versionPrefix)) { + return mySqlVersion; + } } } throw new UnsupportedOperationException("Unsupported MySQL version: " + version); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java index d359f3fef0d..023fa949cf1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java @@ -86,6 +86,16 @@ public class OracleTypeConverter implements TypeConverter { public static final long BYTES_4GB = (long) Math.pow(2, 32); public static final OracleTypeConverter INSTANCE = new OracleTypeConverter(); + private final boolean decimalTypeNarrowing; + + public OracleTypeConverter() { + this(true); + } + + public OracleTypeConverter(boolean decimalTypeNarrowing) { + this.decimalTypeNarrowing = decimalTypeNarrowing; + } + @Override public String identifier() { return DatabaseIdentifier.ORACLE; @@ -119,12 +129,14 @@ public Column convert(BasicTypeDefine typeDefine) { if (scale <= 0) { int newPrecision = (int) (precision - scale); - if (newPrecision == 1) { - builder.dataType(BasicType.BOOLEAN_TYPE); - } else if (newPrecision <= 9) { - builder.dataType(BasicType.INT_TYPE); - } else if (newPrecision <= 18) { - builder.dataType(BasicType.LONG_TYPE); + if (newPrecision <= 18 && decimalTypeNarrowing) { + if (newPrecision == 1) { + builder.dataType(BasicType.BOOLEAN_TYPE); + } else if (newPrecision <= 9) { + builder.dataType(BasicType.INT_TYPE); + } else { + builder.dataType(BasicType.LONG_TYPE); + } } else if (newPrecision < 38) { builder.dataType(new DecimalType(newPrecision, 0)); builder.columnLength((long) newPrecision); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java index ce5ef8af889..bbdd19af8eb 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.connectors.seatunnel.common.source.TypeDefineUtils; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; import lombok.extern.slf4j.Slf4j; @@ -31,9 +32,19 @@ @Slf4j public class OracleTypeMapper implements JdbcDialectTypeMapper { + private final boolean decimalTypeNarrowing; + + public OracleTypeMapper() { + this(JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue()); + } + + public OracleTypeMapper(boolean decimalTypeNarrowing) { + this.decimalTypeNarrowing = decimalTypeNarrowing; + } + @Override public Column mappingColumn(BasicTypeDefine typeDefine) { - return OracleTypeConverter.INSTANCE.convert(typeDefine); + return new OracleTypeConverter(decimalTypeNarrowing).convert(typeDefine); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java index 0e8814726c5..9f03f652f16 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java @@ -96,11 +96,17 @@ public void executeBatch() throws SQLException { @Override public void closeStatements() throws SQLException { - if (!buffer.isEmpty()) { - executeBatch(); + try { + if (!buffer.isEmpty()) { + executeBatch(); + } + } finally { + if (!buffer.isEmpty()) { + buffer.clear(); + } + upsertExecutor.closeStatements(); + deleteExecutor.closeStatements(); } - upsertExecutor.closeStatements(); - deleteExecutor.closeStatements(); } private boolean changeFlag(RowKind rowKind) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java index 86995240d79..0ebef85f52e 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java @@ -57,9 +57,15 @@ public void executeBatch() throws SQLException { @Override public void closeStatements() throws SQLException { - if (!buffer.isEmpty()) { - executeBatch(); + try { + if (!buffer.isEmpty()) { + executeBatch(); + } + } finally { + if (!buffer.isEmpty()) { + buffer.clear(); + } + statementExecutor.closeStatements(); } - statementExecutor.closeStatements(); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java index 8a4b2a579c7..9cf8b95863f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java @@ -112,13 +112,16 @@ public void executeBatch() throws SQLException { @Override public void closeStatements() throws SQLException { - if (!submitted) { - executeBatch(); - } - for (PreparedStatement statement : - Arrays.asList(existStatement, insertStatement, updateStatement)) { - if (statement != null) { - statement.close(); + try { + if (!submitted) { + executeBatch(); + } + } finally { + for (PreparedStatement statement : + Arrays.asList(existStatement, insertStatement, updateStatement)) { + if (statement != null) { + statement.close(); + } } } } 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 860131041a9..6eabba1edc1 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 @@ -34,6 +34,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceTableConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; @@ -139,6 +140,8 @@ public static Map getTables( .partitionNumber(tableConfig.getPartitionNumber()) .partitionStart(tableConfig.getPartitionStart()) .partitionEnd(tableConfig.getPartitionEnd()) + .useSelectCount(tableConfig.getUseSelectCount()) + .skipAnalyze(tableConfig.getSkipAnalyze()) .catalogTable(catalogTable) .build(); @@ -395,6 +398,8 @@ private static ReadonlyConfig extractCatalogConfig(JdbcConnectionConfig config) .ifPresent(val -> catalogConfig.put(JdbcCatalogOptions.PASSWORD.key(), val)); Optional.ofNullable(config.getCompatibleMode()) .ifPresent(val -> catalogConfig.put(JdbcCatalogOptions.COMPATIBLE_MODE.key(), val)); + catalogConfig.put( + JdbcOptions.DECIMAL_TYPE_NARROWING.key(), config.isDecimalTypeNarrowing()); return ReadonlyConfig.fromMap(catalogConfig); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java new file mode 100644 index 00000000000..cec02118928 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class MysqlVersionTest { + + @Test + public void testMysqlVersionParse() { + Assertions.assertEquals(MySqlVersion.V_5_5, MySqlVersion.parse("5.5.0")); + Assertions.assertEquals(MySqlVersion.V_5_5, MySqlVersion.parse("5.5.1")); + Assertions.assertEquals(MySqlVersion.V_5_5, MySqlVersion.parse("5.5.12")); + + Assertions.assertEquals(MySqlVersion.V_5_6, MySqlVersion.parse("5.6.0")); + Assertions.assertEquals(MySqlVersion.V_5_6, MySqlVersion.parse("5.6.1")); + Assertions.assertEquals(MySqlVersion.V_5_6, MySqlVersion.parse("5.6.12")); + + Assertions.assertEquals(MySqlVersion.V_5_7, MySqlVersion.parse("5.7.0")); + Assertions.assertEquals(MySqlVersion.V_5_7, MySqlVersion.parse("5.7.1")); + Assertions.assertEquals(MySqlVersion.V_5_7, MySqlVersion.parse("5.7.12")); + + Assertions.assertEquals(MySqlVersion.V_8, MySqlVersion.parse("8.0.0")); + Assertions.assertEquals(MySqlVersion.V_8, MySqlVersion.parse("8.0.1")); + Assertions.assertEquals(MySqlVersion.V_8, MySqlVersion.parse("8.0.12")); + + Assertions.assertEquals(MySqlVersion.V_8_1, MySqlVersion.parse("8.1.0")); + Assertions.assertEquals(MySqlVersion.V_8_1, MySqlVersion.parse("8.1.4")); + Assertions.assertEquals(MySqlVersion.V_8_1, MySqlVersion.parse("8.1.14")); + + Assertions.assertEquals(MySqlVersion.V_8_2, MySqlVersion.parse("8.2.0")); + Assertions.assertEquals(MySqlVersion.V_8_2, MySqlVersion.parse("8.2.4")); + Assertions.assertEquals(MySqlVersion.V_8_2, MySqlVersion.parse("8.2.14")); + + Assertions.assertEquals(MySqlVersion.V_8_3, MySqlVersion.parse("8.3.0")); + Assertions.assertEquals(MySqlVersion.V_8_3, MySqlVersion.parse("8.3.4")); + Assertions.assertEquals(MySqlVersion.V_8_3, MySqlVersion.parse("8.3.14")); + + Assertions.assertEquals(MySqlVersion.V_8_4, MySqlVersion.parse("8.4.0")); + Assertions.assertEquals(MySqlVersion.V_8_4, MySqlVersion.parse("8.4.4")); + Assertions.assertEquals(MySqlVersion.V_8_4, MySqlVersion.parse("8.4.14")); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java index 26238bad303..d4a8defddab 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java @@ -36,12 +36,14 @@ public class OracleTypeConverterTest { + private static final OracleTypeConverter INSTANCE = new OracleTypeConverter(); + @Test public void testConvertUnsupported() { BasicTypeDefine typeDefine = BasicTypeDefine.builder().name("test").columnType("aaa").dataType("aaa").build(); try { - OracleTypeConverter.INSTANCE.convert(typeDefine); + INSTANCE.convert(typeDefine); Assertions.fail(); } catch (SeaTunnelRuntimeException e) { // ignore @@ -50,6 +52,113 @@ public void testConvertUnsupported() { } } + @Test + public void testConvertNumberWithoutDecimalTypeNarrowing() { + OracleTypeConverter converter = new OracleTypeConverter(false); + + BasicTypeDefine typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number") + .dataType("number") + .build(); + Column column = converter.convert(typeDefine); + + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number(38,127)") + .dataType("number") + .precision(38L) + .scale(127) + .build(); + column = converter.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number") + .dataType("number") + .scale(0) + .build(); + column = converter.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number(1,0)") + .dataType("number") + .precision(1L) + .scale(0) + .build(); + column = converter.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(1, 0), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number(8,0)") + .dataType("number") + .precision(8L) + .scale(0) + .build(); + column = converter.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(8, 0), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number(18,0)") + .dataType("number") + .precision(18L) + .scale(0) + .build(); + column = converter.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(18, 0), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number(38,0)") + .dataType("number") + .precision(38L) + .scale(0) + .build(); + column = converter.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + typeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number(39,0)") + .dataType("number") + .precision(39L) + .scale(0) + .build(); + column = converter.convert(typeDefine); + Assertions.assertEquals(typeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); + Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + } + @Test public void testConvertInteger() { BasicTypeDefine typeDefine = @@ -58,10 +167,24 @@ public void testConvertInteger() { .columnType("integer") .dataType("integer") .build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); + + // generated by int/smallint type in oracle create table sql + BasicTypeDefine numberTypeDefine = + BasicTypeDefine.builder() + .name("test") + .columnType("number") + .dataType("number") + .precision(null) + .scale(0) + .build(); + column = INSTANCE.convert(numberTypeDefine); + Assertions.assertEquals(numberTypeDefine.getName(), column.getName()); + Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); + Assertions.assertEquals(numberTypeDefine.getColumnType(), column.getSourceType()); } @Test @@ -72,7 +195,7 @@ public void testConvertNumber() { .columnType("number") .dataType("number") .build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); @@ -86,7 +209,7 @@ public void testConvertNumber() { .precision(38L) .scale(127) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -98,7 +221,7 @@ public void testConvertNumber() { .dataType("number") .scale(0) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -111,7 +234,7 @@ public void testConvertNumber() { .precision(1L) .scale(0) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.BOOLEAN_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -124,7 +247,7 @@ public void testConvertNumber() { .precision(8L) .scale(0) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.INT_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -137,7 +260,7 @@ public void testConvertNumber() { .precision(18L) .scale(0) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.LONG_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -150,7 +273,7 @@ public void testConvertNumber() { .precision(38L) .scale(0) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -163,7 +286,7 @@ public void testConvertNumber() { .precision(39L) .scale(0) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -177,7 +300,7 @@ public void testConvertFloat() { .columnType("float") .dataType("float") .build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); @@ -189,7 +312,7 @@ public void testConvertFloat() { .columnType("binary_float") .dataType("binary_float") .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.FLOAT_TYPE, column.getDataType()); @@ -197,7 +320,7 @@ public void testConvertFloat() { typeDefine = BasicTypeDefine.builder().name("test").columnType("real").dataType("real").build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.FLOAT_TYPE, column.getDataType()); @@ -212,7 +335,7 @@ public void testConvertDouble() { .columnType("binary_double") .dataType("binary_double") .build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.DOUBLE_TYPE, column.getDataType()); @@ -228,7 +351,7 @@ public void testConvertChar() { .dataType("char") .length(1L) .build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -242,7 +365,7 @@ public void testConvertChar() { .dataType("nchar") .length(1L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -256,7 +379,7 @@ public void testConvertChar() { .dataType("varchar") .length(1L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -270,7 +393,7 @@ public void testConvertChar() { .dataType("varchar2") .length(1L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -284,7 +407,7 @@ public void testConvertChar() { .dataType("nvarchar2") .length(1L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -297,7 +420,7 @@ public void testConvertChar() { .columnType("rowid") .dataType("rowid") .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -311,7 +434,7 @@ public void testConvertChar() { .dataType("xmltype") .length(1L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -325,7 +448,7 @@ public void testConvertChar() { .dataType("sys.xmltype") .length(1L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -339,7 +462,7 @@ public void testConvertChar() { .dataType("long") .length(1L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -348,7 +471,7 @@ public void testConvertChar() { typeDefine = BasicTypeDefine.builder().name("test").columnType("clob").dataType("clob").build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -361,7 +484,7 @@ public void testConvertChar() { .columnType("nclob") .dataType("nclob") .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -373,7 +496,7 @@ public void testConvertChar() { public void testConvertBytes() { BasicTypeDefine typeDefine = BasicTypeDefine.builder().name("test").columnType("blob").dataType("blob").build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -382,7 +505,7 @@ public void testConvertBytes() { typeDefine = BasicTypeDefine.builder().name("test").columnType("raw").dataType("raw").build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -396,7 +519,7 @@ public void testConvertBytes() { .dataType("raw") .length(10L) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -409,7 +532,7 @@ public void testConvertBytes() { .columnType("long raw") .dataType("long raw") .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -421,7 +544,7 @@ public void testConvertBytes() { public void testConvertDatetime() { BasicTypeDefine typeDefine = BasicTypeDefine.builder().name("test").columnType("date").dataType("date").build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -435,7 +558,7 @@ public void testConvertDatetime() { .dataType("timestamp") .scale(6) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -449,7 +572,7 @@ public void testConvertDatetime() { .dataType("timestamp with time zone") .scale(6) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -463,7 +586,7 @@ public void testConvertDatetime() { .dataType("timestamp with local time zone") .scale(6) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -482,7 +605,7 @@ public void testReconvertUnsupported() { null, null); try { - OracleTypeConverter.INSTANCE.reconvert(column); + INSTANCE.reconvert(column); Assertions.fail(); } catch (SeaTunnelRuntimeException e) { // ignore @@ -496,7 +619,7 @@ public void testReconvertBoolean() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.BOOLEAN_TYPE).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("%s(%s)", OracleTypeConverter.ORACLE_NUMBER, 1), @@ -509,7 +632,7 @@ public void testReconvertBoolean() { public void testReconvertByte() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.BYTE_TYPE).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -520,7 +643,7 @@ public void testReconvertShort() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.SHORT_TYPE).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -530,7 +653,7 @@ public void testReconvertShort() { public void testReconvertInt() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.INT_TYPE).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -540,7 +663,7 @@ public void testReconvertInt() { public void testReconvertLong() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.LONG_TYPE).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -551,7 +674,7 @@ public void testReconvertFloat() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.FLOAT_TYPE).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( OracleTypeConverter.ORACLE_BINARY_FLOAT, typeDefine.getColumnType()); @@ -563,7 +686,7 @@ public void testReconvertDouble() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.DOUBLE_TYPE).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( OracleTypeConverter.ORACLE_BINARY_DOUBLE, typeDefine.getColumnType()); @@ -575,7 +698,7 @@ public void testReconvertDecimal() { Column column = PhysicalColumn.builder().name("test").dataType(new DecimalType(0, 0)).build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format( @@ -588,7 +711,7 @@ public void testReconvertDecimal() { column = PhysicalColumn.builder().name("test").dataType(new DecimalType(10, 2)).build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("%s(%s,%s)", OracleTypeConverter.ORACLE_NUMBER, 10, 2), @@ -605,7 +728,7 @@ public void testReconvertBytes() { .columnLength(null) .build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getDataType()); @@ -617,7 +740,7 @@ public void testReconvertBytes() { .columnLength(2000L) .build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("%s(%s)", OracleTypeConverter.ORACLE_RAW, column.getColumnLength()), @@ -631,7 +754,7 @@ public void testReconvertBytes() { .columnLength(BYTES_2GB) .build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getDataType()); @@ -643,7 +766,7 @@ public void testReconvertBytes() { .columnLength(BYTES_2GB + 1) .build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getDataType()); @@ -658,7 +781,7 @@ public void testReconvertString() { .columnLength(null) .build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals("VARCHAR2(4000)", typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_VARCHAR2, typeDefine.getDataType()); @@ -670,7 +793,7 @@ public void testReconvertString() { .columnLength(2000L) .build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format( @@ -685,7 +808,7 @@ public void testReconvertString() { .columnLength(4000L) .build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format( @@ -700,7 +823,7 @@ public void testReconvertString() { .columnLength(40001L) .build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_CLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_CLOB, typeDefine.getDataType()); @@ -714,7 +837,7 @@ public void testReconvertDate() { .dataType(LocalTimeType.LOCAL_DATE_TYPE) .build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_DATE, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_DATE, typeDefine.getDataType()); @@ -728,7 +851,7 @@ public void testReconvertDatetime() { .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) .build(); - BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( OracleTypeConverter.ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE, @@ -744,7 +867,7 @@ public void testReconvertDatetime() { .scale(3) .build(); - typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); + typeDefine = INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("TIMESTAMP(%s) WITH LOCAL TIME ZONE", column.getScale()), @@ -765,7 +888,7 @@ public void testNumberWithNegativeScale() { .precision(38L) .scale(-1) .build(); - Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); + Column column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -778,7 +901,7 @@ public void testNumberWithNegativeScale() { .precision(5L) .scale(-2) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.INT_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -791,7 +914,7 @@ public void testNumberWithNegativeScale() { .precision(9L) .scale(-2) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.LONG_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -804,7 +927,7 @@ public void testNumberWithNegativeScale() { .precision(14L) .scale(-11) .build(); - column = OracleTypeConverter.INSTANCE.convert(typeDefine); + column = INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(25, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); diff --git a/seatunnel-connectors-v2/connector-kafka/pom.xml b/seatunnel-connectors-v2/connector-kafka/pom.xml index 3251de5a8e7..9ffff7dbc0c 100644 --- a/seatunnel-connectors-v2/connector-kafka/pom.xml +++ b/seatunnel-connectors-v2/connector-kafka/pom.xml @@ -77,6 +77,11 @@ seatunnel-format-avro ${project.version} + + org.apache.seatunnel + seatunnel-format-protobuf + ${project.version} + diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index 6d6f43dc4c2..a907c9bc212 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -177,10 +177,24 @@ public class Config { .defaultValue(KafkaSemantics.NON) .withDescription( "Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON."); + public static final Option>> TABLE_LIST = Options.key("table_list") .type(new TypeReference>>() {}) .noDefaultValue() .withDescription( "Topic list config. You can configure only one `table_list` or one `topic` at the same time"); + + public static final Option PROTOBUF_SCHEMA = + Options.key("protobuf_schema") + .stringType() + .noDefaultValue() + .withDescription( + "Data serialization method protobuf metadata, used to parse protobuf data."); + + public static final Option PROTOBUF_MESSAGE_NAME = + Options.key("protobuf_message_name") + .stringType() + .noDefaultValue() + .withDescription("Parsing entity class names from Protobuf data."); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java index f02cebcbe35..a877c76c37d 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java @@ -26,5 +26,6 @@ public enum MessageFormat { COMPATIBLE_KAFKA_CONNECT_JSON, OGG_JSON, AVRO, - MAXWELL_JSON + MAXWELL_JSON, + PROTOBUF } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java index d4a77e74b93..2fb251571f1 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.serialize; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.SerializationSchema; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -33,6 +34,7 @@ import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import org.apache.seatunnel.format.json.maxwell.MaxWellJsonSerializationSchema; import org.apache.seatunnel.format.json.ogg.OggJsonSerializationSchema; +import org.apache.seatunnel.format.protobuf.ProtobufSerializationSchema; import org.apache.seatunnel.format.text.TextSerializationSchema; import org.apache.kafka.clients.producer.ProducerRecord; @@ -46,6 +48,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_MESSAGE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_SCHEMA; + @RequiredArgsConstructor public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer { private final Function topicExtractor; @@ -67,13 +72,17 @@ public ProducerRecord serializeRow(SeaTunnelRow row) { } public static DefaultSeaTunnelRowSerializer create( - String topic, SeaTunnelRowType rowType, MessageFormat format, String delimiter) { + String topic, + SeaTunnelRowType rowType, + MessageFormat format, + String delimiter, + ReadonlyConfig pluginConfig) { return new DefaultSeaTunnelRowSerializer( topicExtractor(topic, rowType, format), partitionExtractor(null), timestampExtractor(), - keyExtractor(null, rowType, format, delimiter), - valueExtractor(rowType, format, delimiter), + keyExtractor(null, rowType, format, delimiter, pluginConfig), + valueExtractor(rowType, format, delimiter, pluginConfig), headersExtractor()); } @@ -82,13 +91,14 @@ public static DefaultSeaTunnelRowSerializer create( Integer partition, SeaTunnelRowType rowType, MessageFormat format, - String delimiter) { + String delimiter, + ReadonlyConfig pluginConfig) { return new DefaultSeaTunnelRowSerializer( topicExtractor(topic, rowType, format), partitionExtractor(partition), timestampExtractor(), - keyExtractor(null, rowType, format, delimiter), - valueExtractor(rowType, format, delimiter), + keyExtractor(null, rowType, format, delimiter, pluginConfig), + valueExtractor(rowType, format, delimiter, pluginConfig), headersExtractor()); } @@ -97,13 +107,14 @@ public static DefaultSeaTunnelRowSerializer create( List keyFields, SeaTunnelRowType rowType, MessageFormat format, - String delimiter) { + String delimiter, + ReadonlyConfig pluginConfig) { return new DefaultSeaTunnelRowSerializer( topicExtractor(topic, rowType, format), partitionExtractor(null), timestampExtractor(), - keyExtractor(keyFields, rowType, format, delimiter), - valueExtractor(rowType, format, delimiter), + keyExtractor(keyFields, rowType, format, delimiter, pluginConfig), + valueExtractor(rowType, format, delimiter, pluginConfig), headersExtractor()); } @@ -157,7 +168,8 @@ private static Function keyExtractor( List keyFields, SeaTunnelRowType rowType, MessageFormat format, - String delimiter) { + String delimiter, + ReadonlyConfig pluginConfig) { if (MessageFormat.COMPATIBLE_DEBEZIUM_JSON.equals(format)) { CompatibleDebeziumJsonSerializationSchema serializationSchema = new CompatibleDebeziumJsonSerializationSchema(rowType, true); @@ -172,14 +184,17 @@ private static Function keyExtractor( Function keyRowExtractor = createKeyRowExtractor(keyType, rowType); SerializationSchema serializationSchema = - createSerializationSchema(keyType, format, delimiter, true); + createSerializationSchema(keyType, format, delimiter, true, pluginConfig); return row -> serializationSchema.serialize(keyRowExtractor.apply(row)); } private static Function valueExtractor( - SeaTunnelRowType rowType, MessageFormat format, String delimiter) { + SeaTunnelRowType rowType, + MessageFormat format, + String delimiter, + ReadonlyConfig pluginConfig) { SerializationSchema serializationSchema = - createSerializationSchema(rowType, format, delimiter, false); + createSerializationSchema(rowType, format, delimiter, false, pluginConfig); return row -> serializationSchema.serialize(row); } @@ -212,7 +227,11 @@ private static Function createKeyRowExtractor( } private static SerializationSchema createSerializationSchema( - SeaTunnelRowType rowType, MessageFormat format, String delimiter, boolean isKey) { + SeaTunnelRowType rowType, + MessageFormat format, + String delimiter, + boolean isKey, + ReadonlyConfig pluginConfig) { switch (format) { case JSON: return new JsonSerializationSchema(rowType); @@ -233,6 +252,11 @@ private static SerializationSchema createSerializationSchema( return new CompatibleDebeziumJsonSerializationSchema(rowType, isKey); case AVRO: return new AvroSerializationSchema(rowType); + case PROTOBUF: + String protobufMessageName = pluginConfig.get(PROTOBUF_MESSAGE_NAME); + String protobufSchema = pluginConfig.get(PROTOBUF_SCHEMA); + return new ProtobufSerializationSchema( + rowType, protobufMessageName, protobufSchema); default: throw new SeaTunnelJsonFormatException( CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java index 44676f29d4c..6639a34a0bd 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java @@ -188,15 +188,21 @@ private SeaTunnelRowSerializer getSerializer( getPartitionKeyFields(pluginConfig, seaTunnelRowType), seaTunnelRowType, messageFormat, - delimiter); + delimiter, + pluginConfig); } if (pluginConfig.get(PARTITION) != null) { return DefaultSeaTunnelRowSerializer.create( - topic, pluginConfig.get(PARTITION), seaTunnelRowType, messageFormat, delimiter); + topic, + pluginConfig.get(PARTITION), + seaTunnelRowType, + messageFormat, + delimiter, + pluginConfig); } // By default, all partitions are sent randomly return DefaultSeaTunnelRowSerializer.create( - topic, Arrays.asList(), seaTunnelRowType, messageFormat, delimiter); + topic, Arrays.asList(), seaTunnelRowType, messageFormat, delimiter, pluginConfig); } private KafkaSemantics getKafkaSemantics(ReadonlyConfig pluginConfig) { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java index 8ce9dbb870a..c129fbdc93c 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java @@ -37,7 +37,6 @@ public class ConsumerMetadata implements Serializable { private String topic; private boolean isPattern = false; private Properties properties; - private String consumerGroup; private StartMode startMode = StartMode.GROUP_OFFSETS; private Map specificStartOffsets; private Long startOffsetsTimestamp; diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java deleted file mode 100644 index 99b6baeaf16..00000000000 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.kafka.source; - -import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorException; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; - -import java.util.Properties; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -public class KafkaConsumerThread implements Runnable { - - private final KafkaConsumer consumer; - private static final String CLIENT_ID_PREFIX = "seatunnel"; - private final ConsumerMetadata metadata; - - private final LinkedBlockingQueue>> tasks; - - public KafkaConsumerThread(KafkaSourceConfig kafkaSourceConfig, ConsumerMetadata metadata) { - this.metadata = metadata; - this.tasks = new LinkedBlockingQueue<>(); - this.consumer = - initConsumer( - kafkaSourceConfig.getBootstrap(), - metadata.getConsumerGroup(), - kafkaSourceConfig.getProperties(), - !kafkaSourceConfig.isCommitOnCheckpoint()); - } - - @Override - public void run() { - try { - while (!Thread.currentThread().isInterrupted()) { - try { - Consumer> task = tasks.poll(1, TimeUnit.SECONDS); - if (task != null) { - task.accept(consumer); - } - } catch (Exception e) { - throw new KafkaConnectorException( - KafkaConnectorErrorCode.CONSUME_THREAD_RUN_ERROR, e); - } - } - } finally { - try { - if (consumer != null) { - consumer.close(); - } - } catch (Throwable t) { - throw new KafkaConnectorException(KafkaConnectorErrorCode.CONSUMER_CLOSE_FAILED, t); - } - } - } - - public LinkedBlockingQueue>> getTasks() { - return tasks; - } - - private KafkaConsumer initConsumer( - String bootstrapServer, - String consumerGroup, - Properties properties, - boolean autoCommit) { - Properties props = new Properties(); - properties.forEach( - (key, value) -> props.setProperty(String.valueOf(key), String.valueOf(value))); - props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, consumerGroup); - props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer); - if (this.metadata.getProperties().get("client.id") == null) { - props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, - CLIENT_ID_PREFIX + "-consumer-" + this.hashCode()); - } else { - props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, - this.metadata.getProperties().get("client.id").toString()); - } - props.setProperty( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - ByteArrayDeserializer.class.getName()); - props.setProperty( - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - ByteArrayDeserializer.class.getName()); - props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, String.valueOf(autoCommit)); - - // Disable auto create topics feature - props.setProperty(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); - return new KafkaConsumer<>(props); - } -} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java new file mode 100644 index 00000000000..8bca82999c7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java @@ -0,0 +1,458 @@ +/* + * 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.kafka.source; + +import org.apache.seatunnel.shade.com.google.common.base.Preconditions; + +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.common.utils.TemporaryClassLoaderContext; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitReader; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitsAddition; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitsChange; +import org.apache.seatunnel.connectors.seatunnel.kafka.config.StartMode; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.StringJoiner; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class KafkaPartitionSplitReader + implements SplitReader, KafkaSourceSplit> { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaPartitionSplitReader.class); + + private static final long POLL_TIMEOUT = 10000L; + private static final String CLIENT_ID_PREFIX = "seatunnel"; + private final KafkaSourceConfig kafkaSourceConfig; + + private final KafkaConsumer consumer; + + private final Map stoppingOffsets; + + private final String groupId; + + private final Set emptySplits = new HashSet<>(); + + public KafkaPartitionSplitReader( + KafkaSourceConfig kafkaSourceConfig, SourceReader.Context context) { + this.kafkaSourceConfig = kafkaSourceConfig; + this.consumer = initConsumer(kafkaSourceConfig, context.getIndexOfSubtask()); + this.stoppingOffsets = new HashMap<>(); + this.groupId = + kafkaSourceConfig.getProperties().getProperty(ConsumerConfig.GROUP_ID_CONFIG); + } + + @Override + public RecordsWithSplitIds> fetch() throws IOException { + ConsumerRecords consumerRecords; + try { + consumerRecords = consumer.poll(Duration.ofMillis(POLL_TIMEOUT)); + } catch (WakeupException | IllegalStateException e) { + // IllegalStateException will be thrown if the consumer is not assigned any partitions. + // This happens if all assigned partitions are invalid or empty (starting offset >= + // stopping offset). We just mark empty partitions as finished and return an empty + // record container, and this consumer will be closed by SplitFetcherManager. + KafkaPartitionSplitRecords recordsBySplits = + new KafkaPartitionSplitRecords(ConsumerRecords.empty()); + markEmptySplitsAsFinished(recordsBySplits); + return recordsBySplits; + } + KafkaPartitionSplitRecords recordsBySplits = + new KafkaPartitionSplitRecords(consumerRecords); + List finishedPartitions = new ArrayList<>(); + for (TopicPartition tp : consumerRecords.partitions()) { + long stoppingOffset = getStoppingOffset(tp); + final List> recordsFromPartition = + consumerRecords.records(tp); + + if (recordsFromPartition.size() > 0) { + final ConsumerRecord lastRecord = + recordsFromPartition.get(recordsFromPartition.size() - 1); + + // After processing a record with offset of "stoppingOffset - 1", the split reader + // should not continue fetching because the record with stoppingOffset may not + // exist. Keep polling will just block forever. + if (lastRecord.offset() >= stoppingOffset - 1) { + recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); + finishSplitAtRecord( + tp, + stoppingOffset, + lastRecord.offset(), + finishedPartitions, + recordsBySplits); + } + } + } + + markEmptySplitsAsFinished(recordsBySplits); + + if (!finishedPartitions.isEmpty()) { + unassignPartitions(finishedPartitions); + } + + return recordsBySplits; + } + + private void finishSplitAtRecord( + TopicPartition tp, + long stoppingOffset, + long currentOffset, + List finishedPartitions, + KafkaPartitionSplitRecords recordsBySplits) { + LOG.debug( + "{} has reached stopping offset {}, current offset is {}", + tp, + stoppingOffset, + currentOffset); + finishedPartitions.add(tp); + recordsBySplits.addFinishedSplit(tp.toString()); + } + + private void markEmptySplitsAsFinished(KafkaPartitionSplitRecords recordsBySplits) { + // Some splits are discovered as empty when handling split additions. These splits should be + // added to finished splits to clean up states in split fetcher and source reader. + if (!emptySplits.isEmpty()) { + recordsBySplits.finishedSplits.addAll(emptySplits); + emptySplits.clear(); + } + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChange) { + // Get all the partition assignments and stopping offsets. + if (!(splitsChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format( + "The SplitChange type of %s is not supported.", + splitsChange.getClass())); + } + + // Assignment. + List newPartitionAssignments = new ArrayList<>(); + // Starting offsets. + Map partitionsStartingFromSpecifiedOffsets = new HashMap<>(); + List partitionsStartingFromEarliest = new ArrayList<>(); + List partitionsStartingFromLatest = new ArrayList<>(); + // Stopping offsets. + List partitionsStoppingAtLatest = new ArrayList<>(); + + // Parse the starting and stopping offsets. + splitsChange + .splits() + .forEach( + s -> { + newPartitionAssignments.add(s.getTopicPartition()); + parseStartingOffsets( + s, + partitionsStartingFromEarliest, + partitionsStartingFromLatest, + partitionsStartingFromSpecifiedOffsets); + parseStoppingOffsets(s, partitionsStoppingAtLatest); + }); + + // Assign new partitions. + newPartitionAssignments.addAll(consumer.assignment()); + consumer.assign(newPartitionAssignments); + + // Seek on the newly assigned partitions to their stating offsets. + seekToStartingOffsets( + partitionsStartingFromEarliest, + partitionsStartingFromLatest, + partitionsStartingFromSpecifiedOffsets); + // Setup the stopping offsets. + acquireAndSetStoppingOffsets(partitionsStoppingAtLatest); + + // After acquiring the starting and stopping offsets, remove the empty splits if necessary. + removeEmptySplits(); + + maybeLogSplitChangesHandlingResult(splitsChange); + } + + private void maybeLogSplitChangesHandlingResult(SplitsChange splitsChange) { + if (LOG.isDebugEnabled()) { + StringJoiner splitsInfo = new StringJoiner(","); + Set assginment = consumer.assignment(); + for (KafkaSourceSplit split : splitsChange.splits()) { + if (!assginment.contains(split.getTopicPartition())) { + continue; + } + + long startingOffset = + retryOnWakeup( + () -> consumer.position(split.getTopicPartition()), + "logging starting position"); + long stoppingOffset = getStoppingOffset(split.getTopicPartition()); + splitsInfo.add( + String.format( + "[%s, start:%d, stop: %d]", + split.getTopicPartition(), startingOffset, stoppingOffset)); + } + LOG.debug("SplitsChange handling result: {}", splitsInfo); + } + } + + private void removeEmptySplits() { + List emptyPartitions = new ArrayList<>(); + // If none of the partitions have any records, + for (TopicPartition tp : consumer.assignment()) { + if (retryOnWakeup( + () -> consumer.position(tp), + "getting starting offset to check if split is empty") + >= getStoppingOffset(tp)) { + emptyPartitions.add(tp); + } + } + if (!emptyPartitions.isEmpty()) { + LOG.debug( + "These assigning splits are empty and will be marked as finished in later fetch: {}", + emptyPartitions); + // Add empty partitions to empty split set for later cleanup in fetch() + emptySplits.addAll( + emptyPartitions.stream() + .map(TopicPartition::toString) + .collect(Collectors.toSet())); + // Un-assign partitions from Kafka consumer + unassignPartitions(emptyPartitions); + } + } + + private void unassignPartitions(Collection partitionsToUnassign) { + Collection newAssignment = new HashSet<>(consumer.assignment()); + newAssignment.removeAll(partitionsToUnassign); + consumer.assign(newAssignment); + } + + private void acquireAndSetStoppingOffsets(List partitionsStoppingAtLatest) { + Map endOffset = consumer.endOffsets(partitionsStoppingAtLatest); + stoppingOffsets.putAll(endOffset); + } + + private void seekToStartingOffsets( + List partitionsStartingFromEarliest, + List partitionsStartingFromLatest, + Map partitionsStartingFromSpecifiedOffsets) { + + if (!partitionsStartingFromEarliest.isEmpty()) { + LOG.trace("Seeking starting offsets to beginning: {}", partitionsStartingFromEarliest); + consumer.seekToBeginning(partitionsStartingFromEarliest); + } + + if (!partitionsStartingFromLatest.isEmpty()) { + LOG.trace("Seeking starting offsets to end: {}", partitionsStartingFromLatest); + consumer.seekToEnd(partitionsStartingFromLatest); + } + + if (!partitionsStartingFromSpecifiedOffsets.isEmpty()) { + LOG.trace( + "Seeking starting offsets to specified offsets: {}", + partitionsStartingFromSpecifiedOffsets); + partitionsStartingFromSpecifiedOffsets.forEach(consumer::seek); + } + } + + private void parseStoppingOffsets( + KafkaSourceSplit split, List partitionsStoppingAtLatest) { + TopicPartition tp = split.getTopicPartition(); + if (split.getEndOffset() >= 0) { + stoppingOffsets.put(tp, split.getEndOffset()); + } else { + partitionsStoppingAtLatest.add(tp); + } + } + + private long getStoppingOffset(TopicPartition tp) { + return stoppingOffsets.getOrDefault(tp, Long.MAX_VALUE); + } + + private void parseStartingOffsets( + KafkaSourceSplit split, + List partitionsStartingFromEarliest, + List partitionsStartingFromLatest, + Map partitionsStartingFromSpecifiedOffsets) { + TopicPartition tp = split.getTopicPartition(); + // Parse starting offsets. + ConsumerMetadata metadata = kafkaSourceConfig.getMapMetadata().get(split.getTablePath()); + if (metadata.getStartMode() == StartMode.EARLIEST) { + partitionsStartingFromEarliest.add(tp); + } else if (metadata.getStartMode() == StartMode.LATEST) { + partitionsStartingFromLatest.add(tp); + } else if (metadata.getStartMode() == StartMode.GROUP_OFFSETS) { + // Do nothing here, the consumer will first try to get the committed offsets of + // these partitions by default. + } else { + partitionsStartingFromSpecifiedOffsets.put(tp, split.getStartOffset()); + } + } + + @Override + public void wakeUp() { + consumer.wakeup(); + } + + @Override + public void close() throws Exception { + consumer.close(); + } + + public void notifyCheckpointComplete( + Map offsetsToCommit, + OffsetCommitCallback offsetCommitCallback) { + consumer.commitAsync(offsetsToCommit, offsetCommitCallback); + } + + private KafkaConsumer initConsumer( + KafkaSourceConfig kafkaSourceConfig, int subtaskId) { + + try (TemporaryClassLoaderContext ignored = + TemporaryClassLoaderContext.of(kafkaSourceConfig.getClass().getClassLoader())) { + Properties props = new Properties(); + kafkaSourceConfig + .getProperties() + .forEach( + (key, value) -> + props.setProperty(String.valueOf(key), String.valueOf(value))); + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, kafkaSourceConfig.getConsumerGroup()); + props.setProperty( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSourceConfig.getBootstrap()); + if (this.kafkaSourceConfig.getProperties().get("client.id") == null) { + props.setProperty( + ConsumerConfig.CLIENT_ID_CONFIG, + CLIENT_ID_PREFIX + "-consumer-" + subtaskId); + } else { + props.setProperty( + ConsumerConfig.CLIENT_ID_CONFIG, + this.kafkaSourceConfig.getProperties().get("client.id").toString() + + "-" + + subtaskId); + } + props.setProperty( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + props.setProperty( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + props.setProperty( + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + String.valueOf(kafkaSourceConfig.isCommitOnCheckpoint())); + + // Disable auto create topics feature + props.setProperty(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); + return new KafkaConsumer<>(props); + } + } + + private V retryOnWakeup(Supplier consumerCall, String description) { + try { + return consumerCall.get(); + } catch (WakeupException we) { + LOG.info( + "Caught WakeupException while executing Kafka consumer call for {}. Will retry the consumer call.", + description); + return consumerCall.get(); + } + } + + private static class KafkaPartitionSplitRecords + implements RecordsWithSplitIds> { + + private final Set finishedSplits = new HashSet<>(); + private final Map stoppingOffsets = new HashMap<>(); + private final ConsumerRecords consumerRecords; + private final Iterator splitIterator; + private Iterator> recordIterator; + private TopicPartition currentTopicPartition; + private Long currentSplitStoppingOffset; + + private KafkaPartitionSplitRecords(ConsumerRecords consumerRecords) { + this.consumerRecords = consumerRecords; + this.splitIterator = consumerRecords.partitions().iterator(); + } + + private void setPartitionStoppingOffset( + TopicPartition topicPartition, long stoppingOffset) { + stoppingOffsets.put(topicPartition, stoppingOffset); + } + + private void addFinishedSplit(String splitId) { + finishedSplits.add(splitId); + } + + @Nullable @Override + public String nextSplit() { + if (splitIterator.hasNext()) { + currentTopicPartition = splitIterator.next(); + recordIterator = consumerRecords.records(currentTopicPartition).iterator(); + currentSplitStoppingOffset = + stoppingOffsets.getOrDefault(currentTopicPartition, Long.MAX_VALUE); + return currentTopicPartition.toString(); + } else { + currentTopicPartition = null; + recordIterator = null; + currentSplitStoppingOffset = null; + return null; + } + } + + @Nullable @Override + public ConsumerRecord nextRecordFromSplit() { + Preconditions.checkNotNull( + currentTopicPartition, + "Make sure nextSplit() did not return null before " + + "iterate over the records split."); + if (recordIterator.hasNext()) { + final ConsumerRecord record = recordIterator.next(); + // Only emit records before stopping offset + if (record.offset() < currentSplitStoppingOffset) { + return record; + } + } + return null; + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + } +} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java new file mode 100644 index 00000000000..6593137aff7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.kafka.source; + +import org.apache.seatunnel.api.serialization.DeserializationSchema; +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.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordEmitter; +import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; +import org.apache.seatunnel.format.compatible.kafka.connect.json.CompatibleKafkaConnectDeserializationSchema; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; + +public class KafkaRecordEmitter + implements RecordEmitter< + ConsumerRecord, SeaTunnelRow, KafkaSourceSplitState> { + + private static final Logger logger = LoggerFactory.getLogger(KafkaRecordEmitter.class); + private final Map mapMetadata; + private final OutputCollector outputCollector; + private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; + + public KafkaRecordEmitter( + Map mapMetadata, + MessageFormatErrorHandleWay messageFormatErrorHandleWay) { + this.mapMetadata = mapMetadata; + this.messageFormatErrorHandleWay = messageFormatErrorHandleWay; + this.outputCollector = new OutputCollector<>(); + } + + @Override + public void emitRecord( + ConsumerRecord consumerRecord, + Collector collector, + KafkaSourceSplitState splitState) + throws Exception { + outputCollector.output = collector; + // todo there is an additional loss in this place for non-multi-table scenarios + DeserializationSchema deserializationSchema = + mapMetadata.get(splitState.getTablePath()).getDeserializationSchema(); + try { + if (deserializationSchema instanceof CompatibleKafkaConnectDeserializationSchema) { + ((CompatibleKafkaConnectDeserializationSchema) deserializationSchema) + .deserialize(consumerRecord, outputCollector); + } else { + deserializationSchema.deserialize(consumerRecord.value(), outputCollector); + } + // consumerRecord.offset + 1 is the offset commit to Kafka and also the start offset + // for the next run + splitState.setCurrentOffset(consumerRecord.offset() + 1); + } catch (IOException e) { + if (this.messageFormatErrorHandleWay == MessageFormatErrorHandleWay.SKIP) { + logger.warn( + "Deserialize message failed, skip this message, message: {}", + new String(consumerRecord.value())); + } + throw e; + } + } + + private static class OutputCollector implements Collector { + private Collector output; + + @Override + public void collect(T record) { + output.collect(record); + } + + @Override + public void collect(SchemaChangeEvent event) { + output.collect(event); + } + + @Override + public void markSchemaChangeBeforeCheckpoint() { + output.markSchemaChangeBeforeCheckpoint(); + } + + @Override + public void markSchemaChangeAfterCheckpoint() { + output.markSchemaChangeAfterCheckpoint(); + } + + @Override + public Object getCheckpointLock() { + return output.getCheckpointLock(); + } + } +} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index 0c8cb4d496a..5688fde5b64 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -27,20 +27,31 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.constants.JobMode; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.SourceReaderOptions; +import org.apache.seatunnel.connectors.seatunnel.kafka.source.fetch.KafkaSourceFetcherManager; import org.apache.seatunnel.connectors.seatunnel.kafka.state.KafkaSourceState; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import com.google.common.base.Supplier; + import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; public class KafkaSource implements SeaTunnelSource, SupportParallelism { + private final ReadonlyConfig readonlyConfig; private JobContext jobContext; private final KafkaSourceConfig kafkaSourceConfig; public KafkaSource(ReadonlyConfig readonlyConfig) { + this.readonlyConfig = readonlyConfig; kafkaSourceConfig = new KafkaSourceConfig(readonlyConfig); } @@ -66,10 +77,28 @@ public List getProducedCatalogTables() { @Override public SourceReader createReader( SourceReader.Context readerContext) { + + BlockingQueue>> elementsQueue = + new LinkedBlockingQueue<>(); + + Supplier kafkaPartitionSplitReaderSupplier = + () -> new KafkaPartitionSplitReader(kafkaSourceConfig, readerContext); + + KafkaSourceFetcherManager kafkaSourceFetcherManager = + new KafkaSourceFetcherManager( + elementsQueue, kafkaPartitionSplitReaderSupplier::get); + KafkaRecordEmitter kafkaRecordEmitter = + new KafkaRecordEmitter( + kafkaSourceConfig.getMapMetadata(), + kafkaSourceConfig.getMessageFormatErrorHandleWay()); + return new KafkaSourceReader( + elementsQueue, + kafkaSourceFetcherManager, + kafkaRecordEmitter, + new SourceReaderOptions(readonlyConfig), kafkaSourceConfig, - readerContext, - kafkaSourceConfig.getMessageFormatErrorHandleWay()); + readerContext); } @Override diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 1c782ca6ab7..960a0184029 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -44,6 +44,7 @@ import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import org.apache.seatunnel.format.json.maxwell.MaxWellJsonDeserializationSchema; import org.apache.seatunnel.format.json.ogg.OggJsonDeserializationSchema; +import org.apache.seatunnel.format.protobuf.ProtobufDeserializationSchema; import org.apache.seatunnel.format.text.TextDeserializationSchema; import org.apache.seatunnel.format.text.constant.TextFormatConstant; @@ -71,6 +72,8 @@ import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PATTERN; +import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_MESSAGE_NAME; +import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_OFFSETS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_TIMESTAMP; @@ -86,6 +89,7 @@ public class KafkaSourceConfig implements Serializable { @Getter private final Properties properties; @Getter private final long discoveryIntervalMillis; @Getter private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; + @Getter private final String consumerGroup; public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { this.bootstrap = readonlyConfig.get(BOOTSTRAP_SERVERS); @@ -95,6 +99,7 @@ public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { this.discoveryIntervalMillis = readonlyConfig.get(KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS); this.messageFormatErrorHandleWay = readonlyConfig.get(MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION); + this.consumerGroup = readonlyConfig.get(CONSUMER_GROUP); } private Properties createKafkaProperties(ReadonlyConfig readonlyConfig) { @@ -128,7 +133,6 @@ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { ConsumerMetadata consumerMetadata = new ConsumerMetadata(); consumerMetadata.setTopic(readonlyConfig.get(TOPIC)); consumerMetadata.setPattern(readonlyConfig.get(PATTERN)); - consumerMetadata.setConsumerGroup(readonlyConfig.get(CONSUMER_GROUP)); consumerMetadata.setProperties(new Properties()); // Create a catalog CatalogTable catalogTable = createCatalogTable(readonlyConfig); @@ -216,7 +220,15 @@ private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { return CatalogTable.of( TableIdentifier.of("", tablePath), tableSchema, - Collections.emptyMap(), + new HashMap() { + { + Optional.ofNullable(readonlyConfig.get(PROTOBUF_MESSAGE_NAME)) + .ifPresent(value -> put(PROTOBUF_MESSAGE_NAME.key(), value)); + + Optional.ofNullable(readonlyConfig.get(PROTOBUF_SCHEMA)) + .ifPresent(value -> put(PROTOBUF_SCHEMA.key(), value)); + } + }, Collections.emptyList(), null); } @@ -225,6 +237,8 @@ private DeserializationSchema createDeserializationSchema( CatalogTable catalogTable, ReadonlyConfig readonlyConfig) { SeaTunnelRowType seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + MessageFormat format = readonlyConfig.get(FORMAT); + if (!readonlyConfig.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { return TextDeserializationSchema.builder() .seaTunnelRowType(seaTunnelRowType) @@ -233,7 +247,6 @@ private DeserializationSchema createDeserializationSchema( .build(); } - MessageFormat format = readonlyConfig.get(FORMAT); switch (format) { case JSON: return new JsonDeserializationSchema(catalogTable, false, false); @@ -270,6 +283,8 @@ private DeserializationSchema createDeserializationSchema( return new DebeziumJsonDeserializationSchema(catalogTable, true, includeSchema); case AVRO: return new AvroDeserializationSchema(catalogTable); + case PROTOBUF: + return new ProtobufDeserializationSchema(catalogTable); default: throw new SeaTunnelJsonFormatException( CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, 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 6f4753110bd..82a0522c411 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 @@ -17,283 +17,156 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; -import org.apache.seatunnel.api.serialization.DeserializationSchema; -import org.apache.seatunnel.api.source.Boundedness; -import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; -import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorException; -import org.apache.seatunnel.format.compatible.kafka.connect.json.CompatibleKafkaConnectDeserializationSchema; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordEmitter; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.SourceReaderOptions; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SingleThreadFetcherManager; +import org.apache.seatunnel.connectors.seatunnel.kafka.source.fetch.KafkaSourceFetcherManager; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import com.google.common.collect.Sets; -import lombok.extern.slf4j.Slf4j; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.time.Duration; +import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.concurrent.CompletableFuture; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.stream.Collectors; +import java.util.concurrent.ConcurrentMap; -@Slf4j -public class KafkaSourceReader implements SourceReader { - - private static final long THREAD_WAIT_TIME = 500L; - private static final long POLL_TIMEOUT = 10000L; +public class KafkaSourceReader + extends SingleThreadMultiplexSourceReaderBase< + ConsumerRecord, + SeaTunnelRow, + KafkaSourceSplit, + KafkaSourceSplitState> { + private static final Logger logger = LoggerFactory.getLogger(KafkaSourceReader.class); private final SourceReader.Context context; - private final KafkaSourceConfig kafkaSourceConfig; - - private final Map tablePathMetadataMap; - private final Set sourceSplits; - private final Map> checkpointOffsetMap; - private final Map consumerThreadMap; - private final ExecutorService executorService; - private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; - private final LinkedBlockingQueue pendingPartitionsQueue; + private final KafkaSourceConfig kafkaSourceConfig; + private final SortedMap> checkpointOffsetMap; - private volatile boolean running = false; + private final ConcurrentMap offsetsOfFinishedSplits; KafkaSourceReader( + BlockingQueue>> elementsQueue, + SingleThreadFetcherManager, KafkaSourceSplit> + splitFetcherManager, + RecordEmitter, SeaTunnelRow, KafkaSourceSplitState> + recordEmitter, + SourceReaderOptions options, KafkaSourceConfig kafkaSourceConfig, - Context context, - MessageFormatErrorHandleWay messageFormatErrorHandleWay) { + Context context) { + super(elementsQueue, splitFetcherManager, recordEmitter, options, context); this.kafkaSourceConfig = kafkaSourceConfig; - this.tablePathMetadataMap = kafkaSourceConfig.getMapMetadata(); this.context = context; - this.messageFormatErrorHandleWay = messageFormatErrorHandleWay; - this.sourceSplits = new HashSet<>(); - this.consumerThreadMap = new ConcurrentHashMap<>(); - this.checkpointOffsetMap = new ConcurrentHashMap<>(); - this.executorService = - Executors.newCachedThreadPool(r -> new Thread(r, "Kafka Source Data Consumer")); - pendingPartitionsQueue = new LinkedBlockingQueue<>(); + this.checkpointOffsetMap = Collections.synchronizedSortedMap(new TreeMap<>()); + this.offsetsOfFinishedSplits = new ConcurrentHashMap<>(); } @Override - public void open() {} + protected void onSplitFinished(Map finishedSplitIds) { + finishedSplitIds.forEach( + (ignored, splitState) -> { + if (splitState.getCurrentOffset() > 0) { + offsetsOfFinishedSplits.put( + splitState.getTopicPartition(), + new OffsetAndMetadata(splitState.getCurrentOffset())); + } else if (splitState.getEndOffset() > 0) { + offsetsOfFinishedSplits.put( + splitState.getTopicPartition(), + new OffsetAndMetadata(splitState.getEndOffset())); + } + }); + } @Override - public void close() throws IOException { - if (executorService != null) { - executorService.shutdownNow(); - } + protected KafkaSourceSplitState initializedState(KafkaSourceSplit split) { + return new KafkaSourceSplitState(split); } @Override - public void pollNext(Collector output) throws Exception { - if (!running) { - Thread.sleep(THREAD_WAIT_TIME); - return; - } + protected KafkaSourceSplit toSplitType(String splitId, KafkaSourceSplitState splitState) { + return splitState.toKafkaSourceSplit(); + } - while (!pendingPartitionsQueue.isEmpty()) { - sourceSplits.add(pendingPartitionsQueue.poll()); + @Override + public List snapshotState(long checkpointId) { + List sourceSplits = super.snapshotState(checkpointId); + if (!kafkaSourceConfig.isCommitOnCheckpoint()) { + return sourceSplits; } - sourceSplits.forEach( - sourceSplit -> - consumerThreadMap.computeIfAbsent( - sourceSplit.getTopicPartition(), - s -> { - ConsumerMetadata currentSplitConsumerMetaData = - tablePathMetadataMap.get(sourceSplit.getTablePath()); - KafkaConsumerThread thread = - new KafkaConsumerThread( - kafkaSourceConfig, - currentSplitConsumerMetaData); - executorService.submit(thread); - return thread; - })); - List finishedSplits = new CopyOnWriteArrayList<>(); - sourceSplits.forEach( - sourceSplit -> { - CompletableFuture completableFuture = new CompletableFuture<>(); - TablePath tablePath = sourceSplit.getTablePath(); - DeserializationSchema deserializationSchema = - tablePathMetadataMap.get(tablePath).getDeserializationSchema(); - try { - consumerThreadMap - .get(sourceSplit.getTopicPartition()) - .getTasks() - .put( - consumer -> { - try { - Set partitions = - Sets.newHashSet( - sourceSplit.getTopicPartition()); - consumer.assign(partitions); - if (sourceSplit.getStartOffset() >= 0) { - consumer.seek( - sourceSplit.getTopicPartition(), - sourceSplit.getStartOffset()); - } - ConsumerRecords records = - consumer.poll( - Duration.ofMillis(POLL_TIMEOUT)); - for (TopicPartition partition : partitions) { - List> - recordList = records.records(partition); - if (Boundedness.BOUNDED.equals( - context.getBoundedness()) - && recordList.isEmpty()) { - completableFuture.complete(true); - return; - } - for (ConsumerRecord record : - recordList) { - try { - if (deserializationSchema - instanceof - CompatibleKafkaConnectDeserializationSchema) { - ((CompatibleKafkaConnectDeserializationSchema) - deserializationSchema) - .deserialize( - record, output); - } else { - deserializationSchema.deserialize( - record.value(), output); - } - } catch (IOException e) { - if (this.messageFormatErrorHandleWay - == MessageFormatErrorHandleWay - .SKIP) { - log.warn( - "Deserialize message failed, skip this message, message: {}", - new String(record.value())); - continue; - } - throw e; - } - - if (Boundedness.BOUNDED.equals( - context.getBoundedness()) - && record.offset() - >= sourceSplit - .getEndOffset()) { - completableFuture.complete(true); - return; - } - } - long lastOffset = -1; - if (!recordList.isEmpty()) { - lastOffset = - recordList - .get(recordList.size() - 1) - .offset(); - sourceSplit.setStartOffset(lastOffset + 1); - } - - if (lastOffset >= sourceSplit.getEndOffset()) { - sourceSplit.setEndOffset(lastOffset); - } - } - } catch (Exception e) { - completableFuture.completeExceptionally(e); - } - completableFuture.complete(false); - }); - if (completableFuture.get()) { - finishedSplits.add(sourceSplit); - } - } catch (Exception e) { - throw new KafkaConnectorException( - KafkaConnectorErrorCode.CONSUME_DATA_FAILED, e); - } - }); - if (Boundedness.BOUNDED.equals(context.getBoundedness())) { - for (KafkaSourceSplit split : finishedSplits) { - split.setFinish(true); - if (split.getStartOffset() == -1) { - // log next running read start offset - split.setStartOffset(split.getEndOffset()); + if (sourceSplits.isEmpty() && offsetsOfFinishedSplits.isEmpty()) { + logger.debug( + "checkpoint {} does not have an offset to submit for splits", checkpointId); + checkpointOffsetMap.put(checkpointId, Collections.emptyMap()); + } else { + Map offsetAndMetadataMap = + checkpointOffsetMap.computeIfAbsent(checkpointId, id -> new HashMap<>()); + for (KafkaSourceSplit kafkaSourceSplit : sourceSplits) { + if (kafkaSourceSplit.getStartOffset() >= 0) { + offsetAndMetadataMap.put( + kafkaSourceSplit.getTopicPartition(), + new OffsetAndMetadata(kafkaSourceSplit.getStartOffset())); } } - if (sourceSplits.stream().allMatch(KafkaSourceSplit::isFinish)) { - context.signalNoMoreElement(); - } + offsetAndMetadataMap.putAll(offsetsOfFinishedSplits); } + return sourceSplits; } @Override - public List snapshotState(long checkpointId) { - checkpointOffsetMap.put( - checkpointId, - sourceSplits.stream() - .collect( - Collectors.toMap( - KafkaSourceSplit::getTopicPartition, - KafkaSourceSplit::getStartOffset))); - return sourceSplits.stream().map(KafkaSourceSplit::copy).collect(Collectors.toList()); - } + public void notifyCheckpointComplete(long checkpointId) { + logger.debug("Committing offsets for checkpoint {}", checkpointId); + if (!kafkaSourceConfig.isCommitOnCheckpoint()) { + logger.debug("Submitting offsets after snapshot completion is prohibited"); + return; + } + Map committedPartitions = + checkpointOffsetMap.get(checkpointId); - @Override - public void addSplits(List splits) { - running = true; - splits.forEach( - s -> { - try { - pendingPartitionsQueue.put(s); - } catch (InterruptedException e) { - throw new KafkaConnectorException( - KafkaConnectorErrorCode.ADD_SPLIT_CHECKPOINT_FAILED, e); - } - }); - } + if (committedPartitions == null) { + logger.debug("Offsets for checkpoint {} have already been committed.", checkpointId); + return; + } - @Override - public void handleNoMoreSplits() { - log.info("receive no more splits message, this reader will not add new split."); + if (committedPartitions.isEmpty()) { + logger.debug("There are no offsets to commit for checkpoint {}.", checkpointId); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); + return; + } + + ((KafkaSourceFetcherManager) splitFetcherManager) + .commitOffsets( + committedPartitions, + (ignored, e) -> { + if (e != null) { + logger.warn( + "Failed to commit consumer offsets for checkpoint {}", + checkpointId, + e); + return; + } + offsetsOfFinishedSplits + .keySet() + .removeIf(committedPartitions::containsKey); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); + }); } - @Override - public void notifyCheckpointComplete(long checkpointId) { - if (!checkpointOffsetMap.containsKey(checkpointId)) { - log.warn("checkpoint {} do not exist or have already been committed.", checkpointId); - } else { - checkpointOffsetMap - .remove(checkpointId) - .forEach( - (topicPartition, offset) -> { - try { - consumerThreadMap - .get(topicPartition) - .getTasks() - .put( - consumer -> { - if (kafkaSourceConfig - .isCommitOnCheckpoint()) { - Map - offsets = new HashMap<>(); - if (offset >= 0) { - offsets.put( - topicPartition, - new OffsetAndMetadata( - offset)); - consumer.commitSync(offsets); - } - } - }); - } catch (InterruptedException e) { - log.error("commit offset to kafka failed", e); - } - }); + private void removeAllOffsetsToCommitUpToCheckpoint(long checkpointId) { + while (!checkpointOffsetMap.isEmpty() && checkpointOffsetMap.firstKey() <= checkpointId) { + checkpointOffsetMap.remove(checkpointOffsetMap.firstKey()); } } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index f868eaed20c..06ce4565c3b 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -151,8 +151,7 @@ private void setPartitionStartOffset() throws ExecutionException, InterruptedExc listOffsets(topicPartitions, OffsetSpec.earliest())); break; case GROUP_OFFSETS: - topicPartitionOffsets.putAll( - listConsumerGroupOffsets(topicPartitions, metadata)); + topicPartitionOffsets.putAll(listConsumerGroupOffsets(topicPartitions)); break; case LATEST: topicPartitionOffsets.putAll(listOffsets(topicPartitions, OffsetSpec.latest())); @@ -366,13 +365,12 @@ private Map listOffsets( .get(); } - public Map listConsumerGroupOffsets( - Collection partitions, ConsumerMetadata metadata) + public Map listConsumerGroupOffsets(Collection partitions) throws ExecutionException, InterruptedException { ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions().topicPartitions(new ArrayList<>(partitions)); return adminClient - .listConsumerGroupOffsets(metadata.getConsumerGroup(), options) + .listConsumerGroupOffsets(kafkaSourceConfig.getConsumerGroup(), options) .partitionsToOffsetAndMetadata() .thenApply( result -> { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java new file mode 100644 index 00000000000..ab2592cc38e --- /dev/null +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.kafka.source; + +public class KafkaSourceSplitState extends KafkaSourceSplit { + + private long currentOffset; + + public KafkaSourceSplitState(KafkaSourceSplit sourceSplit) { + super( + sourceSplit.getTablePath(), + sourceSplit.getTopicPartition(), + sourceSplit.getStartOffset(), + sourceSplit.getEndOffset()); + this.currentOffset = sourceSplit.getStartOffset(); + } + + public long getCurrentOffset() { + return currentOffset; + } + + public void setCurrentOffset(long currentOffset) { + this.currentOffset = currentOffset; + } + + public KafkaSourceSplit toKafkaSourceSplit() { + return new KafkaSourceSplit( + getTablePath(), getTopicPartition(), getCurrentOffset(), getEndOffset()); + } +} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java new file mode 100644 index 00000000000..bc80455725d --- /dev/null +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.kafka.source.fetch; + +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SingleThreadFetcherManager; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcher; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcherTask; +import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitReader; +import org.apache.seatunnel.connectors.seatunnel.kafka.source.KafkaPartitionSplitReader; +import org.apache.seatunnel.connectors.seatunnel.kafka.source.KafkaSourceSplit; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; +import org.apache.kafka.common.TopicPartition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.function.Consumer; +import java.util.function.Supplier; + +public class KafkaSourceFetcherManager + extends SingleThreadFetcherManager, KafkaSourceSplit> { + + private static final Logger logger = LoggerFactory.getLogger(KafkaSourceFetcherManager.class); + + public KafkaSourceFetcherManager( + BlockingQueue>> elementsQueue, + Supplier, KafkaSourceSplit>> + splitReaderSupplier) { + super(elementsQueue, splitReaderSupplier); + } + + public KafkaSourceFetcherManager( + BlockingQueue>> elementsQueue, + Supplier, KafkaSourceSplit>> + splitReaderSupplier, + Consumer> splitFinishedHook) { + super(elementsQueue, splitReaderSupplier, splitFinishedHook); + } + + public void commitOffsets( + Map offsetsToCommit, OffsetCommitCallback callback) { + logger.debug("Committing offsets {}", offsetsToCommit); + if (offsetsToCommit.isEmpty()) { + return; + } + SplitFetcher, KafkaSourceSplit> splitFetcher = + fetchers.get(0); + if (splitFetcher != null) { + // The fetcher thread is still running. This should be the majority of the cases. + enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback); + } else { + splitFetcher = createSplitFetcher(); + enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback); + startFetcher(splitFetcher); + } + } + + private void enqueueOffsetsCommitTask( + SplitFetcher, KafkaSourceSplit> splitFetcher, + Map offsetsToCommit, + OffsetCommitCallback callback) { + KafkaPartitionSplitReader kafkaReader = + (KafkaPartitionSplitReader) splitFetcher.getSplitReader(); + + splitFetcher.addTask( + new SplitFetcherTask() { + @Override + public void run() throws IOException { + kafkaReader.notifyCheckpointComplete(offsetsToCommit, callback); + } + + @Override + public void wakeUp() {} + }); + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java index 505b30fcbde..8eda6612c70 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java @@ -401,7 +401,7 @@ private static byte[] convertToBinary(BsonValue bsonValue) { } private static long convertToLong(BsonValue bsonValue) { - if (bsonValue.isInt64()) { + if (bsonValue.isInt64() || bsonValue.isInt32()) { return bsonValue.asNumber().longValue(); } throw new MongodbConnectorException( diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java index 6ce9596daa9..4ef08c53988 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java @@ -108,7 +108,12 @@ private ImmutablePair getDocumentNumAndAvgSize() { clientProvider.getDefaultCollection().getNamespace().getCollectionName(); BsonDocument statsCmd = new BsonDocument("collStats", new BsonString(collectionName)); Document res = clientProvider.getDefaultDatabase().runCommand(statsCmd); - long total = res.getInteger("count"); + Object count = res.get("count"); + // fix issue https://github.com/apache/seatunnel/issues/7575 + long total = + Optional.ofNullable(count) + .map(v -> Long.parseLong(String.valueOf(count))) + .orElse(0L); Object avgDocumentBytes = res.get("avgObjSize"); long avgObjSize = Optional.ofNullable(avgDocumentBytes) diff --git a/seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java b/seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java new file mode 100644 index 00000000000..b47769c0aca --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java @@ -0,0 +1,55 @@ +/* + * 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.mongodb.serde; + +import org.apache.seatunnel.api.table.type.BasicType; + +import org.bson.BsonDouble; +import org.bson.BsonInt32; +import org.bson.BsonInt64; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class BsonToRowDataConvertersTest { + private final BsonToRowDataConverters converterFactory = new BsonToRowDataConverters(); + + @Test + public void testConvertAnyNumberToDouble() { + // It covered #6997 + BsonToRowDataConverters.BsonToRowDataConverter converter = + converterFactory.createConverter(BasicType.DOUBLE_TYPE); + + Assertions.assertEquals(1.0d, converter.convert(new BsonInt32(1))); + Assertions.assertEquals(1.0d, converter.convert(new BsonInt64(1L))); + + Assertions.assertEquals(4.0d, converter.convert(new BsonDouble(4.0d))); + Assertions.assertEquals(4.4d, converter.convert(new BsonDouble(4.4d))); + } + + @Test + public void testConvertBsonIntToBigInt() { + // It covered #7567 + BsonToRowDataConverters.BsonToRowDataConverter converter = + converterFactory.createConverter(BasicType.LONG_TYPE); + + Assertions.assertEquals(123456L, converter.convert(new BsonInt32(123456))); + + Assertions.assertEquals( + (long) Integer.MAX_VALUE, converter.convert(new BsonInt64(Integer.MAX_VALUE))); + } +} diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java index d896e015398..9e09035e2f0 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java @@ -40,6 +40,7 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import java.io.Closeable; @@ -183,6 +184,35 @@ public void createDatabase(TablePath tablePath, boolean ignoreIfExists) } } + @Override + public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + try { + Identifier identifier = toIdentifier(tablePath); + FileStoreTable table = (FileStoreTable) catalog.getTable(identifier); + Schema schema = buildPaimonSchema(table.schema()); + dropTable(tablePath, ignoreIfNotExists); + catalog.createTable(identifier, schema, ignoreIfNotExists); + } catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) { + throw new TableNotExistException(this.catalogName, tablePath); + } catch (org.apache.paimon.catalog.Catalog.TableAlreadyExistException e) { + throw new DatabaseAlreadyExistException(this.catalogName, tablePath.getDatabaseName()); + } catch (org.apache.paimon.catalog.Catalog.DatabaseNotExistException e) { + throw new DatabaseNotExistException(this.catalogName, tablePath.getDatabaseName()); + } + } + + private Schema buildPaimonSchema(@NonNull org.apache.paimon.schema.TableSchema schema) { + Schema.Builder builder = Schema.newBuilder(); + schema.fields() + .forEach(field -> builder.column(field.name(), field.type(), field.description())); + builder.options(schema.options()); + builder.primaryKey(schema.primaryKeys()); + builder.partitionKeys(schema.partitionKeys()); + builder.comment(schema.comment()); + return builder.build(); + } + @Override public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) throws DatabaseNotExistException, CatalogException { diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java index ef37e52c01e..ed4c80a40d0 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java @@ -26,7 +26,8 @@ public enum PaimonConnectorErrorCode implements SeaTunnelErrorCode { TABLE_PRE_COMMIT_FAILED("PAIMON-03", "Paimon pre commit failed"), GET_TABLE_FAILED("PAIMON-04", "Get table from database failed"), AUTHENTICATE_KERBEROS_FAILED("PAIMON-05", "Authenticate kerberos failed"), - LOAD_CATALOG("PAIMON-06", "Load catalog failed"); + LOAD_CATALOG("PAIMON-06", "Load catalog failed"), + GET_FILED_FAILED("PAIMON-07", "Get field failed"); private final String code; private final String description; diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java index 9c576018a32..580f55b5817 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java @@ -346,17 +346,18 @@ public static SeaTunnelRow convert( * * @param seaTunnelRow SeaTunnel row object * @param seaTunnelRowType SeaTunnel row type - * @param tableSchema Paimon table schema + * @param sinkTableSchema Paimon table schema * @return Paimon row object */ public static InternalRow reconvert( - SeaTunnelRow seaTunnelRow, SeaTunnelRowType seaTunnelRowType, TableSchema tableSchema) { - List sinkTotalFields = tableSchema.fields(); + SeaTunnelRow seaTunnelRow, + SeaTunnelRowType seaTunnelRowType, + TableSchema sinkTableSchema) { + List sinkTotalFields = sinkTableSchema.fields(); int sourceTotalFields = seaTunnelRowType.getTotalFields(); if (sourceTotalFields != sinkTotalFields.size()) { - throw new CommonError() - .writeRowErrorWithFiledsCountNotMatch( - "Paimon", sourceTotalFields, sinkTotalFields.size()); + throw CommonError.writeRowErrorWithFiledsCountNotMatch( + "Paimon", sourceTotalFields, sinkTotalFields.size()); } BinaryRow binaryRow = new BinaryRow(sourceTotalFields); BinaryWriter binaryWriter = new BinaryRowWriter(binaryRow); @@ -377,7 +378,7 @@ public static InternalRow reconvert( binaryWriter.setNullAt(i); continue; } - checkCanWriteWithType(i, seaTunnelRowType, sinkTotalFields); + checkCanWriteWithSchema(i, seaTunnelRowType, sinkTotalFields); String fieldName = seaTunnelRowType.getFieldName(i); switch (fieldTypes[i].getSqlType()) { case TINYINT: @@ -399,14 +400,17 @@ public static InternalRow reconvert( binaryWriter.writeDouble(i, (Double) seaTunnelRow.getField(i)); break; case DECIMAL: - DecimalType fieldType = (DecimalType) seaTunnelRowType.getFieldType(i); + DataField decimalDataField = + SchemaUtil.getDataField(sinkTotalFields, fieldName); + org.apache.paimon.types.DecimalType decimalType = + (org.apache.paimon.types.DecimalType) decimalDataField.type(); binaryWriter.writeDecimal( i, Decimal.fromBigDecimal( (BigDecimal) seaTunnelRow.getField(i), - fieldType.getPrecision(), - fieldType.getScale()), - fieldType.getPrecision()); + decimalType.getPrecision(), + decimalType.getScale()), + decimalType.getPrecision()); break; case STRING: binaryWriter.writeString( @@ -464,9 +468,12 @@ public static InternalRow reconvert( SeaTunnelDataType rowType = seaTunnelRowType.getFieldType(i); Object row = seaTunnelRow.getField(i); InternalRow paimonRow = - reconvert((SeaTunnelRow) row, (SeaTunnelRowType) rowType, tableSchema); + reconvert( + (SeaTunnelRow) row, + (SeaTunnelRowType) rowType, + sinkTableSchema); RowType paimonRowType = - RowTypeConverter.reconvert((SeaTunnelRowType) rowType, tableSchema); + RowTypeConverter.reconvert((SeaTunnelRowType) rowType, sinkTableSchema); binaryWriter.writeRow(i, paimonRow, new InternalRowSerializer(paimonRowType)); break; default: @@ -479,7 +486,7 @@ public static InternalRow reconvert( return binaryRow; } - private static void checkCanWriteWithType( + private static void checkCanWriteWithSchema( int i, SeaTunnelRowType seaTunnelRowType, List fields) { String sourceFieldName = seaTunnelRowType.getFieldName(i); SeaTunnelDataType sourceFieldType = seaTunnelRowType.getFieldType(i); @@ -488,13 +495,27 @@ private static void checkCanWriteWithType( RowTypeConverter.reconvert(sourceFieldName, seaTunnelRowType.getFieldType(i)); DataField exceptDataField = new DataField(i, sourceFieldName, exceptDataType); DataType sinkDataType = sinkDataField.type(); - if (!exceptDataType.getTypeRoot().equals(sinkDataType.getTypeRoot())) { - throw new CommonError() - .writeRowErrorWithSchemaIncompatibleSchema( - "Paimon", - sourceFieldName + StringUtils.SPACE + sourceFieldType.getSqlType(), - exceptDataField.asSQLString(), - sinkDataField.asSQLString()); + if (!exceptDataType.getTypeRoot().equals(sinkDataType.getTypeRoot()) + || !StringUtils.equals(sourceFieldName, sinkDataField.name())) { + throw CommonError.writeRowErrorWithSchemaIncompatibleSchema( + "Paimon", + sourceFieldName + StringUtils.SPACE + sourceFieldType.getSqlType(), + exceptDataField.asSQLString(), + sinkDataField.asSQLString()); + } + if (sourceFieldType instanceof DecimalType + && sinkDataType instanceof org.apache.paimon.types.DecimalType) { + DecimalType sourceDecimalType = (DecimalType) sourceFieldType; + org.apache.paimon.types.DecimalType sinkDecimalType = + (org.apache.paimon.types.DecimalType) sinkDataType; + if (sinkDecimalType.getPrecision() < sourceDecimalType.getPrecision() + || sinkDecimalType.getScale() < sourceDecimalType.getScale()) { + throw CommonError.writeRowErrorWithSchemaIncompatibleSchema( + "Paimon", + sourceFieldName + StringUtils.SPACE + sourceFieldType.getSqlType(), + exceptDataField.asSQLString(), + sinkDataField.asSQLString()); + } } } } diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java index 0da047244fa..fa8ed338208 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.data.PaimonTypeMapper; +import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorException; import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataField; @@ -30,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; /** The util seatunnel schema to paimon schema */ public class SchemaUtil { @@ -69,9 +72,13 @@ public static Column toSeaTunnelType(BasicTypeDefine typeDefine) { } public static DataField getDataField(List fields, String fieldName) { - return fields.parallelStream() - .filter(field -> field.name().equals(fieldName)) - .findFirst() - .get(); + Optional firstField = + fields.stream().filter(field -> field.name().equals(fieldName)).findFirst(); + if (!firstField.isPresent()) { + throw new PaimonConnectorException( + PaimonConnectorErrorCode.GET_FILED_FAILED, + "Can not get the filed [" + fieldName + "] from source table"); + } + return firstField.get(); } } diff --git a/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java b/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java index ebde744d032..c574b77e125 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java +++ b/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java @@ -26,7 +26,10 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.commons.lang3.StringUtils; import org.apache.paimon.data.BinaryArray; import org.apache.paimon.data.BinaryArrayWriter; import org.apache.paimon.data.BinaryMap; @@ -39,6 +42,7 @@ import org.apache.paimon.data.serializer.InternalArraySerializer; import org.apache.paimon.data.serializer.InternalMapSerializer; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -48,6 +52,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import lombok.extern.slf4j.Slf4j; + import java.math.BigDecimal; import java.time.LocalDate; import java.time.LocalDateTime; @@ -58,6 +64,7 @@ import java.util.Map; /** Unit tests for {@link RowConverter} */ +@Slf4j public class RowConverterTest { private SeaTunnelRow seaTunnelRow; @@ -66,49 +73,64 @@ public class RowConverterTest { private SeaTunnelRowType seaTunnelRowType; - private TableSchema tableSchema; - - public static final RowType DEFAULT_ROW_TYPE = - RowType.of( - new DataType[] { - DataTypes.TINYINT(), - DataTypes.SMALLINT(), - DataTypes.INT(), - DataTypes.BIGINT(), - DataTypes.FLOAT(), - DataTypes.DOUBLE(), - DataTypes.DECIMAL(10, 10), - DataTypes.STRING(), - DataTypes.BYTES(), - DataTypes.BOOLEAN(), - DataTypes.DATE(), - DataTypes.TIMESTAMP(), - DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING()), - DataTypes.ARRAY(DataTypes.STRING()) - }, - new String[] { - "c_tinyint", - "c_smallint", - "c_int", - "c_bigint", - "c_float", - "c_double", - "c_decimal", - "c_string", - "c_bytes", - "c_boolean", - "c_date", - "c_timestamp", - "c_map", - "c_array" - }); + private volatile boolean isCaseSensitive = false; + private volatile boolean subtractOneFiledInSource = false; + private volatile int index = 0; + private static final String[] filedNames = { + "c_tinyint", + "c_smallint", + "c_int", + "c_bigint", + "c_float", + "c_double", + "c_decimal", + "c_string", + "c_bytes", + "c_boolean", + "c_date", + "c_timestamp", + "c_map", + "c_array" + }; + + public static final SeaTunnelDataType[] seaTunnelDataTypes = { + BasicType.BYTE_TYPE, + BasicType.SHORT_TYPE, + BasicType.INT_TYPE, + BasicType.LONG_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + new DecimalType(30, 8), + BasicType.STRING_TYPE, + PrimitiveByteArrayType.INSTANCE, + BasicType.BOOLEAN_TYPE, + LocalTimeType.LOCAL_DATE_TYPE, + LocalTimeType.LOCAL_DATE_TIME_TYPE, + new MapType<>(BasicType.STRING_TYPE, BasicType.STRING_TYPE), + ArrayType.STRING_ARRAY_TYPE + }; public static final List KEY_NAME_LIST = Arrays.asList("c_tinyint"); - @BeforeEach - public void before() { - seaTunnelRowType = - new SeaTunnelRowType( + public TableSchema getTableSchema(int decimalPrecision, int decimalScale) { + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.TINYINT(), + DataTypes.SMALLINT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.DECIMAL(decimalPrecision, decimalScale), + DataTypes.STRING(), + DataTypes.BYTES(), + DataTypes.BOOLEAN(), + DataTypes.DATE(), + DataTypes.TIMESTAMP(), + DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING()), + DataTypes.ARRAY(DataTypes.STRING()) + }, new String[] { "c_tinyint", "c_smallint", @@ -124,23 +146,21 @@ public void before() { "c_timestamp", "c_map", "c_array" - }, - new SeaTunnelDataType[] { - BasicType.BYTE_TYPE, - BasicType.SHORT_TYPE, - BasicType.INT_TYPE, - BasicType.LONG_TYPE, - BasicType.FLOAT_TYPE, - BasicType.DOUBLE_TYPE, - new DecimalType(30, 8), - BasicType.STRING_TYPE, - PrimitiveByteArrayType.INSTANCE, - BasicType.BOOLEAN_TYPE, - LocalTimeType.LOCAL_DATE_TYPE, - LocalTimeType.LOCAL_DATE_TIME_TYPE, - new MapType<>(BasicType.STRING_TYPE, BasicType.STRING_TYPE), - ArrayType.STRING_ARRAY_TYPE }); + + return new TableSchema( + 0, + TableSchema.newFields(rowType), + rowType.getFieldCount(), + Collections.EMPTY_LIST, + KEY_NAME_LIST, + Collections.EMPTY_MAP, + ""); + } + + @BeforeEach + public void generateTestData() { + initSeaTunnelRowTypeCaseSensitive(isCaseSensitive, index, subtractOneFiledInSource); byte tinyint = 1; short smallint = 2; int intNum = 3; @@ -215,27 +235,96 @@ public void before() { binaryRowWriter.writeArray( 13, binaryArray2, new InternalArraySerializer(DataTypes.STRING())); internalRow = binaryRow; + } - tableSchema = - new TableSchema( - 0, - TableSchema.newFields(DEFAULT_ROW_TYPE), - DEFAULT_ROW_TYPE.getFieldCount(), - Collections.EMPTY_LIST, - KEY_NAME_LIST, - Collections.EMPTY_MAP, - ""); + private void initSeaTunnelRowTypeCaseSensitive( + boolean isUpperCase, int index, boolean subtractOneFiledInSource) { + String[] oneUpperCaseFiledNames = + Arrays.copyOf( + filedNames, + subtractOneFiledInSource ? filedNames.length - 1 : filedNames.length); + if (isUpperCase) { + oneUpperCaseFiledNames[index] = oneUpperCaseFiledNames[index].toUpperCase(); + } + SeaTunnelDataType[] newSeaTunnelDataTypes = + Arrays.copyOf( + seaTunnelDataTypes, + subtractOneFiledInSource + ? seaTunnelDataTypes.length - 1 + : filedNames.length); + seaTunnelRowType = new SeaTunnelRowType(oneUpperCaseFiledNames, newSeaTunnelDataTypes); } @Test public void seaTunnelToPaimon() { - InternalRow convert = RowConverter.reconvert(seaTunnelRow, seaTunnelRowType, tableSchema); - Assertions.assertEquals(convert, internalRow); + TableSchema sinkTableSchema = getTableSchema(30, 8); + SeaTunnelRuntimeException actualException = + Assertions.assertThrows( + SeaTunnelRuntimeException.class, + () -> + RowConverter.reconvert( + seaTunnelRow, seaTunnelRowType, getTableSchema(10, 10))); + SeaTunnelRuntimeException exceptedException = + CommonError.writeRowErrorWithSchemaIncompatibleSchema( + "Paimon", + "c_decimal" + StringUtils.SPACE + "DECIMAL", + "`c_decimal` DECIMAL(30, 8)", + "`c_decimal` DECIMAL(10, 10)"); + Assertions.assertEquals(exceptedException.getMessage(), actualException.getMessage()); + + InternalRow reconvert = + RowConverter.reconvert(seaTunnelRow, seaTunnelRowType, sinkTableSchema); + Assertions.assertEquals(reconvert, internalRow); + + subtractOneFiledInSource = true; + generateTestData(); + SeaTunnelRuntimeException filedNumsActualException = + Assertions.assertThrows( + SeaTunnelRuntimeException.class, + () -> + RowConverter.reconvert( + seaTunnelRow, seaTunnelRowType, sinkTableSchema)); + SeaTunnelRuntimeException filedNumsExceptException = + CommonError.writeRowErrorWithFiledsCountNotMatch( + "Paimon", + seaTunnelRowType.getTotalFields(), + sinkTableSchema.fields().size()); + Assertions.assertEquals( + filedNumsExceptException.getMessage(), filedNumsActualException.getMessage()); + + subtractOneFiledInSource = false; + isCaseSensitive = true; + + for (int i = 0; i < filedNames.length; i++) { + index = i; + generateTestData(); + String sourceFiledname = seaTunnelRowType.getFieldName(i); + DataType exceptDataType = + RowTypeConverter.reconvert(sourceFiledname, seaTunnelRowType.getFieldType(i)); + DataField exceptDataField = new DataField(i, sourceFiledname, exceptDataType); + SeaTunnelRuntimeException actualException1 = + Assertions.assertThrows( + SeaTunnelRuntimeException.class, + () -> + RowConverter.reconvert( + seaTunnelRow, seaTunnelRowType, sinkTableSchema)); + Assertions.assertEquals( + CommonError.writeRowErrorWithSchemaIncompatibleSchema( + "Paimon", + sourceFiledname + + StringUtils.SPACE + + seaTunnelRowType.getFieldType(i).getSqlType(), + exceptDataField.asSQLString(), + sinkTableSchema.fields().get(i).asSQLString()) + .getMessage(), + actualException1.getMessage()); + } } @Test public void paimonToSeaTunnel() { - SeaTunnelRow convert = RowConverter.convert(internalRow, seaTunnelRowType, tableSchema); + SeaTunnelRow convert = + RowConverter.convert(internalRow, seaTunnelRowType, getTableSchema(10, 10)); Assertions.assertEquals(convert, seaTunnelRow); } } diff --git a/seatunnel-connectors-v2/connector-qdrant/pom.xml b/seatunnel-connectors-v2/connector-qdrant/pom.xml new file mode 100644 index 00000000000..686f0bdb7a4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/pom.xml @@ -0,0 +1,63 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connectors-v2 + ${revision} + + + connector-qdrant + SeaTunnel : Connectors V2 : Qdrant + + + connector.qdrant + + + + + org.apache.seatunnel + connector-common + ${project.version} + + + + io.grpc + grpc-protobuf + 1.65.1 + + + + org.apache.seatunnel + seatunnel-guava + ${project.version} + optional + + + + io.qdrant + client + 1.11.0 + + + + diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java new file mode 100644 index 00000000000..1be03a13111 --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.qdrant.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +public class QdrantConfig { + + public static final String CONNECTOR_IDENTITY = "Qdrant"; + + public static final Option HOST = + Options.key("host") + .stringType() + .defaultValue("localhost") + .withDescription("Qdrant gRPC host"); + + public static final Option PORT = + Options.key("port").intType().defaultValue(6334).withDescription("Qdrant gRPC port"); + + public static final Option API_KEY = + Options.key("api_key").stringType().defaultValue("").withDescription("Qdrant API key"); + + public static final Option COLLECTION_NAME = + Options.key("collection_name") + .stringType() + .noDefaultValue() + .withDescription("Qdrant collection name"); + + public static final Option USE_TLS = + Options.key("use_tls") + .booleanType() + .defaultValue(false) + .withDescription("Whether to use TLS"); +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java new file mode 100644 index 00000000000..1ae612fafbb --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java @@ -0,0 +1,47 @@ +/* + * 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.qdrant.config; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; + +import io.qdrant.client.QdrantClient; +import io.qdrant.client.QdrantGrpcClient; +import lombok.Data; + +import java.io.Serializable; + +@Data +public class QdrantParameters implements Serializable { + private String host; + private int port; + private String apiKey; + private String collectionName; + private boolean useTls; + + public QdrantParameters(ReadonlyConfig config) { + this.host = config.get(QdrantConfig.HOST); + this.port = config.get(QdrantConfig.PORT); + this.apiKey = config.get(QdrantConfig.API_KEY); + this.collectionName = config.get(QdrantConfig.COLLECTION_NAME); + this.useTls = config.get(QdrantConfig.USE_TLS); + } + + public QdrantClient buildQdrantClient() { + return new QdrantClient(QdrantGrpcClient.newBuilder(host, port, useTls).build()); + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.java new file mode 100644 index 00000000000..becf31abf5f --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.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.qdrant.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class QdrantConnectorException extends SeaTunnelRuntimeException { + public QdrantConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } + + public QdrantConnectorException( + SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { + super(seaTunnelErrorCode, errorMessage, cause); + } + + public QdrantConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, Throwable cause) { + super(seaTunnelErrorCode, cause); + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java new file mode 100644 index 00000000000..7ca4428c81e --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java @@ -0,0 +1,190 @@ +/* + * 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.qdrant.sink; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +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.CommonErrorCode; +import org.apache.seatunnel.common.utils.BufferUtils; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; +import org.apache.seatunnel.connectors.seatunnel.qdrant.exception.QdrantConnectorException; + +import org.apache.commons.collections4.CollectionUtils; + +import io.qdrant.client.QdrantClient; +import io.qdrant.client.ValueFactory; +import io.qdrant.client.VectorFactory; +import io.qdrant.client.grpc.JsonWithInt; +import io.qdrant.client.grpc.Points; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +import static io.qdrant.client.PointIdFactory.id; +import static org.apache.seatunnel.api.table.catalog.PrimaryKey.isPrimaryKeyField; + +public class QdrantBatchWriter { + + private final int batchSize; + private final CatalogTable catalogTable; + private final String collectionName; + private final QdrantClient qdrantClient; + + private final List qdrantDataCache; + private volatile int writeCount = 0; + + public QdrantBatchWriter( + CatalogTable catalogTable, Integer batchSize, QdrantParameters params) { + this.catalogTable = catalogTable; + this.qdrantClient = params.buildQdrantClient(); + this.collectionName = params.getCollectionName(); + this.batchSize = batchSize; + this.qdrantDataCache = new ArrayList<>(batchSize); + } + + public void addToBatch(SeaTunnelRow element) { + Points.PointStruct point = buildPoint(element); + qdrantDataCache.add(point); + writeCount++; + } + + public boolean needFlush() { + return this.writeCount >= this.batchSize; + } + + public synchronized void flush() { + if (CollectionUtils.isEmpty(this.qdrantDataCache)) { + return; + } + upsert(); + this.qdrantDataCache.clear(); + this.writeCount = 0; + } + + public void close() { + this.qdrantClient.close(); + } + + private Points.PointStruct buildPoint(SeaTunnelRow element) { + SeaTunnelRowType seaTunnelRowType = catalogTable.getSeaTunnelRowType(); + PrimaryKey primaryKey = catalogTable.getTableSchema().getPrimaryKey(); + + Points.PointStruct.Builder point = Points.PointStruct.newBuilder(); + Points.NamedVectors.Builder namedVectors = Points.NamedVectors.newBuilder(); + for (int i = 0; i < seaTunnelRowType.getFieldNames().length; i++) { + String fieldName = seaTunnelRowType.getFieldNames()[i]; + SeaTunnelDataType fieldType = seaTunnelRowType.getFieldType(i); + Object value = element.getField(i); + + if (isPrimaryKeyField(primaryKey, fieldName)) { + point.setId(pointId(fieldType, value)); + continue; + } + + JsonWithInt.Value payloadValue = buildPayload(fieldType, value); + if (payloadValue != null) { + point.putPayload(fieldName, payloadValue); + continue; + } + + Points.Vector vector = buildVector(fieldType, value); + if (vector != null) { + namedVectors.putVectors(fieldName, vector); + } + } + + if (!point.hasId()) { + point.setId(id(UUID.randomUUID())); + } + + point.setVectors(Points.Vectors.newBuilder().setVectors(namedVectors).build()); + return point.build(); + } + + private void upsert() { + try { + qdrantClient + .upsertAsync( + Points.UpsertPoints.newBuilder() + .setCollectionName(collectionName) + .addAllPoints(qdrantDataCache) + .build()) + .get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Upsert failed", e); + } + } + + public static Points.PointId pointId(SeaTunnelDataType fieldType, Object value) { + SqlType sqlType = fieldType.getSqlType(); + switch (sqlType) { + case INT: + return id(Integer.parseInt(value.toString())); + case STRING: + return id(UUID.fromString(value.toString())); + default: + throw new QdrantConnectorException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + "Unexpected value type for point ID: " + sqlType.name()); + } + } + + public static JsonWithInt.Value buildPayload(SeaTunnelDataType fieldType, Object value) { + SqlType sqlType = fieldType.getSqlType(); + switch (sqlType) { + case SMALLINT: + case INT: + case BIGINT: + return ValueFactory.value(Integer.parseInt(value.toString())); + case FLOAT: + case DOUBLE: + return ValueFactory.value(Long.parseLong(value.toString())); + case STRING: + case DATE: + return ValueFactory.value(value.toString()); + case BOOLEAN: + return ValueFactory.value(Boolean.parseBoolean(value.toString())); + default: + return null; + } + } + + public static Points.Vector buildVector(SeaTunnelDataType fieldType, Object value) { + SqlType sqlType = fieldType.getSqlType(); + switch (sqlType) { + case FLOAT_VECTOR: + case FLOAT16_VECTOR: + case BFLOAT16_VECTOR: + case BINARY_VECTOR: + ByteBuffer floatVectorBuffer = (ByteBuffer) value; + Float[] floats = BufferUtils.toFloatArray(floatVectorBuffer); + return VectorFactory.vector(Arrays.stream(floats).collect(Collectors.toList())); + default: + return null; + } + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java new file mode 100644 index 00000000000..85119032c86 --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.qdrant.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; + +import java.io.IOException; + +public class QdrantSink extends AbstractSimpleSink + implements SupportMultiTableSink { + private final QdrantParameters qdrantParameters; + private final CatalogTable catalogTable; + + public QdrantSink(ReadonlyConfig config, CatalogTable table) { + this.qdrantParameters = new QdrantParameters(config); + this.catalogTable = table; + } + + @Override + public String getPluginName() { + return QdrantConfig.CONNECTOR_IDENTITY; + } + + @Override + public QdrantSinkWriter createWriter(SinkWriter.Context context) throws IOException { + return new QdrantSinkWriter(catalogTable, qdrantParameters); + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java new file mode 100644 index 00000000000..a7ed5599d50 --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java @@ -0,0 +1,55 @@ +/* + * 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.qdrant.sink; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.sink.SinkCommonOptions; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableSink; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class QdrantSinkFactory implements TableSinkFactory { + @Override + public String factoryIdentifier() { + return QdrantConfig.CONNECTOR_IDENTITY; + } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + CatalogTable catalogTable = context.getCatalogTable(); + return () -> new QdrantSink(context.getOptions(), catalogTable); + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .optional( + QdrantConfig.HOST, + QdrantConfig.PORT, + QdrantConfig.API_KEY, + QdrantConfig.USE_TLS, + SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) + .build(); + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java new file mode 100644 index 00000000000..a0e00838b6f --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java @@ -0,0 +1,60 @@ +/* + * 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.qdrant.sink; + +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; + +import java.io.IOException; +import java.util.Optional; + +public class QdrantSinkWriter extends AbstractSinkWriter + implements SupportMultiTableSinkWriter { + + private final QdrantBatchWriter batchWriter; + + public QdrantSinkWriter(CatalogTable catalog, QdrantParameters qdrantParameters) { + int batchSize = 64; + this.batchWriter = new QdrantBatchWriter(catalog, batchSize, qdrantParameters); + } + + @Override + public void write(SeaTunnelRow element) throws IOException { + batchWriter.addToBatch(element); + if (batchWriter.needFlush()) { + batchWriter.flush(); + } + } + + @Override + public Optional prepareCommit() { + batchWriter.flush(); + return Optional.empty(); + } + + private void clearBuffer() {} + + @Override + public void close() throws IOException { + batchWriter.flush(); + batchWriter.close(); + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java new file mode 100644 index 00000000000..39aeb3a8793 --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java @@ -0,0 +1,63 @@ +/* + * 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.qdrant.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitReader; +import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitSource; +import org.apache.seatunnel.connectors.seatunnel.common.source.SingleSplitReaderContext; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; + +import java.util.Collections; +import java.util.List; + +public class QdrantSource extends AbstractSingleSplitSource { + private final QdrantParameters qdrantParameters; + private final CatalogTable catalogTable; + + @Override + public String getPluginName() { + return QdrantConfig.CONNECTOR_IDENTITY; + } + + public QdrantSource(ReadonlyConfig readonlyConfig) { + this.qdrantParameters = new QdrantParameters(readonlyConfig); + this.catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public List getProducedCatalogTables() { + return Collections.singletonList(catalogTable); + } + + @Override + public AbstractSingleSplitReader createReader( + SingleSplitReaderContext readerContext) { + return new QdrantSourceReader(qdrantParameters, readerContext, catalogTable); + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java new file mode 100644 index 00000000000..0639fc07e24 --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java @@ -0,0 +1,63 @@ +/* + * 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.qdrant.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; + +@AutoService(Factory.class) +public class QdrantSourceFactory implements TableSourceFactory { + @Override + public String factoryIdentifier() { + return QdrantConfig.CONNECTOR_IDENTITY; + } + + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> (SeaTunnelSource) new QdrantSource(context.getOptions()); + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(QdrantConfig.COLLECTION_NAME, TableSchemaOptions.SCHEMA) + .optional( + QdrantConfig.HOST, + QdrantConfig.PORT, + QdrantConfig.API_KEY, + QdrantConfig.USE_TLS) + .build(); + } + + @Override + public Class getSourceClass() { + return QdrantSource.class; + } +} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java new file mode 100644 index 00000000000..2c371631295 --- /dev/null +++ b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java @@ -0,0 +1,181 @@ +/* + * 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.qdrant.source; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.utils.BufferUtils; +import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitReader; +import org.apache.seatunnel.connectors.seatunnel.common.source.SingleSplitReaderContext; +import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; +import org.apache.seatunnel.connectors.seatunnel.qdrant.exception.QdrantConnectorException; + +import io.qdrant.client.QdrantClient; +import io.qdrant.client.WithVectorsSelectorFactory; +import io.qdrant.client.grpc.JsonWithInt; +import io.qdrant.client.grpc.Points; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static io.qdrant.client.WithPayloadSelectorFactory.enable; +import static org.apache.seatunnel.api.table.catalog.PrimaryKey.isPrimaryKeyField; + +public class QdrantSourceReader extends AbstractSingleSplitReader { + private final QdrantParameters qdrantParameters; + private final SingleSplitReaderContext context; + private final TableSchema tableSchema; + private final TablePath tablePath; + private QdrantClient qdrantClient; + + public QdrantSourceReader( + QdrantParameters qdrantParameters, + SingleSplitReaderContext context, + CatalogTable catalogTable) { + this.qdrantParameters = qdrantParameters; + this.context = context; + this.tableSchema = catalogTable.getTableSchema(); + this.tablePath = catalogTable.getTablePath(); + } + + @Override + public void open() throws Exception { + qdrantClient = qdrantParameters.buildQdrantClient(); + qdrantClient.healthCheckAsync().get(); + } + + @Override + public void close() { + if (Objects.nonNull(qdrantClient)) { + qdrantClient.close(); + } + } + + @Override + public void internalPollNext(Collector output) throws Exception { + int SCROLL_SIZE = 64; + Points.ScrollPoints request = + Points.ScrollPoints.newBuilder() + .setCollectionName(qdrantParameters.getCollectionName()) + .setLimit(SCROLL_SIZE) + .setWithPayload(enable(true)) + .setWithVectors(WithVectorsSelectorFactory.enable(true)) + .build(); + + while (true) { + Points.ScrollResponse response = qdrantClient.scrollAsync(request).get(); + List points = response.getResultList(); + + for (Points.RetrievedPoint point : points) { + SeaTunnelRow seaTunnelRow = convertToSeaTunnelRow(point); + output.collect(seaTunnelRow); + } + + Points.PointId offset = response.getNextPageOffset(); + + if (!offset.hasNum() && !offset.hasUuid()) break; + + request = request.toBuilder().setOffset(offset).build(); + } + + context.signalNoMoreElement(); + } + + private SeaTunnelRow convertToSeaTunnelRow(Points.RetrievedPoint point) { + SeaTunnelRowType typeInfo = tableSchema.toPhysicalRowDataType(); + PrimaryKey primaryKey = tableSchema.getPrimaryKey(); + Map payloadMap = point.getPayloadMap(); + Points.Vectors vectors = point.getVectors(); + Map vectorsMap = new HashMap<>(); + String DEFAULT_VECTOR_KEY = "default_vector"; + + if (vectors.hasVector()) { + vectorsMap.put(DEFAULT_VECTOR_KEY, vectors.getVector()); + } else if (vectors.hasVectors()) { + vectorsMap = vectors.getVectors().getVectorsMap(); + } + Object[] fields = new Object[typeInfo.getTotalFields()]; + String[] fieldNames = typeInfo.getFieldNames(); + for (int fieldIndex = 0; fieldIndex < typeInfo.getTotalFields(); fieldIndex++) { + SeaTunnelDataType seaTunnelDataType = typeInfo.getFieldType(fieldIndex); + String fieldName = fieldNames[fieldIndex]; + + if (isPrimaryKeyField(primaryKey, fieldName)) { + Points.PointId id = point.getId(); + if (id.hasNum()) { + fields[fieldIndex] = id.getNum(); + } else if (id.hasUuid()) { + fields[fieldIndex] = id.getUuid(); + } + continue; + } + JsonWithInt.Value value = payloadMap.get(fieldName); + Points.Vector vector = vectorsMap.get(fieldName); + switch (seaTunnelDataType.getSqlType()) { + case NULL: + fields[fieldIndex] = null; + break; + case STRING: + fields[fieldIndex] = value.getStringValue(); + break; + case BOOLEAN: + fields[fieldIndex] = value.getBoolValue(); + break; + case TINYINT: + case SMALLINT: + case INT: + case BIGINT: + fields[fieldIndex] = value.getIntegerValue(); + break; + case FLOAT: + case DECIMAL: + case DOUBLE: + fields[fieldIndex] = value.getDoubleValue(); + break; + case BINARY_VECTOR: + case FLOAT_VECTOR: + case FLOAT16_VECTOR: + case BFLOAT16_VECTOR: + List list = vector.getDataList(); + Float[] vectorArray = new Float[list.size()]; + list.toArray(vectorArray); + fields[fieldIndex] = BufferUtils.toByteBuffer(vectorArray); + break; + default: + throw new QdrantConnectorException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + "Unexpected value: " + seaTunnelDataType.getSqlType().name()); + } + } + + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(fields); + seaTunnelRow.setTableId(tablePath.getFullName()); + seaTunnelRow.setRowKind(RowKind.INSERT); + return seaTunnelRow; + } +} diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java index 82ae2728d67..3f5c862cadf 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java @@ -189,11 +189,16 @@ public void close() { protected void setupQueue() throws IOException { if (config.getQueueName() != null) { - declareQueueDefaults(channel, config.getQueueName()); + declareQueueDefaults(channel, config); } } - private void declareQueueDefaults(Channel channel, String queueName) throws IOException { - channel.queueDeclare(queueName, true, false, false, null); + private void declareQueueDefaults(Channel channel, RabbitmqConfig config) throws IOException { + channel.queueDeclare( + config.getQueueName(), + config.getDurable(), + config.getExclusive(), + config.getAutoDelete(), + null); } } diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java index e8e2ce55c30..8475817457a 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java @@ -53,6 +53,9 @@ public class RabbitmqConfig implements Serializable { private Integer prefetchCount; private long deliveryTimeout; private String queueName; + private Boolean durable; + private Boolean exclusive; + private Boolean autoDelete; private String routingKey; private boolean logFailuresOnly = false; private String exchange = ""; @@ -195,6 +198,30 @@ public class RabbitmqConfig implements Serializable { "Whether the messages received are supplied with a unique" + "id to deduplicate messages (in case of failed acknowledgments)."); + public static final Option DURABLE = + Options.key("durable") + .booleanType() + .defaultValue(true) + .withDescription( + "true: The queue will survive a server restart." + + "false: The queue will be deleted on server restart."); + + public static final Option EXCLUSIVE = + Options.key("exclusive") + .booleanType() + .defaultValue(false) + .withDescription( + "true: The queue is used only by the current connection and will be deleted when the connection closes." + + "false: The queue can be used by multiple connections."); + + public static final Option AUTO_DELETE = + Options.key("auto_delete") + .booleanType() + .defaultValue(false) + .withDescription( + "true: The queue will be deleted automatically when the last consumer unsubscribes." + + "false: The queue will not be automatically deleted."); + private void parseSinkOptionProperties(Config pluginConfig) { if (CheckConfigUtil.isValidParam(pluginConfig, RABBITMQ_CONFIG.key())) { pluginConfig @@ -259,6 +286,15 @@ public RabbitmqConfig(Config config) { if (config.hasPath(USE_CORRELATION_ID.key())) { this.usesCorrelationId = config.getBoolean(USE_CORRELATION_ID.key()); } + if (config.hasPath(DURABLE.key())) { + this.durable = config.getBoolean(DURABLE.key()); + } + if (config.hasPath(EXCLUSIVE.key())) { + this.exclusive = config.getBoolean(EXCLUSIVE.key()); + } + if (config.hasPath(AUTO_DELETE.key())) { + this.autoDelete = config.getBoolean(AUTO_DELETE.key()); + } parseSinkOptionProperties(config); } diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java index 7b2aa6bae67..be121818932 100644 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java @@ -19,11 +19,14 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; + import lombok.AllArgsConstructor; import lombok.Data; import java.io.Serializable; import java.util.List; +import java.util.Map; import static org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreConfig.BATCH_SIZE; @@ -45,6 +48,8 @@ public class TablestoreOptions implements Serializable { public int batchSize = Integer.parseInt(BATCH_SIZE.defaultValue()); + public TablestoreOptions() {} + public TablestoreOptions(Config config) { this.endpoint = config.getString(TablestoreConfig.END_POINT.key()); this.instanceName = config.getString(TablestoreConfig.INSTANCE_NAME.key()); @@ -57,4 +62,18 @@ public TablestoreOptions(Config config) { this.batchSize = config.getInt(BATCH_SIZE.key()); } } + + public static TablestoreOptions of(ReadonlyConfig config) { + Map map = config.getSourceMap(); + TablestoreOptions tablestoreOptions = new TablestoreOptions(); + tablestoreOptions.setEndpoint(config.get(TablestoreConfig.END_POINT)); + tablestoreOptions.setInstanceName(config.get(TablestoreConfig.INSTANCE_NAME)); + tablestoreOptions.setAccessKeyId(config.get(TablestoreConfig.ACCESS_KEY_ID)); + tablestoreOptions.setAccessKeySecret(config.get(TablestoreConfig.ACCESS_KEY_SECRET)); + tablestoreOptions.setTable(config.get(TablestoreConfig.TABLE)); + List keys = (List) map.get(TablestoreConfig.PRIMARY_KEYS.key()); + + tablestoreOptions.setPrimaryKeys(keys); + return tablestoreOptions; + } } diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java new file mode 100644 index 00000000000..9bdb060a49d --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.tablestore.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import com.alicloud.openservices.tablestore.model.StreamRecord; + +import java.util.ArrayList; +import java.util.List; + +public class DefaultSeaTunnelRowDeserializer implements SeaTunnelRowDeserializer { + + @Override + public SeaTunnelRow deserialize(StreamRecord r) { + List fields = new ArrayList<>(); + r.getColumns() + .forEach( + k -> { + fields.add(k.getColumn().getValue()); + }); + return new SeaTunnelRow(fields.toArray()); + } +} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java new file mode 100644 index 00000000000..44a2560693f --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.tablestore.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import com.alicloud.openservices.tablestore.model.StreamRecord; + +public interface SeaTunnelRowDeserializer { + + SeaTunnelRow deserialize(StreamRecord streamRecord); +} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java new file mode 100644 index 00000000000..85c0062ed32 --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.tablestore.source; + +import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceReader.Context; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.SupportColumnProjection; +import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.constants.JobMode; +import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions; + +import lombok.extern.slf4j.Slf4j; + +import java.util.List; + +@Slf4j +public class TableStoreDBSource + implements SeaTunnelSource, + SupportParallelism, + SupportColumnProjection { + + private TablestoreOptions tablestoreOptions; + private SeaTunnelRowType typeInfo; + private JobContext jobContext; + + @Override + public String getPluginName() { + return "Tablestore"; + } + + @Override + public List getProducedCatalogTables() { + return SeaTunnelSource.super.getProducedCatalogTables(); + } + + public TableStoreDBSource(ReadonlyConfig config) { + this.tablestoreOptions = TablestoreOptions.of(config); + CatalogTableUtil.buildWithConfig(config); + this.typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); + } + + @Override + public Boundedness getBoundedness() { + return JobMode.BATCH.equals(jobContext.getJobMode()) + ? Boundedness.BOUNDED + : Boundedness.UNBOUNDED; + } + + @Override + public SourceReader createReader(Context readerContext) + throws Exception { + return new TableStoreDBSourceReader(readerContext, tablestoreOptions, typeInfo); + } + + @Override + public SourceSplitEnumerator createEnumerator( + org.apache.seatunnel.api.source.SourceSplitEnumerator.Context + enumeratorContext) + throws Exception { + return new TableStoreDBSourceSplitEnumerator(enumeratorContext, tablestoreOptions); + } + + @Override + public SourceSplitEnumerator + restoreEnumerator( + org.apache.seatunnel.api.source.SourceSplitEnumerator.Context< + TableStoreDBSourceSplit> + enumeratorContext, + TableStoreDBSourceState checkpointState) + throws Exception { + return new TableStoreDBSourceSplitEnumerator( + enumeratorContext, tablestoreOptions, checkpointState); + } + + @Override + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; + } +} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java new file mode 100644 index 00000000000..eefd4aae031 --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java @@ -0,0 +1,175 @@ +package org.apache.seatunnel.connectors.seatunnel.tablestore.source; +/* + * 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.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions; + +import com.alicloud.openservices.tablestore.SyncClient; +import com.alicloud.openservices.tablestore.TunnelClient; +import com.alicloud.openservices.tablestore.model.tunnel.CreateTunnelRequest; +import com.alicloud.openservices.tablestore.model.tunnel.CreateTunnelResponse; +import com.alicloud.openservices.tablestore.model.tunnel.DeleteTunnelRequest; +import com.alicloud.openservices.tablestore.model.tunnel.DeleteTunnelResponse; +import com.alicloud.openservices.tablestore.model.tunnel.DescribeTunnelRequest; +import com.alicloud.openservices.tablestore.model.tunnel.DescribeTunnelResponse; +import com.alicloud.openservices.tablestore.model.tunnel.TunnelType; +import com.alicloud.openservices.tablestore.tunnel.worker.TunnelWorker; +import com.alicloud.openservices.tablestore.tunnel.worker.TunnelWorkerConfig; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedDeque; + +@Slf4j +public class TableStoreDBSourceReader + implements SourceReader { + + protected SourceReader.Context context; + protected TablestoreOptions tablestoreOptions; + protected SeaTunnelRowType seaTunnelRowType; + Queue pendingSplits = new ConcurrentLinkedDeque<>(); + private SyncClient client; + private volatile boolean noMoreSplit; + private TunnelClient tunnelClient; + + public TableStoreDBSourceReader( + SourceReader.Context context, + TablestoreOptions options, + SeaTunnelRowType seaTunnelRowType) { + + this.context = context; + this.tablestoreOptions = options; + this.seaTunnelRowType = seaTunnelRowType; + } + + @Override + public void open() throws Exception { + client = + new SyncClient( + tablestoreOptions.getEndpoint(), + tablestoreOptions.getAccessKeyId(), + tablestoreOptions.getAccessKeySecret(), + tablestoreOptions.getInstanceName()); + tunnelClient = + new TunnelClient( + tablestoreOptions.getEndpoint(), + tablestoreOptions.getAccessKeyId(), + tablestoreOptions.getAccessKeySecret(), + tablestoreOptions.getInstanceName()); + } + + @Override + public void close() throws IOException { + tunnelClient.shutdown(); + client.shutdown(); + } + + @Override + public void pollNext(Collector output) throws Exception { + synchronized (output.getCheckpointLock()) { + TableStoreDBSourceSplit split = pendingSplits.poll(); + if (Objects.nonNull(split)) { + read(split, output); + } + /*if (split == null) { + log.info( + "TableStore Source Reader [{}] waiting for splits", + context.getIndexOfSubtask()); + }*/ + if (noMoreSplit) { + // signal to the source that we have reached the end of the data. + log.info("Closed the bounded tablestore source"); + context.signalNoMoreElement(); + Thread.sleep(2000L); + } else { + Thread.sleep(1000L); + } + } + } + + private void read(TableStoreDBSourceSplit split, Collector output) { + String tunnelId = getTunel(split); + TableStoreProcessor processor = + new TableStoreProcessor(split.getTableName(), split.getPrimaryKey(), output); + TunnelWorkerConfig workerConfig = new TunnelWorkerConfig(processor); + TunnelWorker worker = new TunnelWorker(tunnelId, tunnelClient, workerConfig); + try { + worker.connectAndWorking(); + } catch (Exception e) { + log.error("Start OTS tunnel failed.", e); + worker.shutdown(); + } + } + + public String getTunel(TableStoreDBSourceSplit split) { + deleteTunel(split); + String tunnelId = null; + String tunnelName = split.getTableName() + "_migration2aws_tunnel4" + split.getSplitId(); + + try { + DescribeTunnelRequest drequest = new DescribeTunnelRequest("test", tunnelName); + DescribeTunnelResponse dresp = tunnelClient.describeTunnel(drequest); + tunnelId = dresp.getTunnelInfo().getTunnelId(); + } catch (Exception be) { + CreateTunnelRequest crequest = + new CreateTunnelRequest( + split.getTableName(), tunnelName, TunnelType.valueOf("BaseAndStream")); + CreateTunnelResponse cresp = tunnelClient.createTunnel(crequest); + tunnelId = cresp.getTunnelId(); + } + log.info("Tunnel found, Id: " + tunnelId); + return tunnelId; + } + + public void deleteTunel(TableStoreDBSourceSplit split) { + String tunnelName = split.getTableName() + "_migration2aws_tunnel4" + split.getSplitId(); + try { + DeleteTunnelRequest drequest = + new DeleteTunnelRequest(split.getTableName(), tunnelName); + DeleteTunnelResponse dresp = tunnelClient.deleteTunnel(drequest); + log.info("Tunnel has been deleted: " + dresp.toString()); + } catch (Exception be) { + log.warn("Tunnel deletion failed due to not found: " + tunnelName); + } + } + + @Override + public List snapshotState(long checkpointId) throws Exception { + return new ArrayList<>(pendingSplits); + } + + @Override + public void addSplits(List splits) { + this.pendingSplits.addAll(splits); + } + + @Override + public void handleNoMoreSplits() { + log.info("Reader [{}] received noMoreSplit event.", context.getIndexOfSubtask()); + noMoreSplit = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java new file mode 100644 index 00000000000..24328b0a6f9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.tablestore.source; + +import org.apache.seatunnel.api.source.SourceSplit; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +@AllArgsConstructor +@Getter +@Setter +public class TableStoreDBSourceSplit implements SourceSplit { + + private Integer splitId; + private String tableName; + private String primaryKey; + + @Override + public String splitId() { + return splitId.toString(); + } +} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java new file mode 100644 index 00000000000..3dd58b7e69b --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java @@ -0,0 +1,166 @@ +/* + * 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.tablestore.source; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +@Slf4j +public class TableStoreDBSourceSplitEnumerator + implements SourceSplitEnumerator { + + private final SourceSplitEnumerator.Context enumeratorContext; + private final Map> pendingSplits; + private final TablestoreOptions tablestoreOptions; + + private final Object stateLock = new Object(); + private volatile boolean shouldEnumerate; + + /** + * @param enumeratorContext + * @param tablestoreOptions + */ + public TableStoreDBSourceSplitEnumerator( + Context enumeratorContext, + TablestoreOptions tablestoreOptions) { + this(enumeratorContext, tablestoreOptions, null); + } + + public TableStoreDBSourceSplitEnumerator( + Context enumeratorContext, + TablestoreOptions tablestoreOptions, + TableStoreDBSourceState sourceState) { + this.enumeratorContext = enumeratorContext; + this.tablestoreOptions = tablestoreOptions; + this.pendingSplits = new HashMap<>(); + this.shouldEnumerate = sourceState == null; + if (sourceState != null) { + this.shouldEnumerate = sourceState.isShouldEnumerate(); + this.pendingSplits.putAll(sourceState.getPendingSplits()); + } + } + + @Override + public void open() {} + + @Override + public void run() throws Exception { + Set readers = enumeratorContext.registeredReaders(); + if (shouldEnumerate) { + Set newSplits = getTableStoreDBSourceSplit(); + synchronized (stateLock) { + addPendingSplit(newSplits); + shouldEnumerate = false; + } + assignSplit(readers); + } + } + + private void assignSplit(Set readers) { + for (int reader : readers) { + List assignmentForReader = pendingSplits.remove(reader); + if (assignmentForReader != null && !assignmentForReader.isEmpty()) { + log.info("Assign splits {} to reader {}", assignmentForReader, reader); + try { + enumeratorContext.assignSplit(reader, assignmentForReader); + } catch (Exception e) { + log.error( + "Failed to assign splits {} to reader {}", + assignmentForReader, + reader, + e); + pendingSplits.put(reader, assignmentForReader); + } + } + } + } + + private Set getTableStoreDBSourceSplit() { + + Set allSplit = new HashSet<>(); + String tables = tablestoreOptions.getTable(); + String[] tableArr = tables.split(","); + for (int i = 0; i < tableArr.length; i++) { + allSplit.add( + new TableStoreDBSourceSplit( + i, tableArr[i], tablestoreOptions.getPrimaryKeys().get(i))); + } + return allSplit; + } + + private void addPendingSplit(Collection splits) { + int readerCount = enumeratorContext.currentParallelism(); + for (TableStoreDBSourceSplit split : splits) { + int ownerReader = split.getSplitId() % readerCount; + pendingSplits.computeIfAbsent(ownerReader, k -> new ArrayList<>()).add(split); + } + } + + @Override + public void close() throws IOException { + // TODO Auto-generated method stub + throw new UnsupportedOperationException("Unimplemented method 'close'"); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + log.debug("Add back splits {} to tablestore.", splits); + if (!splits.isEmpty()) { + addPendingSplit(splits); + assignSplit(Collections.singleton(subtaskId)); + enumeratorContext.signalNoMoreSplits(subtaskId); + } + } + + @Override + public int currentUnassignedSplitSize() { + return pendingSplits.size(); + } + + @Override + public void handleSplitRequest(int subtaskId) {} + + @Override + public void registerReader(int subtaskId) { + log.debug("Register reader {} to TablestoreSplitEnumerator.", subtaskId); + if (!pendingSplits.isEmpty()) { + assignSplit(Collections.singleton(subtaskId)); + } + } + + @Override + public TableStoreDBSourceState snapshotState(long checkpointId) throws Exception { + synchronized (stateLock) { + return new TableStoreDBSourceState(shouldEnumerate, pendingSplits); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java similarity index 74% rename from seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java rename to seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java index 6c0a5667da7..05a73a63101 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java @@ -14,22 +14,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -package org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source; +package org.apache.seatunnel.connectors.seatunnel.tablestore.source; import lombok.AllArgsConstructor; -import lombok.Data; +import lombok.Getter; +import lombok.Setter; import java.io.Serializable; import java.util.List; import java.util.Map; -@Data +@Getter +@Setter @AllArgsConstructor -public class SourceIndexInfo implements Serializable { - private String index; - private List source; - private Map query; - private String scrollTime; - private int scrollSize; +public class TableStoreDBSourceState implements Serializable { + + private boolean shouldEnumerate; + private Map> pendingSplits; } diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java new file mode 100644 index 00000000000..f93ae4bfe32 --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.connectors.seatunnel.tablestore.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreConfig; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; + +@AutoService(Factory.class) +public class TableStoreDbSourceFactory implements TableSourceFactory { + + @Override + public String factoryIdentifier() { + return "Tablestore"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + TablestoreConfig.END_POINT, + TablestoreConfig.INSTANCE_NAME, + TablestoreConfig.ACCESS_KEY_ID, + TablestoreConfig.ACCESS_KEY_SECRET, + TablestoreConfig.TABLE, + TablestoreConfig.PRIMARY_KEYS) + .build(); + } + + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> + (SeaTunnelSource) new TableStoreDBSource(context.getOptions()); + } + + @Override + public Class getSourceClass() { + return TableStoreDBSource.class; + } +} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java new file mode 100644 index 00000000000..ba5334a85eb --- /dev/null +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java @@ -0,0 +1,95 @@ +/* + * 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.tablestore.source; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.tablestore.serialize.SeaTunnelRowDeserializer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alicloud.openservices.tablestore.model.StreamRecord; +import com.alicloud.openservices.tablestore.tunnel.worker.IChannelProcessor; +import com.alicloud.openservices.tablestore.tunnel.worker.ProcessRecordsInput; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class TableStoreProcessor implements IChannelProcessor { + private String tableName = null; + private String primaryKey = null; + private Collector output = null; + protected SeaTunnelRowDeserializer seaTunnelRowDeserializer; + private static final Logger log = LoggerFactory.getLogger(TableStoreProcessor.class); + + public TableStoreProcessor( + String tableName, String primaryKey, Collector output) { + this.tableName = tableName; + this.primaryKey = primaryKey; + this.output = output; + } + + @Override + public void process(ProcessRecordsInput input) { + log.info("Default record processor, would print records count"); + + log.info( + String.format( + "Process %d records, NextToken: %s", + input.getRecords().size(), input.getNextToken())); + + for (StreamRecord r : input.getRecords()) { + try { + List fields = new ArrayList<>(); + Arrays.stream(r.getPrimaryKey().getPrimaryKeyColumns()) + .forEach( + k -> { + fields.add(k.getValue().toString()); + }); + r.getColumns() + .forEach( + k -> { + fields.add(k.getColumn().getValue().toString()); + }); + SeaTunnelRow row = new SeaTunnelRow(fields.toArray()); + row.setTableId(tableName); + switch ((r.getRecordType())) { + case PUT: + row.setRowKind(RowKind.INSERT); + break; + case UPDATE: + row.setRowKind(RowKind.UPDATE_AFTER); + break; + case DELETE: + row.setRowKind(RowKind.DELETE); + break; + } + output.collect(row); + } catch (Exception e) { + log.error("send to target failed with record: " + r.toString(), e); + } + } + } + + @Override + public void shutdown() { + log.info("process shutdown du to finished for table: " + tableName); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/pom.xml b/seatunnel-connectors-v2/connector-typesense/pom.xml new file mode 100644 index 00000000000..57a8682cb44 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/pom.xml @@ -0,0 +1,90 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connectors-v2 + ${revision} + + + connector-typesense + SeaTunnel : Connectors V2 : Typesense + + + 0.8.1 + + + + + com.fasterxml.jackson.core + jackson-databind + 2.14.1 + + + + org.typesense + typesense-java + 0.8.1 + + + org.apache.seatunnel + connector-common + ${project.version} + compile + + + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + shade + + package + + + + okhttp3 + shaded.okhttp3 + + + okio + shaded.okio + + + false + + + + + + + + diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java new file mode 100644 index 00000000000..fd73c247497 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.ConfigUtil; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.InfoPreviewResult; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PreviewResult; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkNotNull; + +@Slf4j +public class TypesenseCatalog implements Catalog { + + private final String catalogName; + private final String defaultDatabase; + + private final ReadonlyConfig config; + private TypesenseClient typesenseClient; + + public TypesenseCatalog(String catalogName, String defaultDatabase, ReadonlyConfig config) { + this.catalogName = checkNotNull(catalogName, "catalogName cannot be null"); + this.defaultDatabase = defaultDatabase; + this.config = checkNotNull(config, "Typesense Config cannot be null"); + } + + @Override + public void open() throws CatalogException { + typesenseClient = TypesenseClient.createInstance(config); + } + + @Override + public void close() throws CatalogException { + // Nothing + } + + @Override + public String name() { + return catalogName; + } + + @Override + public String getDefaultDatabase() throws CatalogException { + return defaultDatabase; + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + return typesenseClient.collectionExists(databaseName); + } + + @Override + public List listDatabases() throws CatalogException { + return typesenseClient.collectionList(); + } + + @Override + public List listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(catalogName, databaseName); + } + return Arrays.asList(databaseName); + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + checkNotNull(tablePath); + return databaseExists(tablePath.getTableName()); + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + checkNotNull(tablePath, "tablePath cannot be null"); + TableSchema.Builder builder = TableSchema.builder(); + Map> fieldTypeMapping = + typesenseClient.getFieldTypeMapping(tablePath.getTableName()); + buildColumnsWithErrorCheck( + tablePath, + builder, + fieldTypeMapping.entrySet().iterator(), + nameAndType -> { + return PhysicalColumn.of( + nameAndType.getKey(), + TypesenseTypeConverter.INSTANCE + .convert(nameAndType.getValue()) + .getDataType(), + (Long) null, + true, + null, + null); + }); + + return CatalogTable.of( + TableIdentifier.of( + catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), + builder.build(), + buildTableOptions(tablePath), + Collections.emptyList(), + ""); + } + + private Map buildTableOptions(TablePath tablePath) { + Map options = new HashMap<>(); + options.put("connector", "typesense"); + options.put("config", ConfigUtil.convertToJsonString(tablePath)); + return options; + } + + @Override + public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + typesenseClient.createCollection(tablePath.getTableName()); + } + + @Override + public void dropTable(TablePath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath); + if (!tableExists(tablePath) && !ignoreIfNotExists) { + throw new TableNotExistException(catalogName, tablePath); + } + try { + typesenseClient.dropCollection(tablePath.getTableName()); + } catch (Exception ex) { + throw new CatalogException( + String.format( + "Failed to drop table %s in catalog %s", + tablePath.getTableName(), catalogName), + ex); + } + } + + @Override + public void createDatabase(TablePath tablePath, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + createTable(tablePath, null, ignoreIfExists); + } + + @Override + public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + dropTable(tablePath, ignoreIfNotExists); + } + + @Override + public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) { + typesenseClient.truncateCollectionData(tablePath.getTableName()); + } + + @Override + public boolean isExistsData(TablePath tablePath) { + return typesenseClient.collectionDocNum(tablePath.getTableName()) > 0; + } + + @Override + public PreviewResult previewAction( + ActionType actionType, TablePath tablePath, Optional catalogTable) { + if (actionType == ActionType.CREATE_TABLE) { + return new InfoPreviewResult("create collection " + tablePath.getTableName()); + } else if (actionType == ActionType.DROP_TABLE) { + return new InfoPreviewResult("delete collection " + tablePath.getTableName()); + } else if (actionType == ActionType.TRUNCATE_TABLE) { + return new InfoPreviewResult( + "delete and create collection " + tablePath.getTableName()); + } else if (actionType == ActionType.CREATE_DATABASE) { + return new InfoPreviewResult("create collection " + tablePath.getTableName()); + } else if (actionType == ActionType.DROP_DATABASE) { + return new InfoPreviewResult("delete collection " + tablePath.getTableName()); + } else { + throw new UnsupportedOperationException("Unsupported action type: " + actionType); + } + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java new file mode 100644 index 00000000000..0b619ad3052 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class TypesenseCatalogFactory implements CatalogFactory { + + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + return new TypesenseCatalog(catalogName, "", options); + } + + @Override + public String factoryIdentifier() { + return "Typesense"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().build(); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java new file mode 100644 index 00000000000..c4cb862e29b --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.catalog; + +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.converter.BasicTypeConverter; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.api.table.converter.TypeConverter; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType; + +import com.google.auto.service.AutoService; + +import java.util.Map; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType.INT32; +import static org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType.INT64; + +@AutoService(TypeConverter.class) +public class TypesenseTypeConverter implements BasicTypeConverter> { + public static final TypesenseTypeConverter INSTANCE = new TypesenseTypeConverter(); + + @Override + public String identifier() { + return "Typesense"; + } + + @Override + public Column convert(BasicTypeDefine typeDefine) { + PhysicalColumn.PhysicalColumnBuilder builder = + PhysicalColumn.builder() + .name(typeDefine.getName()) + .sourceType(typeDefine.getColumnType()) + .nullable(typeDefine.isNullable()) + .defaultValue(typeDefine.getDefaultValue()) + .comment(typeDefine.getComment()); + String type = typeDefine.getDataType().toLowerCase(); + switch (type) { + case INT32: + builder.dataType(BasicType.INT_TYPE); + break; + case INT64: + builder.dataType(BasicType.LONG_TYPE); + break; + case TypesenseType.FLOAT: + builder.dataType(BasicType.FLOAT_TYPE); + break; + case TypesenseType.BOOL: + builder.dataType(BasicType.BOOLEAN_TYPE); + break; + case TypesenseType.OBJET: + Map> typeInfo = + (Map) typeDefine.getNativeType().getOptions(); + SeaTunnelRowType object = + new SeaTunnelRowType( + typeInfo.keySet().toArray(new String[0]), + typeInfo.values().stream() + .map(this::convert) + .map(Column::getDataType) + .toArray(SeaTunnelDataType[]::new)); + builder.dataType(object); + break; + case TypesenseType.STRING: + case TypesenseType.IMAGE: + default: + builder.dataType(BasicType.STRING_TYPE); + break; + } + return builder.build(); + } + + @Override + public BasicTypeDefine reconvert(Column column) { + throw new UnsupportedOperationException("Unsupported operation"); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java new file mode 100644 index 00000000000..01b848492fb --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.client; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.converter.BasicTypeDefine; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; +import org.apache.seatunnel.connectors.seatunnel.typesense.util.URLParamsConverter; + +import org.apache.commons.lang3.StringUtils; + +import org.typesense.api.Client; +import org.typesense.api.Collections; +import org.typesense.api.Configuration; +import org.typesense.api.FieldTypes; +import org.typesense.model.CollectionResponse; +import org.typesense.model.CollectionSchema; +import org.typesense.model.DeleteDocumentsParameters; +import org.typesense.model.Field; +import org.typesense.model.ImportDocumentsParameters; +import org.typesense.model.SearchParameters; +import org.typesense.model.SearchResult; +import org.typesense.resources.Node; + +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.extern.slf4j.Slf4j; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig.QUERY_BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.CREATE_COLLECTION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.DELETE_COLLECTION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.DROP_COLLECTION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.FIELD_TYPE_MAPPING_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.INSERT_DOC_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_EXISTS_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_LIST_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_NUM_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.TRUNCATE_COLLECTION_ERROR; + +@Slf4j +public class TypesenseClient { + private final Client tsClient; + + TypesenseClient(Client tsClient) { + this.tsClient = tsClient; + } + + public static TypesenseClient createInstance(ReadonlyConfig config) { + List hosts = config.get(TypesenseConnectionConfig.HOSTS); + String protocol = config.get(TypesenseConnectionConfig.protocol); + String apiKey = config.get(TypesenseConnectionConfig.APIKEY); + return createInstance(hosts, apiKey, protocol); + } + + public static TypesenseClient createInstance( + List hosts, String apiKey, String protocol) { + List nodes = new ArrayList<>(); + + hosts.stream() + .map(host -> host.split(":")) + .forEach( + split -> + nodes.add( + new Node( + protocol, + split[0], + StringUtils.isBlank(split[1]) + ? "8018" + : split[1]))); + + Configuration configuration = new Configuration(nodes, Duration.ofSeconds(5), apiKey); + Client client = new Client(configuration); + return new TypesenseClient(client); + } + + public void insert(String collection, List documentList) { + + ImportDocumentsParameters queryParameters = new ImportDocumentsParameters(); + queryParameters.action("upsert"); + String text = ""; + for (String s : documentList) { + text = text + s + "\n"; + } + try { + tsClient.collections(collection).documents().import_(text, queryParameters); + } catch (Exception e) { + log.error(INSERT_DOC_ERROR.getDescription()); + throw new TypesenseConnectorException( + INSERT_DOC_ERROR, INSERT_DOC_ERROR.getDescription()); + } + } + + public SearchResult search(String collection, String query, int offset) throws Exception { + return search(collection, query, offset, QUERY_BATCH_SIZE.defaultValue()); + } + + public SearchResult search(String collection, String query, int offset, int pageSize) + throws Exception { + SearchParameters searchParameters; + if (StringUtils.isNotBlank(query)) { + String jsonQuery = URLParamsConverter.convertParamsToJson(query); + ObjectMapper objectMapper = new ObjectMapper(); + searchParameters = objectMapper.readValue(jsonQuery, SearchParameters.class); + } else { + searchParameters = new SearchParameters().q("*"); + } + log.debug("Typesense query param:{}", searchParameters); + searchParameters.offset(offset); + searchParameters.perPage(pageSize); + SearchResult searchResult = + tsClient.collections(collection).documents().search(searchParameters); + return searchResult; + } + + public boolean collectionExists(String collection) { + try { + Collections collections = tsClient.collections(); + CollectionResponse[] collectionResponses = collections.retrieve(); + for (CollectionResponse collectionRespons : collectionResponses) { + String collectionName = collectionRespons.getName(); + if (collection.equals(collectionName)) { + return true; + } + } + } catch (Exception e) { + log.error(QUERY_COLLECTION_EXISTS_ERROR.getDescription()); + throw new TypesenseConnectorException( + QUERY_COLLECTION_EXISTS_ERROR, QUERY_COLLECTION_EXISTS_ERROR.getDescription()); + } + return false; + } + + public List collectionList() { + try { + Collections collections = tsClient.collections(); + CollectionResponse[] collectionResponses = collections.retrieve(); + List list = new ArrayList<>(); + for (CollectionResponse collectionRespons : collectionResponses) { + String collectionName = collectionRespons.getName(); + list.add(collectionName); + } + return list; + } catch (Exception e) { + log.error(QUERY_COLLECTION_LIST_ERROR.getDescription()); + throw new TypesenseConnectorException( + QUERY_COLLECTION_LIST_ERROR, QUERY_COLLECTION_LIST_ERROR.getDescription()); + } + } + + public Map getField(String collection) { + if (collectionExists(collection)) { + Map fieldMap = new HashMap<>(); + try { + CollectionResponse collectionResponse = tsClient.collections(collection).retrieve(); + List fields = collectionResponse.getFields(); + for (Field field : fields) { + String fieldName = field.getName(); + String type = field.getType(); + fieldMap.put(fieldName, type); + } + } catch (Exception e) { + log.error(FIELD_TYPE_MAPPING_ERROR.getDescription()); + throw new TypesenseConnectorException( + FIELD_TYPE_MAPPING_ERROR, FIELD_TYPE_MAPPING_ERROR.getDescription()); + } + return fieldMap; + } else { + return null; + } + } + + public Map> getFieldTypeMapping(String collection) { + Map> allTypesenseSearchFieldTypeInfoMap = + new HashMap<>(); + try { + CollectionResponse collectionResponse = tsClient.collections(collection).retrieve(); + List fields = collectionResponse.getFields(); + for (Field field : fields) { + String fieldName = field.getName(); + String type = field.getType(); + BasicTypeDefine.BasicTypeDefineBuilder typeDefine = + BasicTypeDefine.builder() + .name(fieldName) + .columnType(type) + .dataType(type) + .nativeType(new TypesenseType(type, new HashMap<>())); + allTypesenseSearchFieldTypeInfoMap.put(fieldName, typeDefine.build()); + } + } catch (Exception e) { + log.error(FIELD_TYPE_MAPPING_ERROR.getDescription()); + throw new TypesenseConnectorException( + FIELD_TYPE_MAPPING_ERROR, FIELD_TYPE_MAPPING_ERROR.getDescription()); + } + return allTypesenseSearchFieldTypeInfoMap; + } + + public boolean createCollection(String collection) { + if (collectionExists(collection)) { + return true; + } + List fields = new ArrayList<>(); + fields.add(new Field().name(".*").type(FieldTypes.AUTO)); + return createCollection(collection, fields); + } + + public boolean createCollection(String collection, List fields) { + CollectionSchema collectionSchema = new CollectionSchema(); + collectionSchema.name(collection).fields(fields).enableNestedFields(true); + try { + tsClient.collections().create(collectionSchema); + return true; + } catch (Exception e) { + log.error(CREATE_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + CREATE_COLLECTION_ERROR, CREATE_COLLECTION_ERROR.getDescription()); + } + } + + public boolean dropCollection(String collection) { + try { + tsClient.collections(collection).delete(); + return true; + } catch (Exception e) { + log.error(DROP_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + DROP_COLLECTION_ERROR, DROP_COLLECTION_ERROR.getDescription()); + } + } + + public boolean truncateCollectionData(String collection) { + DeleteDocumentsParameters deleteDocumentsParameters = new DeleteDocumentsParameters(); + deleteDocumentsParameters.filterBy("id:!=1||id:=1"); + try { + tsClient.collections(collection).documents().delete(deleteDocumentsParameters); + } catch (Exception e) { + log.error(TRUNCATE_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + TRUNCATE_COLLECTION_ERROR, TRUNCATE_COLLECTION_ERROR.getDescription()); + } + return true; + } + + public boolean deleteCollectionData(String collection, String id) { + try { + tsClient.collections(collection).documents(id).delete(); + } catch (Exception e) { + log.error(DELETE_COLLECTION_ERROR.getDescription()); + throw new TypesenseConnectorException( + DELETE_COLLECTION_ERROR, DELETE_COLLECTION_ERROR.getDescription()); + } + return true; + } + + public long collectionDocNum(String collection) { + SearchParameters q = new SearchParameters().q("*"); + try { + SearchResult searchResult = tsClient.collections(collection).documents().search(q); + return searchResult.getFound(); + } catch (Exception e) { + log.error(QUERY_COLLECTION_NUM_ERROR.getDescription()); + throw new TypesenseConnectorException( + QUERY_COLLECTION_NUM_ERROR, QUERY_COLLECTION_NUM_ERROR.getDescription()); + } + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java new file mode 100644 index 00000000000..2ac05a6d312 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.client; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +import java.util.Map; + +@Getter +@AllArgsConstructor +public class TypesenseType { + + public static final String STRING = "string"; + public static final String INT32 = "int32"; + public static final String INT64 = "int64"; + public static final String FLOAT = "float"; + public static final String BOOL = "bool"; + public static final String IMAGE = "image"; + public static final String OBJET = "object"; + private String type; + private Map options; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java new file mode 100644 index 00000000000..9912c6aa2e1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.SchemaSaveMode; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.seatunnel.api.sink.DataSaveMode.APPEND_DATA; +import static org.apache.seatunnel.api.sink.DataSaveMode.DROP_DATA; +import static org.apache.seatunnel.api.sink.DataSaveMode.ERROR_WHEN_DATA_EXISTS; + +public class SinkConfig { + + public static final Option COLLECTION = + Options.key("collection") + .stringType() + .noDefaultValue() + .withDescription("Typesense collection name"); + + public static final Option> PRIMARY_KEYS = + Options.key("primary_keys") + .listType(String.class) + .noDefaultValue() + .withDescription("Primary key fields used to generate the document `id`"); + + public static final Option KEY_DELIMITER = + Options.key("key_delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys (\"_\" by default), e.g., \"$\" would result in document `id` \"KEY1$KEY2$KEY3\"."); + + public static final Option MAX_BATCH_SIZE = + Options.key("max_batch_size") + .intType() + .defaultValue(10) + .withDescription("batch bulk doc max size"); + + public static final Option MAX_RETRY_COUNT = + Options.key("max_retry_count") + .intType() + .defaultValue(3) + .withDescription("one bulk request max try count"); + + public static final Option SCHEMA_SAVE_MODE = + Options.key("schema_save_mode") + .enumType(SchemaSaveMode.class) + .defaultValue(SchemaSaveMode.CREATE_SCHEMA_WHEN_NOT_EXIST) + .withDescription("schema_save_mode"); + + public static final Option DATA_SAVE_MODE = + Options.key("data_save_mode") + .singleChoice( + DataSaveMode.class, + Arrays.asList(DROP_DATA, APPEND_DATA, ERROR_WHEN_DATA_EXISTS)) + .defaultValue(APPEND_DATA) + .withDescription("data_save_mode"); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java new file mode 100644 index 00000000000..a1642331a6c --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +public class SourceConfig { + + public static final Option COLLECTION = + Options.key("collection") + .stringType() + .noDefaultValue() + .withDescription("Typesense collection name"); + + public static final Option QUERY = + Options.key("query") + .stringType() + .noDefaultValue() + .withDescription("Typesense query param"); + + public static final Option QUERY_BATCH_SIZE = + Options.key("batch_size") + .intType() + .defaultValue(100) + .withDescription("Typesense query batch size"); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java new file mode 100644 index 00000000000..2f36dc044c5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import java.util.List; + +public class TypesenseConnectionConfig { + + public static final Option> HOSTS = + Options.key("hosts") + .listType() + .noDefaultValue() + .withDescription( + "Typesense cluster http address, the format is host:port, allowing multiple hosts to be specified. Such as [\"host1:8018\", \"host2:8018\"]"); + + public static final Option APIKEY = + Options.key("api_key") + .stringType() + .noDefaultValue() + .withDescription("Typesense api key"); + + public static final Option protocol = + Options.key("protocol") + .stringType() + .defaultValue("http") + .withDescription("Default is http , for Typesense Cloud use https"); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java new file mode 100644 index 00000000000..9fd4892979f --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.dto; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; + +import lombok.Data; + +@Data +public class CollectionInfo { + + private String collection; + private String type; + private String[] primaryKeys; + private String keyDelimiter; + + public CollectionInfo(String collection, ReadonlyConfig config) { + this.collection = collection; + if (config.getOptional(SinkConfig.PRIMARY_KEYS).isPresent()) { + primaryKeys = config.get(SinkConfig.PRIMARY_KEYS).toArray(new String[0]); + } + keyDelimiter = config.get(SinkConfig.KEY_DELIMITER); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java new file mode 100644 index 00000000000..9e1b55f8cbd --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.dto; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.Serializable; + +@Data +@AllArgsConstructor +public class SourceCollectionInfo implements Serializable { + private String collection; + private String query; + private long found; + private int offset; + private int queryBatchSize; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java new file mode 100644 index 00000000000..8edcf608d55 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; + +public enum TypesenseConnectorErrorCode implements SeaTunnelErrorCode { + QUERY_PARAM_ERROR("TYPESENSE-01", "Query parameter error"), + QUERY_COLLECTION_EXISTS_ERROR("TYPESENSE-02", "Whether the collection stores query exceptions"), + QUERY_COLLECTION_LIST_ERROR("TYPESENSE-03", "Collection list acquisition exception"), + FIELD_TYPE_MAPPING_ERROR("TYPESENSE-04", "Failed to obtain the field"), + CREATE_COLLECTION_ERROR("TYPESENSE-05", "Create collection failed"), + DROP_COLLECTION_ERROR("TYPESENSE-06", "Drop collection failed"), + TRUNCATE_COLLECTION_ERROR("TYPESENSE-07", "Truncate collection failed"), + QUERY_COLLECTION_NUM_ERROR("TYPESENSE-08", "Query collection doc number failed"), + INSERT_DOC_ERROR("TYPESENSE-09", "Insert documents failed"), + DELETE_COLLECTION_ERROR("TYPESENSE-10", "Truncate collection failed"); + private final String code; + private final String description; + + TypesenseConnectorErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return code; + } + + @Override + public String getDescription() { + return description; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java new file mode 100644 index 00000000000..3c1cb1cc167 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class TypesenseConnectorException extends SeaTunnelRuntimeException { + public TypesenseConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } + + public TypesenseConnectorException( + SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { + super(seaTunnelErrorCode, errorMessage, cause); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java new file mode 100644 index 00000000000..5dc56f0bb77 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import lombok.AllArgsConstructor; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +@AllArgsConstructor +public class KeyExtractor implements Function, Serializable { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + @Override + public String apply(SeaTunnelRow row) { + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + String value = fieldFormatters[i].format(row); + builder.append(value); + } + return builder.toString(); + } + + public static Function createKeyExtractor( + SeaTunnelRowType rowType, String[] primaryKeys, String keyDelimiter) { + if (primaryKeys == null) { + return row -> null; + } + + List fieldFormatters = new ArrayList<>(primaryKeys.length); + for (String fieldName : primaryKeys) { + int fieldIndex = rowType.indexOf(fieldName); + SeaTunnelDataType fieldType = rowType.getFieldType(fieldIndex); + FieldFormatter fieldFormatter = createFieldFormatter(fieldIndex, fieldType); + fieldFormatters.add(fieldFormatter); + } + return new KeyExtractor(fieldFormatters.toArray(new FieldFormatter[0]), keyDelimiter); + } + + private static FieldFormatter createFieldFormatter( + int fieldIndex, SeaTunnelDataType fieldType) { + return row -> { + switch (fieldType.getSqlType()) { + case ROW: + case ARRAY: + case MAP: + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Unsupported type: " + fieldType); + case DATE: + LocalDate localDate = (LocalDate) row.getField(fieldIndex); + return localDate.toString(); + case TIME: + LocalTime localTime = (LocalTime) row.getField(fieldIndex); + return localTime.toString(); + case TIMESTAMP: + LocalDateTime localDateTime = (LocalDateTime) row.getField(fieldIndex); + return localDateTime.toString(); + default: + return row.getField(fieldIndex).toString(); + } + }; + } + + private interface FieldFormatter extends Serializable { + String format(SeaTunnelRow row); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java new file mode 100644 index 00000000000..94da50cf09b --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public interface SeaTunnelRowSerializer { + String serializeRow(SeaTunnelRow row); + + String serializeRowForDelete(SeaTunnelRow row); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java new file mode 100644 index 00000000000..07faca4cc92 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonError; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.KeyExtractor; + +import org.apache.commons.lang3.StringUtils; + +import java.time.temporal.Temporal; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +public class TypesenseRowSerializer implements SeaTunnelRowSerializer { + + private final SeaTunnelRowType seaTunnelRowType; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + private final Function keyExtractor; + + public TypesenseRowSerializer( + CollectionInfo collectionInfo, SeaTunnelRowType seaTunnelRowType) { + this.seaTunnelRowType = seaTunnelRowType; + this.keyExtractor = + KeyExtractor.createKeyExtractor( + seaTunnelRowType, + collectionInfo.getPrimaryKeys(), + collectionInfo.getKeyDelimiter()); + } + + @Override + public String serializeRow(SeaTunnelRow row) { + String key = keyExtractor.apply(row); + Map document = toDocumentMap(row, seaTunnelRowType); + if (StringUtils.isNotBlank(key)) { + document.put("id", key); + } + String documentStr; + try { + documentStr = objectMapper.writeValueAsString(document); + } catch (JsonProcessingException e) { + throw CommonError.jsonOperationError("Typesense", "document:" + document.toString(), e); + } + return documentStr; + } + + @Override + public String serializeRowForDelete(SeaTunnelRow row) { + String key = keyExtractor.apply(row); + Map document = toDocumentMap(row, seaTunnelRowType); + String id = document.get("id").toString(); + if (StringUtils.isNotBlank(key)) { + id = key; + } + return id; + } + + private Map toDocumentMap(SeaTunnelRow row, SeaTunnelRowType rowType) { + String[] fieldNames = rowType.getFieldNames(); + Map doc = new HashMap<>(fieldNames.length); + Object[] fields = row.getFields(); + for (int i = 0; i < fieldNames.length; i++) { + Object value = fields[i]; + if (value == null) { + } else if (value instanceof SeaTunnelRow) { + doc.put( + fieldNames[i], + toDocumentMap( + (SeaTunnelRow) value, (SeaTunnelRowType) rowType.getFieldType(i))); + } else { + doc.put(fieldNames[i], convertValue(value)); + } + } + return doc; + } + + private Object convertValue(Object value) { + if (value instanceof Temporal) { + // jackson not support jdk8 new time api + return value.toString(); + } else if (value instanceof Map) { + for (Map.Entry entry : ((Map) value).entrySet()) { + ((Map) value).put(entry.getKey(), convertValue(entry.getValue())); + } + return value; + } else if (value instanceof List) { + for (int i = 0; i < ((List) value).size(); i++) { + ((List) value).set(i, convertValue(((List) value).get(i))); + } + return value; + } else { + return value; + } + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java new file mode 100644 index 00000000000..d13901b7081 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.collection; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public interface CollectionSerializer { + String serialize(SeaTunnelRow row); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java new file mode 100644 index 00000000000..a2b0da248e1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.collection; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public class FixedValueCollectionSerializer implements CollectionSerializer { + + private final String index; + + public FixedValueCollectionSerializer(String index) { + this.index = index; + } + + @Override + public String serialize(SeaTunnelRow row) { + return index; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java new file mode 100644 index 00000000000..762506d4980 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.seatunnel.api.table.type.BasicType.BOOLEAN_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.BYTE_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.DOUBLE_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.SHORT_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.VOID_TYPE; + +public class DefaultSeaTunnelRowDeserializer implements SeaTunnelRowDeserializer { + + private final SeaTunnelRowType rowTypeInfo; + + private final ObjectMapper mapper = new ObjectMapper(); + + private final String nullDefault = "null"; + + private final Map dateTimeFormatterMap = + new HashMap() { + { + put("yyyy-MM-dd HH".length(), DateTimeFormatter.ofPattern("yyyy-MM-dd HH")); + put( + "yyyy-MM-dd HH:mm".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm")); + put( + "yyyyMMdd HH:mm:ss".length(), + DateTimeFormatter.ofPattern("yyyyMMdd HH:mm:ss")); + put( + "yyyy-MM-dd HH:mm:ss".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss")); + put( + "yyyy-MM-dd HH:mm:ss.S".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.S")); + put( + "yyyy-MM-dd HH:mm:ss.SS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SS")); + put( + "yyyy-MM-dd HH:mm:ss.SSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")); + put( + "yyyy-MM-dd HH:mm:ss.SSSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSS")); + put( + "yyyy-MM-dd HH:mm:ss.SSSSSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSS")); + put( + "yyyy-MM-dd HH:mm:ss.SSSSSSSSS".length(), + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS")); + } + }; + + public DefaultSeaTunnelRowDeserializer(SeaTunnelRowType rowTypeInfo) { + this.rowTypeInfo = rowTypeInfo; + } + + @Override + public SeaTunnelRow deserialize(TypesenseRecord rowRecord) { + return convert(rowRecord); + } + + SeaTunnelRow convert(TypesenseRecord rowRecord) { + Object[] seaTunnelFields = new Object[rowTypeInfo.getTotalFields()]; + String fieldName = null; + Object value = null; + SeaTunnelDataType seaTunnelDataType = null; + Map doc = rowRecord.getDoc(); + try { + for (int i = 0; i < rowTypeInfo.getTotalFields(); i++) { + fieldName = rowTypeInfo.getFieldName(i); + value = doc.get(fieldName); // 字段值 + if (value != null) { + seaTunnelDataType = + rowTypeInfo.getFieldType(i); // seaTunnelDataType 为SeaTunnel类型 + seaTunnelFields[i] = convertValue(seaTunnelDataType, value); + } + } + } catch (Exception ex) { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + String.format( + "error fieldName=%s,fieldValue=%s,seaTunnelDataType=%s,rowRecord=%s", + fieldName, value, seaTunnelDataType, JsonUtils.toJsonString(rowRecord)), + ex); + } + return new SeaTunnelRow(seaTunnelFields); + } + + Object convertValue(SeaTunnelDataType fieldType, Object fieldValue) + throws JsonProcessingException { + if (STRING_TYPE.equals(fieldType)) { + return fieldValue.toString(); + } else { + if (nullDefault.equals(fieldValue.toString())) { + return null; + } + if (BOOLEAN_TYPE.equals(fieldType)) { + return Boolean.parseBoolean(fieldValue.toString()); + } else if (BYTE_TYPE.equals(fieldType)) { + return Byte.valueOf(fieldValue.toString()); + } else if (SHORT_TYPE.equals(fieldType)) { + return Short.parseShort(fieldValue.toString()); + } else if (INT_TYPE.equals(fieldType)) { + return Integer.parseInt(fieldValue.toString()); + } else if (LONG_TYPE.equals(fieldType)) { + return Long.parseLong(fieldValue.toString()); + } else if (FLOAT_TYPE.equals(fieldType)) { + return Float.parseFloat(fieldValue.toString()); + } else if (DOUBLE_TYPE.equals(fieldType)) { + return Double.parseDouble(fieldValue.toString()); + } else if (LocalTimeType.LOCAL_DATE_TYPE.equals(fieldType)) { + LocalDateTime localDateTime = parseDate(fieldValue.toString()); + return localDateTime.toLocalDate(); + } else if (LocalTimeType.LOCAL_TIME_TYPE.equals(fieldType)) { + LocalDateTime localDateTime = parseDate(fieldValue.toString()); + return localDateTime.toLocalTime(); + } else if (LocalTimeType.LOCAL_DATE_TIME_TYPE.equals(fieldType)) { + return parseDate(fieldValue.toString()); + } else if (fieldType instanceof DecimalType) { + return new BigDecimal(fieldValue.toString()); + } else if (fieldType instanceof ArrayType) { + ArrayType arrayType = (ArrayType) fieldType; + SeaTunnelDataType elementType = arrayType.getElementType(); + List stringList = (List) fieldValue; + Object arr = Array.newInstance(elementType.getTypeClass(), stringList.size()); + for (int i = 0; i < stringList.size(); i++) { + Object convertValue = convertValue(elementType, stringList.get(i)); + Array.set(arr, i, convertValue); + } + return arr; + } else if (fieldType instanceof MapType) { + MapType mapType = (MapType) fieldType; + SeaTunnelDataType keyType = mapType.getKeyType(); + + SeaTunnelDataType valueType = mapType.getValueType(); + Map stringMap = + mapper.readValue( + fieldValue.toString(), + new TypeReference>() {}); + Map convertMap = new HashMap(); + for (Map.Entry entry : stringMap.entrySet()) { + Object convertKey = convertValue(keyType, entry.getKey()); + Object convertValue = convertValue(valueType, entry.getValue()); + convertMap.put(convertKey, convertValue); + } + return convertMap; + } else if (fieldType instanceof SeaTunnelRowType) { + SeaTunnelRowType rowType = (SeaTunnelRowType) fieldType; + Map collect = (Map) fieldValue; + Object[] seaTunnelFields = new Object[rowType.getTotalFields()]; + for (int i = 0; i < rowType.getTotalFields(); i++) { + String fieldName = rowType.getFieldName(i); + SeaTunnelDataType fieldDataType = rowType.getFieldType(i); + Object value = collect.get(fieldName); + if (value != null) { + seaTunnelFields[i] = convertValue(fieldDataType, value); + } + } + return new SeaTunnelRow(seaTunnelFields); + } else if (fieldType instanceof PrimitiveByteArrayType) { + return Base64.getDecoder().decode(fieldValue.toString()); + } else if (VOID_TYPE.equals(fieldType) || fieldType == null) { + return null; + } else { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, + "Unexpected value: " + fieldType); + } + } + } + + private LocalDateTime parseDate(String fieldValue) { + // handle strings of timestamp type + try { + long ts = Long.parseLong(fieldValue); + return LocalDateTime.ofInstant(Instant.ofEpochMilli(ts), ZoneId.systemDefault()); + } catch (NumberFormatException e) { + // no op + } + String formatDate = fieldValue.replace("T", " ").replace("Z", ""); + if (fieldValue.length() == "yyyyMMdd".length() + || fieldValue.length() == "yyyy-MM-dd".length()) { + formatDate = fieldValue + " 00:00:00"; + } + DateTimeFormatter dateTimeFormatter = dateTimeFormatterMap.get(formatDate.length()); + if (dateTimeFormatter == null) { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, "unsupported date format"); + } + return LocalDateTime.parse(formatDate, dateTimeFormatter); + } + + Object recursiveGet(Map collect, String keyWithRecursive) { + Object value = null; + boolean isFirst = true; + for (String key : keyWithRecursive.split("\\.")) { + if (isFirst) { + value = collect.get(key); + isFirst = false; + } else if (value instanceof ObjectNode) { + value = ((ObjectNode) value).get(key); + } + } + return value; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java new file mode 100644 index 00000000000..a5864e596bc --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +public interface SeaTunnelRowDeserializer { + + SeaTunnelRow deserialize(TypesenseRecord rowRecord); +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java new file mode 100644 index 00000000000..154b94ddc65 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.ToString; + +import java.util.Map; + +@Getter +@ToString +@AllArgsConstructor +public class TypesenseRecord { + private Map doc; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java new file mode 100644 index 00000000000..e52638f83ef --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; +import org.apache.seatunnel.api.sink.SaveModeHandler; +import org.apache.seatunnel.api.sink.SchemaSaveMode; +import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.sink.SupportSaveMode; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.FactoryUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseAggregatedCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseSinkState; + +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.MAX_BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.MAX_RETRY_COUNT; + +public class TypesenseSink + implements SeaTunnelSink< + SeaTunnelRow, + TypesenseSinkState, + TypesenseCommitInfo, + TypesenseAggregatedCommitInfo>, + SupportMultiTableSink, + SupportSaveMode { + + private ReadonlyConfig config; + private CatalogTable catalogTable; + private final int maxBatchSize; + private final int maxRetryCount; + + public TypesenseSink(ReadonlyConfig config, CatalogTable catalogTable) { + this.config = config; + this.catalogTable = catalogTable; + maxBatchSize = config.get(MAX_BATCH_SIZE); + maxRetryCount = config.get(MAX_RETRY_COUNT); + } + + @Override + public String getPluginName() { + return "Typesense"; + } + + @Override + public TypesenseSinkWriter createWriter(SinkWriter.Context context) { + return new TypesenseSinkWriter(context, catalogTable, config, maxBatchSize, maxRetryCount); + } + + @Override + public Optional getSaveModeHandler() { + CatalogFactory catalogFactory = + FactoryUtil.discoverFactory( + Thread.currentThread().getContextClassLoader(), + CatalogFactory.class, + getPluginName()); + if (catalogFactory == null) { + return Optional.empty(); + } + Catalog catalog = catalogFactory.createCatalog(catalogFactory.factoryIdentifier(), config); + SchemaSaveMode schemaSaveMode = config.get(SinkConfig.SCHEMA_SAVE_MODE); + DataSaveMode dataSaveMode = config.get(SinkConfig.DATA_SAVE_MODE); + + TablePath tablePath = TablePath.of("", catalogTable.getTableId().getTableName()); + catalog.open(); + return Optional.of( + new DefaultSaveModeHandler( + schemaSaveMode, dataSaveMode, catalog, tablePath, null, null)); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java new file mode 100644 index 00000000000..4c937609406 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.connector.TableSink; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; + +import com.google.auto.service.AutoService; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.COLLECTION; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.KEY_DELIMITER; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.PRIMARY_KEYS; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.APIKEY; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.HOSTS; + +@AutoService(Factory.class) +public class TypesenseSinkFactory implements TableSinkFactory { + + @Override + public String factoryIdentifier() { + return "Typesense"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + HOSTS, + COLLECTION, + APIKEY, + SinkConfig.SCHEMA_SAVE_MODE, + SinkConfig.DATA_SAVE_MODE) + .optional(PRIMARY_KEYS, KEY_DELIMITER) + .build(); + } + + @Override + public TableSink createSink(TableSinkFactoryContext context) { + ReadonlyConfig readonlyConfig = context.getOptions(); + String original = readonlyConfig.get(COLLECTION); + CatalogTable newTable = + CatalogTable.of( + TableIdentifier.of( + context.getCatalogTable().getCatalogName(), + context.getCatalogTable().getTablePath().getDatabaseName(), + original), + context.getCatalogTable()); + return () -> new TypesenseSink(readonlyConfig, newTable); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java new file mode 100644 index 00000000000..9fdaf68a242 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.utils.RetryUtils; +import org.apache.seatunnel.common.utils.RetryUtils.RetryMaterial; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.SeaTunnelRowSerializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.TypesenseRowSerializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseSinkState; + +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static org.apache.seatunnel.api.table.type.RowKind.INSERT; +import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.INSERT_DOC_ERROR; + +@Slf4j +public class TypesenseSinkWriter + implements SinkWriter, + SupportMultiTableSinkWriter { + + private final Context context; + private final int maxBatchSize; + private final SeaTunnelRowSerializer seaTunnelRowSerializer; + + private final List requestEsList; + + private final String collection; + private TypesenseClient typesenseClient; + private RetryMaterial retryMaterial; + private static final long DEFAULT_SLEEP_TIME_MS = 200L; + + public TypesenseSinkWriter( + Context context, + CatalogTable catalogTable, + ReadonlyConfig config, + int maxBatchSize, + int maxRetryCount) { + this.context = context; + this.maxBatchSize = maxBatchSize; + + collection = catalogTable.getTableId().getTableName(); + CollectionInfo collectionInfo = + new CollectionInfo(catalogTable.getTableId().getTableName(), config); + typesenseClient = TypesenseClient.createInstance(config); + this.seaTunnelRowSerializer = + new TypesenseRowSerializer(collectionInfo, catalogTable.getSeaTunnelRowType()); + + this.requestEsList = new ArrayList<>(maxBatchSize); + this.retryMaterial = + new RetryMaterial(maxRetryCount, true, exception -> true, DEFAULT_SLEEP_TIME_MS); + } + + @Override + public void write(SeaTunnelRow element) { + if (RowKind.UPDATE_BEFORE.equals(element.getRowKind())) { + return; + } + + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + String indexRequestRow = seaTunnelRowSerializer.serializeRow(element); + requestEsList.add(indexRequestRow); + if (requestEsList.size() >= maxBatchSize) { + insert(collection, requestEsList); + } + break; + case UPDATE_BEFORE: + case DELETE: + String id = seaTunnelRowSerializer.serializeRowForDelete(element); + typesenseClient.deleteCollectionData(collection, id); + break; + default: + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Unsupported write row kind: " + element.getRowKind()); + } + } + + @Override + public Optional prepareCommit() { + insert(this.collection, this.requestEsList); + return Optional.empty(); + } + + private void insert(String collection, List requestEsList) { + try { + RetryUtils.retryWithException( + () -> { + typesenseClient.insert(collection, requestEsList); + return null; + }, + retryMaterial); + requestEsList.clear(); + } catch (Exception e) { + log.error(INSERT_DOC_ERROR.getDescription()); + throw new TypesenseConnectorException( + INSERT_DOC_ERROR, INSERT_DOC_ERROR.getDescription()); + } + } + + @Override + public void abortPrepare() {} + + @Override + public void close() { + insert(collection, requestEsList); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java new file mode 100644 index 00000000000..a3860a0495a --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.SupportColumnProjection; +import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import lombok.extern.slf4j.Slf4j; + +import java.util.Collections; +import java.util.List; + +@Slf4j +public class TypesenseSource + implements SeaTunnelSource, + SupportParallelism, + SupportColumnProjection { + + private final ReadonlyConfig config; + + private CatalogTable catalogTable; + + public TypesenseSource(ReadonlyConfig config) { + this.config = config; + if (config.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { + catalogTable = CatalogTableUtil.buildWithConfig(config); + } + } + + @Override + public String getPluginName() { + return "Typesense"; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public List getProducedCatalogTables() { + return Collections.singletonList(catalogTable); + } + + @Override + public SourceReader createReader( + SourceReader.Context readerContext) throws Exception { + return new TypesenseSourceReader(readerContext, config, catalogTable.getSeaTunnelRowType()); + } + + @Override + public SourceSplitEnumerator createEnumerator( + SourceSplitEnumerator.Context enumeratorContext) { + return new TypesenseSourceSplitEnumerator(enumeratorContext, config); + } + + @Override + public SourceSplitEnumerator restoreEnumerator( + SourceSplitEnumerator.Context enumeratorContext, + TypesenseSourceState checkpointState) { + return new TypesenseSourceSplitEnumerator(enumeratorContext, config); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java new file mode 100644 index 00000000000..254763ebbb1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; + +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig.COLLECTION; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.APIKEY; +import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.HOSTS; + +@AutoService(Factory.class) +public class TypesenseSourceFactory implements TableSourceFactory { + + @Override + public String factoryIdentifier() { + return "Typesense"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().required(HOSTS, APIKEY).optional(COLLECTION).build(); + } + + @Override + public + TableSource createSource(TableSourceFactoryContext context) { + return () -> (SeaTunnelSource) new TypesenseSource(context.getOptions()); + } + + @Override + public Class getSourceClass() { + return TypesenseSource.class; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java new file mode 100644 index 00000000000..16946519968 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.DefaultSeaTunnelRowDeserializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.SeaTunnelRowDeserializer; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.TypesenseRecord; + +import org.typesense.model.SearchResult; +import org.typesense.model.SearchResultHit; + +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.Deque; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +@Slf4j +public class TypesenseSourceReader implements SourceReader { + + SourceReader.Context context; + + private final ReadonlyConfig config; + + private final SeaTunnelRowDeserializer deserializer; + + private TypesenseClient typesenseClient; + + Deque splits = new LinkedList<>(); + + boolean noMoreSplit; + + private final long pollNextWaitTime = 1000L; + + public TypesenseSourceReader( + SourceReader.Context context, ReadonlyConfig config, SeaTunnelRowType rowTypeInfo) { + this.context = context; + this.config = config; + this.deserializer = new DefaultSeaTunnelRowDeserializer(rowTypeInfo); + } + + @Override + public void open() { + typesenseClient = TypesenseClient.createInstance(this.config); + } + + @Override + public void close() { + // Nothing , because typesense does not require + } + + @Override + public List snapshotState(long checkpointId) throws Exception { + return new ArrayList<>(splits); + } + + @Override + public void addSplits(List splits) { + this.splits.addAll(splits); + } + + @Override + public void handleNoMoreSplits() { + noMoreSplit = true; + } + + @Override + public void pollNext(Collector output) throws Exception { + synchronized (output.getCheckpointLock()) { + TypesenseSourceSplit split = splits.poll(); + if (split != null) { + SourceCollectionInfo sourceCollectionInfo = split.getSourceCollectionInfo(); + int pageSize = sourceCollectionInfo.getQueryBatchSize(); + while (true) { + SearchResult searchResult = + typesenseClient.search( + sourceCollectionInfo.getCollection(), + sourceCollectionInfo.getQuery(), + sourceCollectionInfo.getOffset(), + sourceCollectionInfo.getQueryBatchSize()); + Integer found = searchResult.getFound(); + List hits = searchResult.getHits(); + for (SearchResultHit hit : hits) { + Map document = hit.getDocument(); + SeaTunnelRow seaTunnelRow = + deserializer.deserialize(new TypesenseRecord(document)); + output.collect(seaTunnelRow); + } + if ((double) found / pageSize - 1 + > sourceCollectionInfo.getOffset() / pageSize) { + sourceCollectionInfo.setOffset(sourceCollectionInfo.getOffset() + pageSize); + } else { + break; + } + } + + } else if (noMoreSplit) { + log.info("Closed the bounded Typesense source"); + context.signalNoMoreElement(); + } else { + Thread.sleep(pollNextWaitTime); + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java new file mode 100644 index 00000000000..39cf8530eec --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.ToString; + +@ToString +@AllArgsConstructor +public class TypesenseSourceSplit implements SourceSplit { + + private static final long serialVersionUID = -1L; + + private String splitId; + + @Getter private SourceCollectionInfo sourceCollectionInfo; + + @Override + public String splitId() { + return splitId; + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java new file mode 100644 index 00000000000..adb149ae4f7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +@Slf4j +public class TypesenseSourceSplitEnumerator + implements SourceSplitEnumerator { + + private final SourceSplitEnumerator.Context context; + + private final ReadonlyConfig config; + + private TypesenseClient typesenseClient; + + private final Object stateLock = new Object(); + + private Map> pendingSplit; + + private volatile boolean shouldEnumerate; + + public TypesenseSourceSplitEnumerator( + SourceSplitEnumerator.Context context, ReadonlyConfig config) { + this(context, null, config); + } + + public TypesenseSourceSplitEnumerator( + SourceSplitEnumerator.Context context, + TypesenseSourceState sourceState, + ReadonlyConfig config) { + this.context = context; + this.config = config; + this.pendingSplit = new HashMap<>(); + this.shouldEnumerate = sourceState == null; + if (sourceState != null) { + this.shouldEnumerate = sourceState.isShouldEnumerate(); + this.pendingSplit.putAll(sourceState.getPendingSplit()); + } + } + + @Override + public void open() { + // Nothing + } + + @Override + public void run() throws Exception { + Set readers = context.registeredReaders(); + if (shouldEnumerate) { + List newSplits = getTypesenseSplit(); + + synchronized (stateLock) { + addPendingSplit(newSplits); + shouldEnumerate = false; + } + + assignSplit(readers); + } + + log.debug( + "No more splits to assign." + " Sending NoMoreSplitsEvent to reader {}.", readers); + readers.forEach(context::signalNoMoreSplits); + } + + private void addPendingSplit(Collection splits) { + int readerCount = context.currentParallelism(); + for (TypesenseSourceSplit split : splits) { + int ownerReader = getSplitOwner(split.splitId(), readerCount); + log.info("Assigning {} to {} reader.", split, ownerReader); + pendingSplit.computeIfAbsent(ownerReader, r -> new ArrayList<>()).add(split); + } + } + + private void assignSplit(Collection readers) { + log.debug("Assign pendingSplits to readers {}", readers); + + for (int reader : readers) { + List assignmentForReader = pendingSplit.remove(reader); + if (assignmentForReader != null && !assignmentForReader.isEmpty()) { + log.info("Assign splits {} to reader {}", assignmentForReader, reader); + try { + context.assignSplit(reader, assignmentForReader); + } catch (Exception e) { + log.error( + "Failed to assign splits {} to reader {}", + assignmentForReader, + reader, + e); + pendingSplit.put(reader, assignmentForReader); + } + } + } + } + + private static int getSplitOwner(String tp, int numReaders) { + return (tp.hashCode() & Integer.MAX_VALUE) % numReaders; + } + + private List getTypesenseSplit() { + List splits = new ArrayList<>(); + + String collection = config.get(SourceConfig.COLLECTION); + String query = config.get(SourceConfig.QUERY); + int queryBatchSize = config.get(SourceConfig.QUERY_BATCH_SIZE); + splits.add( + new TypesenseSourceSplit( + collection, + new SourceCollectionInfo(collection, query, 0, 0, queryBatchSize))); + return splits; + } + + @Override + public void close() throws IOException { + // Nothing + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + if (!splits.isEmpty()) { + addPendingSplit(splits); + assignSplit(Collections.singletonList(subtaskId)); + } + } + + @Override + public int currentUnassignedSplitSize() { + return pendingSplit.size(); + } + + @Override + public void handleSplitRequest(int subtaskId) { + throw new TypesenseConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Unsupported handleSplitRequest: " + subtaskId); + } + + @Override + public void registerReader(int subtaskId) { + log.debug("Register reader {} to IoTDBSourceSplitEnumerator.", subtaskId); + if (!pendingSplit.isEmpty()) { + assignSplit(Collections.singletonList(subtaskId)); + } + } + + @Override + public TypesenseSourceState snapshotState(long checkpointId) throws Exception { + synchronized (stateLock) { + return new TypesenseSourceState(shouldEnumerate, pendingSplit); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java new file mode 100644 index 00000000000..e29c7326102 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.source; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +@AllArgsConstructor +@Getter +public class TypesenseSourceState implements Serializable { + private boolean shouldEnumerate; + private Map> pendingSplit; +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java new file mode 100644 index 00000000000..b0f814dd8fd --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.state; + +import java.io.Serializable; + +public class TypesenseAggregatedCommitInfo implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java new file mode 100644 index 00000000000..05e0ea83f72 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.state; + +import java.io.Serializable; + +public class TypesenseCommitInfo implements Serializable {} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java similarity index 83% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java rename to seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java index a55d706c099..99f341d78cc 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java @@ -15,8 +15,8 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.llm; +package org.apache.seatunnel.connectors.seatunnel.typesense.state; -public enum ModelProvider { - OPENAI -} +import java.io.Serializable; + +public class TypesenseSinkState implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java new file mode 100644 index 00000000000..cc6f26cd43f --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.util; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +public class URLParamsConverter { + + public static String convertParamsToJson(String paramsString) { + return Optional.ofNullable(paramsString) + .filter(s -> !s.isEmpty()) + .map(URLParamsConverter::parseParams) + .map( + paramsMap -> { + try { + return new ObjectMapper().writeValueAsString(paramsMap); + } catch (IOException e) { + throw new RuntimeException("Error converting params to JSON", e); + } + }) + .orElseThrow( + () -> + new IllegalArgumentException( + "Parameter string must not be null or empty.")); + } + + private static Map parseParams(String paramsString) { + return Arrays.stream( + Optional.ofNullable(paramsString) + .filter(s -> !s.isEmpty()) + .orElseThrow( + () -> + new IllegalArgumentException( + "Parameter string must not be null or empty.")) + .split("&")) + .map(part -> part.split("=", 2)) + .peek( + keyValue -> { + if (keyValue.length != 2) { + throw new TypesenseConnectorException( + TypesenseConnectorErrorCode.QUERY_PARAM_ERROR, + "Query parameter error: " + Arrays.toString(keyValue)); + } + }) + .collect(Collectors.toMap(keyValue -> keyValue[0], keyValue -> keyValue[1])); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java new file mode 100644 index 00000000000..bb604869ff1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.serializer; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; +import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; +import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.TypesenseRowSerializer; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; + +public class TypesenseRowSerializerTest { + @Test + public void testSerializeUpsert() { + String collection = "test"; + String primaryKey = "id"; + Map confMap = new HashMap<>(); + confMap.put(SinkConfig.COLLECTION.key(), collection); + confMap.put(SinkConfig.PRIMARY_KEYS.key(), Arrays.asList(primaryKey)); + + ReadonlyConfig pluginConf = ReadonlyConfig.fromMap(confMap); + CollectionInfo collectionInfo = new CollectionInfo(collection, pluginConf); + SeaTunnelRowType schema = + new SeaTunnelRowType( + new String[] {primaryKey, "name"}, + new SeaTunnelDataType[] {STRING_TYPE, STRING_TYPE}); + TypesenseRowSerializer typesenseRowSerializer = + new TypesenseRowSerializer(collectionInfo, schema); + String id = "0001"; + String name = "jack"; + SeaTunnelRow row = new SeaTunnelRow(new Object[] {id, name}); + row.setRowKind(RowKind.UPDATE_AFTER); + Assertions.assertEquals(typesenseRowSerializer.serializeRowForDelete(row), id); + row.setRowKind(RowKind.INSERT); + String data = "{\"name\":\"jack\",\"id\":\"0001\"}"; + Assertions.assertEquals(typesenseRowSerializer.serializeRow(row), data); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java new file mode 100644 index 00000000000..568c96dbeed --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.sink; + +import org.apache.seatunnel.connectors.seatunnel.typesense.source.TypesenseSourceFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TypesenseFactoryTest { + + @Test + void optionRule() { + Assertions.assertNotNull((new TypesenseSourceFactory()).optionRule()); + Assertions.assertNotNull((new TypesenseSinkFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java new file mode 100644 index 00000000000..716cb7cc20a --- /dev/null +++ b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.typesense.util; + +import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class URLParamsConverterTest { + + @Test + public void convertParamsToJson() { + String json = URLParamsConverter.convertParamsToJson("q=*&filter_by=num_employees:10"); + Assertions.assertEquals(json, "{\"q\":\"*\",\"filter_by\":\"num_employees:10\"}"); + Assertions.assertThrows( + TypesenseConnectorException.class, + () -> URLParamsConverter.convertParamsToJson("q=*&filter_by=num_employees:10&b")); + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index 6bd4065d088..cf7314e619a 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -79,7 +79,9 @@ connector-web3j connector-milvus connector-activemq + connector-qdrant connector-sls + connector-typesense diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd index c4a4db1bc12..89327e8e38f 100644 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd @@ -45,14 +45,9 @@ for %%I in (%*) do ( set "JAVA_OPTS=%JvmOption%" set "SEATUNNEL_CONFIG=%CONF_DIR%\seatunnel.yaml" -for %%I in (%*) do ( - set "arg=%%I" - if "!arg:~0,10!"=="JvmOption=" ( - set "JAVA_OPTS=%JAVA_OPTS% !arg:~10!" - ) -) set "JAVA_OPTS=%JAVA_OPTS% -Dlog4j2.contextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector" +set "JAVA_OPTS=%JAVA_OPTS% -Dlog4j2.isThreadContextMapInheritable=true" REM Server Debug Config REM Usage instructions: @@ -104,6 +99,14 @@ if "%NODE_ROLE%" == "master" ( exit 1 ) +REM Parse JvmOption from command line, it should be parsed after jvm_options +for %%I in (%*) do ( + set "arg=%%I" + if "!arg:~0,10!"=="JvmOption=" ( + set "JAVA_OPTS=%JAVA_OPTS% !arg:~10!" + ) +) + IF NOT EXIST "%HAZELCAST_CONFIG%" ( echo Error: File %HAZELCAST_CONFIG% does not exist. exit /b 1 diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh index f9c7a33b406..565644db22b 100755 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh @@ -68,8 +68,7 @@ fi for i in "$@" do if [[ "${i}" == *"JvmOption"* ]]; then - JVM_OPTION="${i}" - JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" + : elif [[ "${i}" == "-d" || "${i}" == "--daemon" ]]; then DAEMON=true elif [[ "${i}" == "-r" || "${i}" == "--role" ]]; then @@ -84,6 +83,7 @@ done # Log4j2 Config JAVA_OPTS="${JAVA_OPTS} -Dlog4j2.contextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector" +JAVA_OPTS="${JAVA_OPTS} -Dlog4j2.isThreadContextMapInheritable=true" if [ -e "${CONF_DIR}/log4j2.properties" ]; then JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.logging.type=log4j2 -Dlog4j2.configurationFile=${CONF_DIR}/log4j2.properties" JAVA_OPTS="${JAVA_OPTS} -Dseatunnel.logs.path=${APP_DIR}/logs" @@ -139,6 +139,15 @@ JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.config=${HAZELCAST_CONFIG}" # port in your IDE. After that, you can happily debug your code. # JAVA_OPTS="${JAVA_OPTS} -Xdebug -Xrunjdwp:server=y,transport=dt_socket,address=5001,suspend=n" +# Parse JvmOption from command line, it should be parsed after jvm_options +for i in "$@" +do + if [[ "${i}" == *"JvmOption"* ]]; then + JVM_OPTION="${i}" + JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" + fi +done + CLASS_PATH=${APP_DIR}/lib/*:${APP_JAR} echo "start ${NODE_ROLE} node" diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd index b7208616714..cecd3797cda 100644 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd @@ -58,16 +58,6 @@ if defined JvmOption ( set "JAVA_OPTS=%JAVA_OPTS% %JvmOption%" ) -for %%i in (%*) do ( - set "arg=%%i" - if "!arg:~0,9!"=="JvmOption" ( - set "JVM_OPTION=!arg:~9!" - set "JAVA_OPTS=!JAVA_OPTS! !JVM_OPTION!" - goto :break_loop - ) -) -:break_loop - set "JAVA_OPTS=%JAVA_OPTS% -Dhazelcast.client.config=%HAZELCAST_CLIENT_CONFIG%" set "JAVA_OPTS=%JAVA_OPTS% -Dseatunnel.config=%SEATUNNEL_CONFIG%" set "JAVA_OPTS=%JAVA_OPTS% -Dhazelcast.config=%HAZELCAST_CONFIG%" @@ -105,4 +95,15 @@ for /f "usebackq delims=" %%a in ("%APP_DIR%\config\jvm_client_options") do ( ) ) +REM Parse JvmOption from command line, it should be parsed after jvm_client_options +for %%i in (%*) do ( + set "arg=%%i" + if "!arg:~0,9!"=="JvmOption" ( + set "JVM_OPTION=!arg:~9!" + set "JAVA_OPTS=!JAVA_OPTS! !JVM_OPTION!" + goto :break_loop + ) +) +:break_loop + java %JAVA_OPTS% -cp %CLASS_PATH% %APP_MAIN% %args% diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh index e7fbee00b36..dc4c3f91e56 100755 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh @@ -69,15 +69,6 @@ if test ${JvmOption} ;then JAVA_OPTS="${JAVA_OPTS} ${JvmOption}" fi -for i in "$@" -do - if [[ "${i}" == *"JvmOption"* ]]; then - JVM_OPTION="${i}" - JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" - break - fi -done - JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.client.config=${HAZELCAST_CLIENT_CONFIG}" JAVA_OPTS="${JAVA_OPTS} -Dseatunnel.config=${SEATUNNEL_CONFIG}" JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.config=${HAZELCAST_CONFIG}" @@ -108,4 +99,14 @@ while IFS= read -r line || [[ -n "$line" ]]; do fi done < ${APP_DIR}/config/jvm_client_options +# Parse JvmOption from command line, it should be parsed after jvm_client_options +for i in "$@" +do + if [[ "${i}" == *"JvmOption"* ]]; then + JVM_OPTION="${i}" + JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" + break + fi +done + java ${JAVA_OPTS} -cp ${CLASS_PATH} ${APP_MAIN} ${args} diff --git a/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties b/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties index 2dc1b8ca501..0bf06d2b773 100644 --- a/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties +++ b/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties @@ -25,7 +25,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -35,7 +35,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index fb6935f0894..584224c8fd9 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -518,6 +518,12 @@ ${project.version} provided + + org.apache.seatunnel + connector-cdc-opengauss + ${project.version} + provided + org.apache.seatunnel connector-tdengine @@ -592,6 +598,13 @@ provided + + org.apache.seatunnel + connector-qdrant + ${project.version} + provided + + com.aliyun.phoenix diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE index f5d9bebbe54..5cc3074d417 100644 --- a/seatunnel-dist/release-docs/LICENSE +++ b/seatunnel-dist/release-docs/LICENSE @@ -267,6 +267,7 @@ The text of each license is the standard Apache 2.0 license. (Apache-2.0) woodstox-core (com.fasterxml.woodstox:woodstox-core:5.0.3 - https://github.com/FasterXML/woodstox) (Apache-2.0) jcip-annotations (com.github.stephenc.jcip:jcip-annotations:1.0-1 - https://github.com/stephenc/jcip-annotations) (Apache-2.0) gson (com.google.code.gson:gson:2.2.4 - https://github.com/google/gson) + (Apache-2.0) gson (com.google.code.gson:gson:2.8.9 - https://github.com/google/gson) (Apache-2.0) nimbus-jose-jwt (com.nimbusds:nimbus-jose-jwt:7.9 - https://bitbucket.org/connect2id/nimbus-jose-jwt) (Apache-2.0) beanutils (commons-beanutils:commons-beanutils:1.9.4 - https://commons.apache.org/proper/commons-beanutils/) (Apache-2.0) commons-cli (commons-cli:commons-cli:1.2 - https://commons.apache.org/proper/commons-cli/) @@ -318,8 +319,10 @@ The text of each license is the standard Apache 2.0 license. (The Apache Software License, Version 2.0) hazelcast (com.hazelcast:hazelcast:5.1 - https://github.com/hazelcast/hazelcast) (Apache-2.0) disruptor (com.lmax:disruptor:3.4.4 https://lmax-exchange.github.io/disruptor/) (Apache-2.0) error_prone_annotations (com.google.errorprone:error_prone_annotations:2.2.0 https://mvnrepository.com/artifact/com.google.errorprone/error_prone_annotations/2.2.0) + (Apache-2.0) error_prone_annotations (com.google.errorprone:error_prone_annotations:2.18.0 https://mvnrepository.com/artifact/com.google.errorprone/error_prone_annotations/2.18.0) (Apache-2.0) failureaccess (com.google.guava:failureaccess:1.0 https://mvnrepository.com/artifact/com.google.guava/failureaccess/1.0) (Apache-2.0) j2objc-annotations (com.google.j2objc:j2objc-annotations:1.1 https://mvnrepository.com/artifact/com.google.j2objc/j2objc-annotations/1.1) + (Apache-2.0) j2objc-annotations (com.google.j2objc:j2objc-annotations:2.8 https://mvnrepository.com/artifact/com.google.j2objc/j2objc-annotations/2.8) (Apache-2.0) listenablefuture (com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava https://mvnrepository.com/artifact/com.google.guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava) (Apache-2.0) accessors-smart (net.minidev:accessors-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/accessors-smart) (Apache-2.0) json-smart (net.minidev:json-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/json-smart) diff --git a/seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt b/seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt new file mode 100644 index 00000000000..ad410e11302 --- /dev/null +++ b/seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt @@ -0,0 +1,201 @@ +Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed 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. \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml new file mode 100644 index 00000000000..f95e5cdb1a0 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml @@ -0,0 +1,75 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-cdc-opengauss-e2e + SeaTunnel : E2E : Connector V2 : CDC Opengauss + + + + + org.apache.seatunnel + connector-jdbc + ${project.version} + pom + import + + + + + + + + org.apache.seatunnel + connector-cdc-opengauss + ${project.version} + test + + + + org.apache.seatunnel + connector-jdbc + ${project.version} + test + + + + org.testcontainers + postgresql + ${testcontainer.version} + test + + + + + org.postgresql + postgresql + 42.5.1 + + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java new file mode 100644 index 00000000000..dc80a083a76 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java @@ -0,0 +1,727 @@ +/* + * 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.cdc.postgres; + +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; +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.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.await; +import static org.awaitility.Awaitility.given; +import static org.junit.Assert.assertNotNull; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK do not support cdc") +public class OpengaussCDCIT extends TestSuiteBase implements TestResource { + private static final int OPENGAUSS_PORT = 5432; + private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + private static final String USERNAME = "gaussdb"; + private static final String PASSWORD = "openGauss@123"; + private static final String OPENGAUSSQL_DATABASE = "opengauss_cdc"; + private static final String OPENGAUSSQL_DEFAULT_DATABASE = "postgres"; + private static final String OPENGAUSS_SCHEMA = "inventory"; + + private static final String SOURCE_TABLE_1 = "opengauss_cdc_table_1"; + private static final String SOURCE_TABLE_2 = "opengauss_cdc_table_2"; + private static final String SOURCE_TABLE_3 = "opengauss_cdc_table_3"; + private static final String SINK_TABLE_1 = "sink_opengauss_cdc_table_1"; + private static final String SINK_TABLE_2 = "sink_opengauss_cdc_table_2"; + private static final String SINK_TABLE_3 = "sink_opengauss_cdc_table_3"; + + private static final String SOURCE_TABLE_NO_PRIMARY_KEY = "full_types_no_primary_key"; + + private static final String OPENGAUSS_HOST = "opengauss_cdc_e2e"; + + protected static final DockerImageName OPENGAUSS_IMAGE = + DockerImageName.parse("opengauss/opengauss:5.0.0") + .asCompatibleSubstituteFor("postgres"); + + private static final String SOURCE_SQL_TEMPLATE = "select * from %s.%s order by id"; + + public static final GenericContainer OPENGAUSS_CONTAINER = + new GenericContainer<>(OPENGAUSS_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(OPENGAUSS_HOST) + .withEnv("GS_PASSWORD", PASSWORD) + .withLogConsumer(new Slf4jLogConsumer(log)); + + private String driverUrl() { + return "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.5.1/postgresql-42.5.1.jar"; + } + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/JDBC/lib && cd /tmp/seatunnel/plugins/JDBC/lib && wget " + + driverUrl()); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + @BeforeAll + @Override + public void startUp() throws Exception { + log.info("The second stage: Starting opengauss containers..."); + OPENGAUSS_CONTAINER.setPortBindings( + Lists.newArrayList(String.format("%s:%s", OPENGAUSS_PORT, OPENGAUSS_PORT))); + Startables.deepStart(Stream.of(OPENGAUSS_CONTAINER)).join(); + log.info("Opengauss Containers are started"); + given().ignoreExceptions() + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(2, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted(this::initializeOpengaussSql); + + String[] command1 = { + "/bin/sh", + "-c", + "sed -i 's/^#password_encryption_type = 2/password_encryption_type = 1/' /var/lib/opengauss/data/postgresql.conf" + }; + Container.ExecResult result1 = OPENGAUSS_CONTAINER.execInContainer(command1); + Assertions.assertEquals(0, result1.getExitCode()); + + String[] command2 = { + "/bin/sh", + "-c", + "sed -i 's/host replication gaussdb 0.0.0.0\\/0 md5/host replication gaussdb 0.0.0.0\\/0 sha256/' /var/lib/opengauss/data/pg_hba.conf" + }; + Container.ExecResult result2 = OPENGAUSS_CONTAINER.execInContainer(command2); + Assertions.assertEquals(0, result2.getExitCode()); + String[] command3 = { + "/bin/sh", + "-c", + "echo \"host all dailai 0.0.0.0/0 md5\" >> /var/lib/opengauss/data/pg_hba.conf" + }; + Container.ExecResult result3 = OPENGAUSS_CONTAINER.execInContainer(command3); + Assertions.assertEquals(0, result3.getExitCode()); + + reloadConf(); + + createNewUserForJdbcSink(); + } + + @TestTemplate + public void testOpengaussCdcCheckDataE2e(TestContainer container) { + try { + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob("/opengausscdc_to_opengauss.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query(getQuerySQL(OPENGAUSS_SCHEMA, SOURCE_TABLE_1)), + query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); + }); + + // insert update delete + upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query(getQuerySQL(OPENGAUSS_SCHEMA, SOURCE_TABLE_1)), + query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); + }); + } finally { + // Clear related content to ensure that multiple operations are not affected + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); + } + } + + @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support multi table") + public void testOpengaussCdcMultiTableE2e(TestContainer container) { + try { + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob( + "/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertAll( + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_1)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_1))), + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_2)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_2))))); + + // insert update delete + upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); + upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertAll( + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_1)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_1))), + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_2)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_2))))); + } finally { + // Clear related content to ensure that multiple operations are not affected + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_2); + } + } + + @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support multi table") + public void testMultiTableWithRestore(TestContainer container) + throws IOException, InterruptedException { + try { + CompletableFuture.supplyAsync( + () -> { + try { + return container.executeJob( + "/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + // insert update delete + upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertAll( + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_1)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_1))))); + + Pattern jobIdPattern = + Pattern.compile( + ".*Init JobMaster for Job opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf \\(([0-9]*)\\).*", + Pattern.DOTALL); + Matcher matcher = jobIdPattern.matcher(container.getServerLogs()); + String jobId; + if (matcher.matches()) { + jobId = matcher.group(1); + } else { + throw new RuntimeException("Can not find jobId"); + } + + Assertions.assertEquals(0, container.savepointJob(jobId).getExitCode()); + + // Restore job with add a new table + CompletableFuture.supplyAsync( + () -> { + try { + container.restoreJob( + "/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf", + jobId); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + + upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertAll( + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_1)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_1))), + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_2)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_2))))); + + log.info("****************** container logs start ******************"); + String containerLogs = container.getServerLogs(); + log.info(containerLogs); + // pg cdc logs contain ERROR + // Assertions.assertFalse(containerLogs.contains("ERROR")); + log.info("****************** container logs end ******************"); + } finally { + // Clear related content to ensure that multiple operations are not affected + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_2); + } + } + + @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support multi table") + public void testAddFiledWithRestore(TestContainer container) + throws IOException, InterruptedException { + try { + CompletableFuture.supplyAsync( + () -> { + try { + return container.executeJob( + "/opengausscdc_to_opengauss_test_add_Filed.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + }); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertAll( + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_3)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_3))))); + + Pattern jobIdPattern = + Pattern.compile( + ".*Init JobMaster for Job opengausscdc_to_opengauss_test_add_Filed.conf \\(([0-9]*)\\).*", + Pattern.DOTALL); + Matcher matcher = jobIdPattern.matcher(container.getServerLogs()); + String jobId; + if (matcher.matches()) { + jobId = matcher.group(1); + } else { + throw new RuntimeException("Can not find jobId"); + } + + Assertions.assertEquals(0, container.savepointJob(jobId).getExitCode()); + + // add filed add insert source table data + addFieldsForTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_3); + addFieldsForTable(OPENGAUSS_SCHEMA, SINK_TABLE_3); + insertSourceTableForAddFields(OPENGAUSS_SCHEMA, SOURCE_TABLE_3); + + // Restore job + CompletableFuture.supplyAsync( + () -> { + try { + container.restoreJob( + "/opengausscdc_to_opengauss_test_add_Filed.conf", jobId); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> + Assertions.assertAll( + () -> + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_3)), + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SINK_TABLE_3))))); + } finally { + // Clear related content to ensure that multiple operations are not affected + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_3); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_3); + } + } + + @TestTemplate + public void testOpengaussCdcCheckDataWithNoPrimaryKey(TestContainer container) + throws Exception { + + try { + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob( + "/opengausscdc_to_opengauss_with_no_primary_key.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + + // snapshot stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_NO_PRIMARY_KEY)), + query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); + }); + + // insert update delete + upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_NO_PRIMARY_KEY)), + query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); + }); + } finally { + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); + } + } + + @TestTemplate + public void testOpengaussCdcCheckDataWithCustomPrimaryKey(TestContainer container) + throws Exception { + + try { + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob( + "/opengausscdc_to_opengauss_with_custom_primary_key.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + + // snapshot stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_NO_PRIMARY_KEY)), + query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); + }); + + // insert update delete + upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); + + // stream stage + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + query( + getQuerySQL( + OPENGAUSS_SCHEMA, + SOURCE_TABLE_NO_PRIMARY_KEY)), + query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); + }); + } finally { + clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); + clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); + } + } + + private void addFieldsForTable(String database, String tableName) { + executeSql("ALTER TABLE " + database + "." + tableName + " ADD COLUMN f_big BIGINT"); + } + + private void insertSourceTableForAddFields(String database, String tableName) { + executeSql( + "INSERT INTO " + + database + + "." + + tableName + + " VALUES (2, '2', 32767, 65535, 2147483647);"); + } + + private void clearTable(String database, String tableName) { + executeSql("truncate table " + database + "." + tableName); + } + + private void upsertDeleteSourceTable(String database, String tableName) { + + executeSql( + "INSERT INTO " + + database + + "." + + tableName + + " VALUES (2, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true,\n" + + " 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456',\n" + + " '2020-07-17', '18:00:22', 500);"); + + executeSql( + "INSERT INTO " + + database + + "." + + tableName + + " VALUES (3, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true,\n" + + " 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456',\n" + + " '2020-07-17', '18:00:22', 500);"); + + executeSql("DELETE FROM " + database + "." + tableName + " where id = 2;"); + + executeSql("UPDATE " + database + "." + tableName + " SET f_big = 10000 where id = 3;"); + } + + private void executeSql(String sql) { + try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); + Statement statement = connection.createStatement()) { + statement.execute("SET search_path TO inventory;"); + statement.execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private String getQuerySQL(String database, String tableName) { + return String.format(SOURCE_SQL_TEMPLATE, database, tableName); + } + + private List> query(String sql) { + try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE)) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + List> result = new ArrayList<>(); + int columnCount = resultSet.getMetaData().getColumnCount(); + while (resultSet.next()) { + ArrayList objects = new ArrayList<>(); + for (int i = 1; i <= columnCount; i++) { + Object object = resultSet.getObject(i); + if (object instanceof byte[]) { + byte[] bytes = (byte[]) object; + object = new String(bytes, StandardCharsets.UTF_8); + } + objects.add(object); + } + log.debug( + String.format( + "Print opengauss-CDC query, sql: %s, data: %s", sql, objects)); + result.add(objects); + } + return result; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + protected void createNewUserForJdbcSink() throws Exception { + try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); + Statement stmt = connection.createStatement()) { + // create a user for jdbc sink + stmt.execute("CREATE USER dailai WITH PASSWORD 'openGauss@123';"); + stmt.execute("GRANT ALL PRIVILEGES TO dailai;"); + } + } + + protected void reloadConf() throws Exception { + try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); + Statement stmt = connection.createStatement()) { + stmt.execute("select pg_reload_conf();"); + } + } + + protected void initializeOpengaussSql() throws Exception { + try (Connection connection = getJdbcConnection(OPENGAUSSQL_DEFAULT_DATABASE); + Statement stmt = connection.createStatement()) { + stmt.execute("create database " + OPENGAUSSQL_DATABASE); + } + final String ddlFile = String.format("ddl/%s.sql", "inventory"); + final URL ddlTestFile = OpengaussCDCIT.class.getClassLoader().getResource(ddlFile); + assertNotNull("Cannot locate " + ddlFile, ddlTestFile); + try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); + Statement statement = connection.createStatement()) { + final List statements = + Arrays.stream( + Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() + .map(String::trim) + .filter(x -> !x.startsWith("--") && !x.isEmpty()) + .map( + x -> { + final Matcher m = + COMMENT_PATTERN.matcher(x); + return m.matches() ? m.group(1) : x; + }) + .collect(Collectors.joining("\n")) + .split(";\n")) + .collect(Collectors.toList()); + for (String stmt : statements) { + statement.execute(stmt); + } + } + } + + private Connection getJdbcConnection(String dbName) throws SQLException { + return DriverManager.getConnection( + "jdbc:postgresql://" + + OPENGAUSS_CONTAINER.getHost() + + ":" + + OPENGAUSS_CONTAINER.getMappedPort(OPENGAUSS_PORT) + + "/" + + dbName, + USERNAME, + PASSWORD); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + if (OPENGAUSS_CONTAINER != null) { + OPENGAUSS_CONTAINER.close(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql new file mode 100644 index 00000000000..5d4564041b7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql @@ -0,0 +1,199 @@ +-- +-- 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: opengauss_cdc +-- ---------------------------------------------------------------------------------------------------------------- +-- Create and populate our products using a single insert with many rows +DROP SCHEMA IF EXISTS inventory CASCADE; +CREATE SCHEMA inventory; +SET search_path TO inventory; + +CREATE TABLE opengauss_cdc_table_1 +( + id INTEGER NOT NULL, + f_bytea BYTEA, + f_small SMALLINT, + f_int INTEGER, + f_big BIGINT, + f_real REAL, + f_double_precision DOUBLE PRECISION, + f_numeric NUMERIC(10, 5), + f_decimal DECIMAL(10, 1), + f_boolean BOOLEAN, + f_text TEXT, + f_char CHAR, + f_character CHARACTER(3), + f_character_varying CHARACTER VARYING(20), + f_timestamp3 TIMESTAMP(3), + f_timestamp6 TIMESTAMP(6), + f_date DATE, + f_time TIME(0), + f_default_numeric NUMERIC, + PRIMARY KEY (id) +); + +CREATE TABLE opengauss_cdc_table_2 +( + id INTEGER NOT NULL, + f_bytea BYTEA, + f_small SMALLINT, + f_int INTEGER, + f_big BIGINT, + f_real REAL, + f_double_precision DOUBLE PRECISION, + f_numeric NUMERIC(10, 5), + f_decimal DECIMAL(10, 1), + f_boolean BOOLEAN, + f_text TEXT, + f_char CHAR, + f_character CHARACTER(3), + f_character_varying CHARACTER VARYING(20), + f_timestamp3 TIMESTAMP(3), + f_timestamp6 TIMESTAMP(6), + f_date DATE, + f_time TIME(0), + f_default_numeric NUMERIC, + PRIMARY KEY (id) +); + +CREATE TABLE sink_opengauss_cdc_table_1 +( + id INTEGER NOT NULL, + f_bytea BYTEA, + f_small SMALLINT, + f_int INTEGER, + f_big BIGINT, + f_real REAL, + f_double_precision DOUBLE PRECISION, + f_numeric NUMERIC(10, 5), + f_decimal DECIMAL(10, 1), + f_boolean BOOLEAN, + f_text TEXT, + f_char CHAR, + f_character CHARACTER(3), + f_character_varying CHARACTER VARYING(20), + f_timestamp3 TIMESTAMP(3), + f_timestamp6 TIMESTAMP(6), + f_date DATE, + f_time TIME(0), + f_default_numeric NUMERIC, + PRIMARY KEY (id) +); + +CREATE TABLE sink_opengauss_cdc_table_2 +( + id INTEGER NOT NULL, + f_bytea BYTEA, + f_small SMALLINT, + f_int INTEGER, + f_big BIGINT, + f_real REAL, + f_double_precision DOUBLE PRECISION, + f_numeric NUMERIC(10, 5), + f_decimal DECIMAL(10, 1), + f_boolean BOOLEAN, + f_text TEXT, + f_char CHAR, + f_character CHARACTER(3), + f_character_varying CHARACTER VARYING(20), + f_timestamp3 TIMESTAMP(3), + f_timestamp6 TIMESTAMP(6), + f_date DATE, + f_time TIME(0), + f_default_numeric NUMERIC, + PRIMARY KEY (id) +); + +CREATE TABLE full_types_no_primary_key +( + id INTEGER NOT NULL, + f_bytea BYTEA, + f_small SMALLINT, + f_int INTEGER, + f_big BIGINT, + f_real REAL, + f_double_precision DOUBLE PRECISION, + f_numeric NUMERIC(10, 5), + f_decimal DECIMAL(10, 1), + f_boolean BOOLEAN, + f_text TEXT, + f_char CHAR, + f_character CHARACTER(3), + f_character_varying CHARACTER VARYING(20), + f_timestamp3 TIMESTAMP(3), + f_timestamp6 TIMESTAMP(6), + f_date DATE, + f_time TIME(0), + f_default_numeric NUMERIC +); + +CREATE TABLE opengauss_cdc_table_3 +( + id INTEGER NOT NULL, + f_bytea BYTEA, + f_small SMALLINT, + f_int INTEGER, + PRIMARY KEY (id) +); + +CREATE TABLE sink_opengauss_cdc_table_3 +( + id INTEGER NOT NULL, + f_bytea BYTEA, + f_small SMALLINT, + f_int INTEGER, + PRIMARY KEY (id) +); + +ALTER TABLE opengauss_cdc_table_1 + REPLICA IDENTITY FULL; + +ALTER TABLE opengauss_cdc_table_2 + REPLICA IDENTITY FULL; + +ALTER TABLE opengauss_cdc_table_3 + REPLICA IDENTITY FULL; + +ALTER TABLE sink_opengauss_cdc_table_1 + REPLICA IDENTITY FULL; + +ALTER TABLE sink_opengauss_cdc_table_2 + REPLICA IDENTITY FULL; + +ALTER TABLE full_types_no_primary_key + REPLICA IDENTITY FULL; + +INSERT INTO opengauss_cdc_table_1 +VALUES (1, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true, + 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', + '2020-07-17', '18:00:22', 500); + +INSERT INTO opengauss_cdc_table_2 +VALUES (1, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true, + 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', + '2020-07-17', '18:00:22', 500); + +INSERT INTO opengauss_cdc_table_3 +VALUES (1, '2', 32767, 65535); + +INSERT INTO full_types_no_primary_key +VALUES (1, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true, + 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', + '2020-07-17', '18:00:22', 500); + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf new file mode 100644 index 00000000000..c1227226362 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf @@ -0,0 +1,62 @@ +# +# 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 + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = opengauss_cdc + table = inventory.sink_opengauss_cdc_table_1 + primary_keys = ["id"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf new file mode 100644 index 00000000000..3c925f55311 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf @@ -0,0 +1,62 @@ +# +# 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 + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_3"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = opengauss_cdc + table = inventory.sink_opengauss_cdc_table_3 + primary_keys = ["id"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf new file mode 100644 index 00000000000..d916cb5e4e4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf @@ -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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + exactly_once = true + table-names-config = [ + { + table = "opengauss_cdc.inventory.full_types_no_primary_key" + primaryKeys = ["id"] + } + ] + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = opengauss_cdc + table = inventory.sink_opengauss_cdc_table_1 + primary_keys = ["id"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf new file mode 100644 index 00000000000..f75d7dc97d8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf @@ -0,0 +1,63 @@ +# +# 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 + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = "opengauss_cdc" + schema = "inventory" + tablePrefix = "sink_" + primary_keys = ["id"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf new file mode 100644 index 00000000000..d5ac1000012 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf @@ -0,0 +1,63 @@ +# +# 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 + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1","opengauss_cdc.inventory.opengauss_cdc_table_2"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = "opengauss_cdc" + schema = "inventory" + tablePrefix = "sink_" + primary_keys = ["id"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf new file mode 100644 index 00000000000..5decfb10606 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf @@ -0,0 +1,63 @@ +# +# 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 + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + read_limit.bytes_per_second=7000000 + read_limit.rows_per_second=400 +} + +source { + Opengauss-CDC { + result_table_name = "customers_opengauss_cdc" + username = "gaussdb" + password = "openGauss@123" + database-names = ["opengauss_cdc"] + schema-names = ["inventory"] + base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + decoding.plugin.name = "pgoutput" + table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] + exactly_once = false + } +} + +transform { + +} + +sink { + jdbc { + source_table_name = "customers_opengauss_cdc" + url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" + driver = "org.postgresql.Driver" + user = "dailai" + password = "openGauss@123" + + compatible_mode="postgresLow" + generate_sink_sql = true + # You need to configure both database and table + database = opengauss_cdc + table = inventory.sink_opengauss_cdc_table_1 + primary_keys = ["id"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java index e92c3993780..650918e3fc6 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java @@ -20,6 +20,7 @@ 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.shade.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.TablePath; @@ -68,17 +69,27 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; +import java.util.function.Function; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @Slf4j public class ElasticsearchIT extends TestSuiteBase implements TestResource { - private List testDataset; + private static final long INDEX_REFRESH_MILL_DELAY = 5000L; + + private List testDataset1; + + private List testDataset2; private ElasticsearchContainer container; @@ -114,7 +125,8 @@ public void startUp() throws Exception { Optional.empty(), Optional.empty(), Optional.empty()); - testDataset = generateTestDataSet(); + testDataset1 = generateTestDataSet1(); + testDataset2 = generateTestDataSet2(); createIndexForResourceNull("st_index"); createIndexDocs(); createIndexWithFullType(); @@ -123,15 +135,18 @@ public void startUp() throws Exception { /** create a index,and bulk some documents */ private void createIndexDocs() { + createIndexDocsByName("st_index"); + } + + private void createIndexDocsByName(String indexName) { + createIndexDocsByName(indexName, testDataset1); + } + + private void createIndexDocsByName(String indexName, List testDataSet) { StringBuilder requestBody = new StringBuilder(); - Map indexInner = new HashMap<>(); - indexInner.put("_index", "st"); - - Map> indexParam = new HashMap<>(); - indexParam.put("index", indexInner); - String indexHeader = "{\"index\":{\"_index\":\"st_index\"}\n"; - for (int i = 0; i < testDataset.size(); i++) { - String row = testDataset.get(i); + String indexHeader = String.format("{\"index\":{\"_index\":\"%s\"}\n", indexName); + for (int i = 0; i < testDataSet.size(); i++) { + String row = testDataSet.get(i); requestBody.append(indexHeader); requestBody.append(row); requestBody.append("\n"); @@ -159,7 +174,7 @@ private void createIndexWithFullType() throws IOException, InterruptedException + "\n"); Assertions.assertFalse(response.isErrors(), response.getResponse()); // waiting index refresh - Thread.sleep(2000L); + Thread.sleep(INDEX_REFRESH_MILL_DELAY); Assertions.assertEquals( 2, esRestClient.getIndexDocsCount("st_index_full_type").get(0).getDocsCount()); } @@ -175,16 +190,121 @@ private void createIndexForResourceNull(String indexName) throws IOException { } @TestTemplate - public void testElasticsearch(TestContainer container) + public void testElasticsearchWithSchema(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = container.executeJob("/elasticsearch/elasticsearch_source_and_sink.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - List sinkData = readSinkData("st_index2"); + List sinkData = readSinkDataWithSchema("st_index2"); // for DSL is: {"range":{"c_int":{"gte":10,"lte":20}}} Assertions.assertIterableEquals(mapTestDatasetForDSL(), sinkData); } + @TestTemplate + @DisabledOnContainer( + value = {}, + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple table read") + public void testElasticsSearchWithMultiSourceByFilter(TestContainer container) + throws InterruptedException, IOException { + // read read_filter_index1,read_filter_index2 + // write into read_filter_index1_copy,read_filter_index2_copy + createIndexDocsByName("read_filter_index1", testDataset1); + createIndexDocsByName("read_filter_index2", testDataset2); + + Container.ExecResult execResult = + container.executeJob( + "/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + + HashMap rangeParam = new HashMap<>(); + rangeParam.put("gte", 10); + rangeParam.put("lte", 20); + HashMap range1 = new HashMap<>(); + range1.put("c_int", rangeParam); + Map query1 = new HashMap<>(); + query1.put("range", range1); + + Map query2 = new HashMap<>(); + HashMap range2 = new HashMap<>(); + range2.put("c_int2", rangeParam); + query2.put("range", range2); + + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(INDEX_REFRESH_MILL_DELAY)); + Set sinkData1 = + new HashSet<>( + getDocsWithTransformDate( + // read all field + Collections.emptyList(), + // read indexName + "read_filter_index1_copy", + // allowed c_null serialized if null + Lists.newArrayList("c_null"), + // query condition + query1, + // transformDate field:c_date + Lists.newArrayList("c_date"), + // order field + "c_int")); + + List index1Data = + mapTestDatasetForDSL( + // use testDataset1 + testDataset1, + // filter testDataset1 match sinkData1 + doc -> { + if (doc.has("c_int")) { + int cInt = doc.get("c_int").asInt(); + return cInt >= 10 && cInt <= 20; + } + return false; + }, + // mapping document all field to string + JsonNode::toString); + Assertions.assertEquals(sinkData1.size(), index1Data.size()); + index1Data.forEach(sinkData1::remove); + // data is completely consistent, and the size is zero after deletion + Assertions.assertEquals(0, sinkData1.size()); + + List index2Data = + mapTestDatasetForDSL( + testDataset2, + // use customer predicate filter data to match sinkData2 + doc -> { + if (doc.has("c_int2")) { + int cInt = doc.get("c_int2").asInt(); + return cInt >= 10 && cInt <= 20; + } + return false; + }, + // mapping doc to string,keep only three fields + doc -> { + Map map = new HashMap<>(); + map.put("c_int2", doc.get("c_int2")); + map.put("c_null2", doc.get("c_null2")); + map.put("c_date2", doc.get("c_date2")); + return JsonUtils.toJsonString(map); + }); + + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(INDEX_REFRESH_MILL_DELAY)); + Set sinkData2 = + new HashSet<>( + getDocsWithTransformDate( + // read three fields from index + Lists.newArrayList("c_int2", "c_null2", "c_date2"), + "read_filter_index2_copy", + //// allowed c_null serialized if null + Lists.newArrayList("c_null2"), + query2, + // // transformDate field:c_date2 + Lists.newArrayList("c_date2"), + // order by c_int2 + "c_int2")); + Assertions.assertEquals(sinkData2.size(), index2Data.size()); + index2Data.forEach(sinkData2::remove); + Assertions.assertEquals(0, sinkData2.size()); + } + @DisabledOnContainer( value = {}, type = {EngineType.FLINK}, @@ -234,7 +354,7 @@ public void testElasticsearchWithFullType(TestContainer container) Container.ExecResult execResult = container.executeJob("/elasticsearch/elasticsearch_source_and_sink_full_type.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - Thread.sleep(2000L); + Thread.sleep(INDEX_REFRESH_MILL_DELAY); Assertions.assertEquals( 1, esRestClient.getIndexDocsCount("st_index_full_type_target").get(0).getDocsCount()); @@ -248,12 +368,12 @@ public void testElasticsearchWithoutSchema(TestContainer container) container.executeJob( "/elasticsearch/elasticsearch_source_without_schema_and_sink.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - List sinkData = readSinkDataWithOutSchema(); + List sinkData = readSinkDataWithOutSchema("st_index4"); // for DSL is: {"range":{"c_int":{"gte":10,"lte":20}}} Assertions.assertIterableEquals(mapTestDatasetForDSL(), sinkData); } - private List generateTestDataSet() throws JsonProcessingException { + private List generateTestDataSet1() throws JsonProcessingException { String[] fields = new String[] { "c_map", @@ -304,17 +424,83 @@ private List generateTestDataSet() throws JsonProcessingException { return documents; } - private List readSinkDataWithOutSchema() throws InterruptedException { + private List generateTestDataSet2() throws JsonProcessingException { + String[] fields = + new String[] { + "c_map2", + "c_array2", + "c_string2", + "c_boolean2", + "c_tinyint2", + "c_smallint2", + "c_bigint2", + "c_float2", + "c_double2", + "c_decimal2", + "c_bytes2", + "c_int2", + "c_date2", + "c_timestamp2", + "c_null2" + }; + + List documents = new ArrayList<>(); + ObjectMapper objectMapper = new ObjectMapper(); + for (int i = 0; i < 100; i++) { + Map doc = new HashMap<>(); + Object[] values = + new Object[] { + Collections.singletonMap("key2", Short.parseShort(String.valueOf(i))), + new Byte[] { + Byte.parseByte("11"), Byte.parseByte("22"), Byte.parseByte("33") + }, + "string2", + Boolean.FALSE, + Byte.parseByte("2"), + Short.parseShort("2"), + Long.parseLong("2"), + Float.parseFloat("2.2"), + Double.parseDouble("2.2"), + BigDecimal.valueOf(22, 1), + "test2".getBytes(), + i, + LocalDate.now().toString(), + System.currentTimeMillis(), + // Null values are also a basic use case for testing + null + }; + for (int j = 0; j < fields.length; j++) { + doc.put(fields[j], values[j]); + } + documents.add(objectMapper.writeValueAsString(doc)); + } + return documents; + } + + private List readSinkDataWithOutSchema(String indexName) throws InterruptedException { Map> esFieldType = - esRestClient.getFieldTypeMapping("st_index4", Lists.newArrayList()); - Thread.sleep(2000); + esRestClient.getFieldTypeMapping(indexName, Lists.newArrayList()); + Thread.sleep(INDEX_REFRESH_MILL_DELAY); List source = new ArrayList<>(esFieldType.keySet()); - return getDocsWithTransformDate(source, "st_index4"); + return getDocsWithTransformDate(source, indexName); } - private List readSinkData(String index) throws InterruptedException { + // Null values are also a basic use case for testing + // To ensure consistency in comparisons, we need to explicitly serialize null values. + private List readSinkDataWithOutSchema(String indexName, List nullAllowedFields) + throws InterruptedException { + Map> esFieldType = + esRestClient.getFieldTypeMapping(indexName, Lists.newArrayList()); + Thread.sleep(INDEX_REFRESH_MILL_DELAY); + List source = new ArrayList<>(esFieldType.keySet()); + return getDocsWithTransformDate(source, indexName, nullAllowedFields); + } + + // The timestamp type in Elasticsearch is incompatible with that in Seatunnel, + // and we need to handle the conversion here. + private List readSinkDataWithSchema(String index) throws InterruptedException { // wait for index refresh - Thread.sleep(2000); + Thread.sleep(INDEX_REFRESH_MILL_DELAY); List source = Lists.newArrayList( "c_map", @@ -338,7 +524,7 @@ private List readSinkData(String index) throws InterruptedException { private List readMultiSinkData(String index, List source) throws InterruptedException { // wait for index refresh - Thread.sleep(2000); + Thread.sleep(INDEX_REFRESH_MILL_DELAY); Map query = new HashMap<>(); query.put("match_all", Maps.newHashMap()); @@ -394,6 +580,19 @@ private List getDocsWithTransformTimestamp(List source, String i } private List getDocsWithTransformDate(List source, String index) { + return getDocsWithTransformDate(source, index, Collections.emptyList()); + } + + /** + * use default query: c_int >= 10 and c_int <=20 + * + * @param source The field to be read + * @param index indexName + * @param nullAllowedFields If the value of the field is null, it will be serialized to 'null' + * @return serialized data as jsonString + */ + private List getDocsWithTransformDate( + List source, String index, List nullAllowedFields) { HashMap rangeParam = new HashMap<>(); rangeParam.put("gte", 10); rangeParam.put("lte", 20); @@ -409,6 +608,11 @@ private List getDocsWithTransformDate(List source, String index) x.remove("_index"); x.remove("_type"); x.remove("_id"); + for (String field : nullAllowedFields) { + if (!x.containsKey(field)) { + x.put(field, null); + } + } x.replace( "c_date", LocalDate.parse( @@ -427,7 +631,75 @@ private List getDocsWithTransformDate(List source, String index) return docs; } + /** + * use customer query read data + * + * @param source The field to be read + * @param index read index + * @param nullAllowedFields If the value of the field is null, it will be serialized to 'null' + * @param query dls query + * @param dateFields dateField will format with yyyy-MM-dd'T'HH:mm + * @param orderField how to oder data + * @return serialized data as jsonString + */ + private List getDocsWithTransformDate( + List source, + String index, + List nullAllowedFields, + Map query, + List dateFields, + String orderField) { + ScrollResult scrollResult = esRestClient.searchByScroll(index, source, query, "1m", 1000); + scrollResult + .getDocs() + .forEach( + x -> { + x.remove("_index"); + x.remove("_type"); + x.remove("_id"); + for (String field : nullAllowedFields) { + if (!x.containsKey(field)) { + x.put(field, null); + } + } + for (String dateField : dateFields) { + if (x.containsKey(dateField)) { + x.replace( + dateField, + LocalDate.parse( + x.get(dateField).toString(), + DateTimeFormatter.ofPattern( + "yyyy-MM-dd'T'HH:mm")) + .toString()); + } + } + }); + List docs = + scrollResult.getDocs().stream() + .sorted( + Comparator.comparingInt( + o -> Integer.parseInt(o.get(orderField).toString()))) + .map(JsonUtils::toJsonString) + .collect(Collectors.toList()); + return docs; + } + + /** + * default testDataset1 + * + * @return testDataset1 as jsonString array + */ private List mapTestDatasetForDSL() { + return mapTestDatasetForDSL(testDataset1); + } + + /** + * default query filter,c_int >=10 and c_int <= 20 + * + * @param testDataset testDataset + * @return c_int >=10 and c_int <= 20 filtered data + */ + private List mapTestDatasetForDSL(List testDataset) { return testDataset.stream() .map(JsonUtils::parseObject) .filter( @@ -442,6 +714,25 @@ private List mapTestDatasetForDSL() { .collect(Collectors.toList()); } + /** + * Use custom filtering criteria to query data + * + * @param testDataset testDataset + * @param predicate customer query filter + * @param mapStrFunc mapping doc to string + * @return filtered data + */ + private List mapTestDatasetForDSL( + List testDataset, + Predicate predicate, + Function mapStrFunc) { + return testDataset.stream() + .map(JsonUtils::parseObject) + .filter(predicate) + .map(mapStrFunc) + .collect(Collectors.toList()); + } + @AfterEach @Override public void tearDown() { @@ -488,7 +779,7 @@ public void testCatalog() throws InterruptedException, JsonProcessingException { requestBody.append("\n"); } esRestClient.bulk(requestBody.toString()); - Thread.sleep(2000); // Wait for data to be indexed + Thread.sleep(INDEX_REFRESH_MILL_DELAY); // Wait for data to be indexed // Verify data exists List sourceFields = Arrays.asList("field1", "field2"); @@ -500,7 +791,7 @@ public void testCatalog() throws InterruptedException, JsonProcessingException { // Truncate the table elasticSearchCatalog.truncateTable(tablePath, false); - Thread.sleep(2000); // Wait for data to be indexed + Thread.sleep(INDEX_REFRESH_MILL_DELAY); // Wait for data to be indexed // Verify data is deleted scrollResult = esRestClient.searchByScroll("st_index3", sourceFields, query, "1m", 100); @@ -526,4 +817,18 @@ private List generateTestData() throws JsonProcessingException { } return data; } + + /** + * elastic query all dsl + * + * @return elastic query all dsl + */ + private Map queryAll() { + // "query": { + // "match_all": {} + // } + Map matchAll = new HashMap<>(); + matchAll.put("match_all", new HashMap<>()); + return matchAll; + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf new file mode 100644 index 00000000000..9ac7d3743aa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf @@ -0,0 +1,92 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + parallelism = 1 + job.mode = "BATCH" + #checkpoint.interval = 10000 +} + +source { + Elasticsearch { + hosts = ["https://elasticsearch:9200"] + username = "elastic" + password = "elasticsearch" + tls_verify_certificate = false + tls_verify_hostname = false + index_list = [ + { + index = "read_filter_index1" + query = {"range": {"c_int": {"gte": 10, "lte": 20}}} + source = [ + c_map, + c_array, + c_string, + c_boolean, + c_tinyint, + c_smallint, + c_bigint, + c_float, + c_double, + c_decimal, + c_bytes, + c_int, + c_date, + c_timestamp, + c_null + ] + array_column = { + c_array = "array" + } + } + { + index = "read_filter_index2" + query = {"range": {"c_int2": {"gte": 10, "lte": 20}}} + source = [ + c_int2, + c_null2, + c_date2 + ] + + } + + ] + + } +} + +transform { +} + +sink { + Elasticsearch { + hosts = ["https://elasticsearch:9200"] + username = "elastic" + password = "elasticsearch" + tls_verify_certificate = false + tls_verify_hostname = false + + index = "${table_name}_copy" + index_type = "st" + "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" + "data_save_mode"="APPEND_DATA" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java index 63789b0d281..6c3f449d785 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java @@ -81,18 +81,4 @@ public void close() { s3Client.shutdown(); } } - - public static void main(String[] args) { - S3Utils s3Utils = new S3Utils(); - s3Utils.uploadTestFiles( - "/Users/gaojun/workspace/seatunnel/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx", - "test/test/seatunnel/e2e.xlsx", - false); - - s3Utils.createDir("test/test1"); - s3Utils.uploadTestFiles( - "/Users/gaojun/workspace/seatunnel/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx", - "test/test1/seatunnel/e2e.xlsx", - false); - } } 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 50075629249..1957e1bd08e 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 @@ -59,8 +59,13 @@ public class HbaseIT extends TestSuiteBase implements TestResource { private static final String TABLE_NAME = "seatunnel_test"; + private static final String ASSIGN_CF_TABLE_NAME = "assign_cf_table"; + private static final String MULTI_TABLE_ONE_NAME = "hbase_sink_1"; + + private static final String MULTI_TABLE_TWO_NAME = "hbase_sink_2"; + private static final String FAMILY_NAME = "info"; private Connection hbaseConnection; @@ -81,9 +86,14 @@ public void startUp() throws Exception { // Create table for hbase sink test log.info("initial"); hbaseCluster.createTable(TABLE_NAME, Arrays.asList(FAMILY_NAME)); + // Create table for hbase assign cf table sink test hbaseCluster.createTable(ASSIGN_CF_TABLE_NAME, Arrays.asList("cf1", "cf2")); table = TableName.valueOf(TABLE_NAME); tableAssign = TableName.valueOf(ASSIGN_CF_TABLE_NAME); + + // Create table for hbase multi-table sink test + hbaseCluster.createTable(MULTI_TABLE_ONE_NAME, Arrays.asList(FAMILY_NAME)); + hbaseCluster.createTable(MULTI_TABLE_TWO_NAME, Arrays.asList(FAMILY_NAME)); } @AfterAll @@ -97,7 +107,11 @@ public void tearDown() throws Exception { @TestTemplate public void testHbaseSink(TestContainer container) throws IOException, InterruptedException { - fakeToHbase(container); + deleteData(table); + Container.ExecResult sinkExecResult = container.executeJob("/fake-to-hbase.conf"); + Assertions.assertEquals(0, sinkExecResult.getExitCode()); + ArrayList results = readData(table); + Assertions.assertEquals(results.size(), 5); Container.ExecResult sourceExecResult = container.executeJob("/hbase-to-assert.conf"); Assertions.assertEquals(0, sourceExecResult.getExitCode()); } @@ -251,6 +265,25 @@ public void testHbaseSinkAssignCfSink(TestContainer container) Assertions.assertEquals(cf2Count, 5); } + @DisabledOnContainer( + value = {}, + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK does not support multiple table write") + public void testHbaseMultiTableSink(TestContainer container) + throws IOException, InterruptedException { + TableName multiTable1 = TableName.valueOf(MULTI_TABLE_ONE_NAME); + TableName multiTable2 = TableName.valueOf(MULTI_TABLE_TWO_NAME); + deleteData(multiTable1); + deleteData(multiTable2); + Container.ExecResult sinkExecResult = + container.executeJob("/fake-to-hbase-with-multipletable.conf"); + Assertions.assertEquals(0, sinkExecResult.getExitCode()); + ArrayList results = readData(multiTable1); + Assertions.assertEquals(results.size(), 1); + results = readData(multiTable2); + Assertions.assertEquals(results.size(), 1); + } + @TestTemplate public void testHbaseSourceWithBatchQuery(TestContainer container) throws IOException, InterruptedException { @@ -318,4 +351,16 @@ private int countData(TableName table) throws IOException { scanner.close(); return count; } + + public ArrayList readData(TableName table) throws IOException { + 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); + } + scanner.close(); + return results; + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.conf new file mode 100644 index 00000000000..8972bf13249 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.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 { + # You can set engine configuration here + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + tables_configs = [ + { + schema = { + table = "hbase_sink_1" + fields { + name = STRING + c_string = STRING + c_double = DOUBLE + c_bigint = BIGINT + c_float = FLOAT + c_int = INT + c_smallint = SMALLINT + c_boolean = BOOLEAN + time = BIGINT + } + } + rows = [ + { + kind = INSERT + fields = ["label_1", "sink_1", 4.3, 200, 2.5, 2, 5, true, 1627529632356] + } + ] + }, + { + schema = { + table = "hbase_sink_2" + fields { + name = STRING + c_string = STRING + c_double = DOUBLE + c_bigint = BIGINT + c_float = FLOAT + c_int = INT + c_smallint = SMALLINT + c_boolean = BOOLEAN + } + } + rows = [ + { + kind = INSERT + fields = ["label_2", "sink_2", 4.3, 200, 2.5, 2, 5, true] + } + ] + } + ] + } +} + +sink { + Hbase { + zookeeper_quorum = "hbase_e2e:2181" + table = "${table_name}" + rowkey_column = ["name"] + family_name { + all_columns = info + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf new file mode 100644 index 00000000000..6dc7530b4bd --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf @@ -0,0 +1,129 @@ +# +# 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"] + 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 = 11 + }, + { + rule_type = MIN_ROW + rule_value = 11 + } + ], + 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 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index 7cd6be4fd95..920b26e1793 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -31,6 +31,10 @@ import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.iris.IrisCatalog; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceTableConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; +import org.apache.seatunnel.connectors.seatunnel.jdbc.utils.JdbcCatalogUtils; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -72,7 +76,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -93,7 +99,8 @@ public abstract class AbstractJdbcIT extends TestSuiteBase implements TestResour "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl()); + + driverUrl() + + " --no-check-certificate"); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; @@ -116,7 +123,7 @@ public abstract class AbstractJdbcIT extends TestSuiteBase implements TestResour protected URLClassLoader getUrlClassLoader() throws MalformedURLException { if (urlClassLoader == null) { urlClassLoader = - new URLClassLoader( + new InsecureURLClassLoader( new URL[] {new URL(driverUrl())}, AbstractJdbcIT.class.getClassLoader()); Thread.currentThread().setContextClassLoader(urlClassLoader); @@ -454,6 +461,43 @@ public void testCatalog() { } } + @Test + public void testCatalogWithCatalogUtils() throws SQLException, ClassNotFoundException { + if (StringUtils.isBlank(jdbcCase.getTablePathFullName())) { + return; + } + + List tablesConfig = new ArrayList<>(); + JdbcSourceTableConfig tableConfig = + JdbcSourceTableConfig.builder() + .query("SELECT * FROM " + jdbcCase.getSourceTable()) + .useSelectCount(false) + .build(); + tablesConfig.add(tableConfig); + Map tables = + JdbcCatalogUtils.getTables( + JdbcConnectionConfig.builder() + .url(jdbcCase.getJdbcUrl().replace(HOST, dbServer.getHost())) + .driverName(jdbcCase.getDriverClass()) + .username(jdbcCase.getUserName()) + .password(jdbcCase.getPassword()) + .build(), + tablesConfig); + Set tablePaths = tables.keySet(); + + tablePaths.forEach( + tablePath -> { + log.info( + "Expected: {} Actual: {}", + tablePath.getFullName(), + jdbcCase.getTablePathFullName()); + Assertions.assertTrue( + tablePath + .getFullName() + .equalsIgnoreCase(jdbcCase.getTablePathFullName())); + }); + } + protected Object[] toArrayResult(ResultSet resultSet, String[] fieldNames) throws SQLException, IOException { List result = new ArrayList<>(0); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java new file mode 100644 index 00000000000..fc8a169abd4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc; + +import javax.net.ssl.HttpsURLConnection; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.X509TrustManager; + +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLClassLoader; +import java.security.SecureRandom; +import java.security.cert.X509Certificate; + +public class InsecureURLClassLoader extends URLClassLoader { + public InsecureURLClassLoader(URL[] urls, ClassLoader parent) throws MalformedURLException { + super(urls, parent); + disableCertificateValidation(); + } + + private static void disableCertificateValidation() { + TrustManager[] trustAllCerts = + new TrustManager[] { + new X509TrustManager() { + public X509Certificate[] getAcceptedIssuers() { + return null; + } + + public void checkClientTrusted(X509Certificate[] certs, String authType) {} + + public void checkServerTrusted(X509Certificate[] certs, String authType) {} + } + }; + + try { + SSLContext sc = SSLContext.getInstance("SSL"); + sc.init(null, trustAllCerts, new SecureRandom()); + HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory()); + } catch (Exception e) { + e.printStackTrace(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java index 006d3d5f351..3dd7b64b95d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java @@ -57,4 +57,8 @@ public class JdbcCase { private String catalogDatabase; private String catalogSchema; private String catalogTable; + + // The full path of the table created when initializing data + // According to whether jdbc api supports setting + private String tablePathFullName; } 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/JdbcMysqlIT.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/JdbcMysqlIT.java index 6bd97863b25..1b148ce494d 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/JdbcMysqlIT.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/JdbcMysqlIT.java @@ -180,6 +180,7 @@ JdbcCase getJdbcCase() { .testData(testDataSet) .catalogDatabase(CATALOG_DATABASE) .catalogTable(MYSQL_SINK) + .tablePathFullName(MYSQL_DATABASE + "." + MYSQL_SOURCE) .build(); } 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 b7c4a54b59b..19f7f118f28 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 @@ -25,11 +25,15 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; +import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.commons.lang3.tuple.Pair; +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.OracleContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -77,6 +81,9 @@ public class JdbcOracleIT extends AbstractJdbcIT { + " VARCHAR_10_COL varchar2(10),\n" + " CHAR_10_COL char(10),\n" + " CLOB_COL clob,\n" + + " NUMBER_1 number(1),\n" + + " NUMBER_6 number(6),\n" + + " NUMBER_10 number(10),\n" + " NUMBER_3_SF_2_DP number(3, 2),\n" + " NUMBER_7_SF_N2_DP number(7, -2),\n" + " INTEGER_COL integer,\n" @@ -97,6 +104,9 @@ public class JdbcOracleIT extends AbstractJdbcIT { + " VARCHAR_10_COL varchar2(10),\n" + " CHAR_10_COL char(10),\n" + " CLOB_COL clob,\n" + + " NUMBER_1 number(1),\n" + + " NUMBER_6 number(6),\n" + + " NUMBER_10 number(10),\n" + " NUMBER_3_SF_2_DP number(3, 2),\n" + " NUMBER_7_SF_N2_DP number(7, -2),\n" + " INTEGER_COL integer,\n" @@ -115,6 +125,9 @@ public class JdbcOracleIT extends AbstractJdbcIT { "VARCHAR_10_COL", "CHAR_10_COL", "CLOB_COL", + "NUMBER_1", + "NUMBER_6", + "NUMBER_10", "NUMBER_3_SF_2_DP", "NUMBER_7_SF_N2_DP", "INTEGER_COL", @@ -148,6 +161,14 @@ public void testSampleDataFromColumnSuccess() throws Exception { dialect.sampleDataFromColumn(connection, table, "INTEGER_COL", 1, 1024); } + @TestTemplate + public void testOracleWithoutDecimalTypeNarrowing(TestContainer container) throws Exception { + Container.ExecResult execResult = + container.executeJob( + "/jdbc_oracle_source_to_sink_without_decimal_type_narrowing.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + @Override JdbcCase getJdbcCase() { Map containerEnv = new HashMap<>(); @@ -184,6 +205,8 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) + // oracle jdbc not support getTables/getCatalog/getSchema , is empty + .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } @@ -207,6 +230,9 @@ Pair> initTestData() { String.format("f%s", i), String.format("f%s", i), String.format("f%s", i), + 1, + i * 10, + i * 1000, BigDecimal.valueOf(1.1), BigDecimal.valueOf(2400), i, 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/JdbcOracleMultipleTablesIT.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/JdbcOracleMultipleTablesIT.java new file mode 100644 index 00000000000..2b0bd05d5aa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleMultipleTablesIT.java @@ -0,0 +1,307 @@ +/* + * 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; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +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.apache.commons.lang3.tuple.Pair; + +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.junit.jupiter.api.function.Executable; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.OracleContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.images.PullPolicy; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; +import org.testcontainers.utility.MountableFile; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Connection; +import java.sql.Date; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +@Slf4j +public class JdbcOracleMultipleTablesIT extends TestSuiteBase implements TestResource { + private static final String ORACLE_IMAGE = "gvenzl/oracle-xe:21-slim-faststart"; + private static final String ORACLE_NETWORK_ALIASES = "e2e_oracleDb"; + private static final int ORACLE_PORT = 1521; + private static final String USERNAME = "TESTUSER"; + private static final String PASSWORD = "testPassword"; + private static final String DATABASE = "XE"; + private static final String SCHEMA = USERNAME; + private static final Pair> TEST_DATASET = generateTestDataset(); + private static final List TABLES = Arrays.asList("TABLE1", "TABLE2"); + private static final List SOURCE_TABLES = + TABLES.stream().map(table -> SCHEMA + "." + table).collect(Collectors.toList()); + + private static final List SINK_TABLES = + TABLES.stream() + .map(table -> SCHEMA + "." + "SINK_" + table) + .collect(Collectors.toList()); + private static final String CREATE_TABLE_SQL = + "create table %s\n" + + "(\n" + + " VARCHAR_10_COL varchar2(10),\n" + + " CHAR_10_COL char(10),\n" + + " CLOB_COL clob,\n" + + " NUMBER_1 number(1),\n" + + " NUMBER_6 number(6),\n" + + " NUMBER_10 number(10),\n" + + " NUMBER_3_SF_2_DP number(3, 2),\n" + + " NUMBER_7_SF_N2_DP number(7, -2),\n" + + " INTEGER_COL integer,\n" + + " FLOAT_COL float(10),\n" + + " REAL_COL real,\n" + + " BINARY_FLOAT_COL binary_float,\n" + + " BINARY_DOUBLE_COL binary_double,\n" + + " DATE_COL date,\n" + + " TIMESTAMP_WITH_3_FRAC_SEC_COL timestamp(3),\n" + + " TIMESTAMP_WITH_LOCAL_TZ timestamp with local time zone,\n" + + " XML_TYPE_COL \"SYS\".\"XMLTYPE\"" + + ")"; + + private OracleContainer oracleContainer; + private Connection connection; + + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " + + "https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/12.2.0.1/ojdbc8-12.2.0.1.jar && wget https://repo1.maven.org/maven2/com/oracle/database/xml/xdb6/12.2.0.1/xdb6-12.2.0.1.jar && wget https://repo1.maven.org/maven2/com/oracle/database/xml/xmlparserv2/12.2.0.1/xmlparserv2-12.2.0.1.jar"); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + @BeforeAll + @Override + public void startUp() throws Exception { + DockerImageName imageName = DockerImageName.parse(ORACLE_IMAGE); + oracleContainer = + new OracleContainer(imageName) + .withUsername(USERNAME) + .withPassword(PASSWORD) + .withDatabaseName(SCHEMA) + .withCopyFileToContainer( + MountableFile.forClasspathResource("sql/oracle_init.sql"), + "/container-entrypoint-startdb.d/init.sql") + .withNetwork(NETWORK) + .withNetworkAliases(ORACLE_NETWORK_ALIASES) + .withExposedPorts(ORACLE_PORT) + .withImagePullPolicy((PullPolicy.alwaysPull())) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(ORACLE_IMAGE))); + + oracleContainer.setPortBindings( + Lists.newArrayList(String.format("%s:%s", ORACLE_PORT, ORACLE_PORT))); + + Startables.deepStart(Stream.of(oracleContainer)).join(); + + connection = oracleContainer.createConnection(""); + createTables(SOURCE_TABLES); + createTables(SINK_TABLES); + initSourceTablesData(); + } + + @DisabledOnContainer( + value = {}, + type = {EngineType.FLINK}, + disabledReason = "Currently FLINK do not support multiple tables") + @TestTemplate + public void testMysqlJdbcMultipleTableE2e(TestContainer container) + throws IOException, InterruptedException, SQLException { + clearSinkTables(); + + Container.ExecResult execResult = + container.executeJob("/jdbc_oracle_source_with_multiple_tables_to_sink.conf"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + + List asserts = + TABLES.stream() + .map( + (Function) + table -> + () -> + Assertions.assertIterableEquals( + query( + String.format( + "SELECT * FROM %s.%s order by INTEGER_COL asc", + SCHEMA, table)), + query( + String.format( + "SELECT * FROM %s.%s order by INTEGER_COL asc", + SCHEMA, + "SINK_" + table)))) + .collect(Collectors.toList()); + Assertions.assertAll(asserts); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + if (connection != null) { + connection.close(); + } + if (oracleContainer != null) { + oracleContainer.close(); + } + } + + private void createTables(List tables) throws SQLException { + try (Statement statement = connection.createStatement()) { + tables.forEach( + tableName -> { + try { + statement.execute(String.format(CREATE_TABLE_SQL, tableName)); + } catch (SQLException e) { + throw new RuntimeException(e); + } + }); + } + } + + private void initSourceTablesData() throws SQLException { + String columns = Arrays.stream(TEST_DATASET.getLeft()).collect(Collectors.joining(", ")); + String placeholders = + Arrays.stream(TEST_DATASET.getLeft()) + .map(f -> "?") + .collect(Collectors.joining(", ")); + for (String table : SOURCE_TABLES) { + String sql = + "INSERT INTO " + table + " (" + columns + " ) VALUES (" + placeholders + ")"; + try (PreparedStatement statement = connection.prepareStatement(sql)) { + for (SeaTunnelRow row : TEST_DATASET.getRight()) { + for (int i = 0; i < row.getArity(); i++) { + statement.setObject(i + 1, row.getField(i)); + } + statement.addBatch(); + } + statement.executeBatch(); + } + } + } + + private List> query(String sql) { + try (Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sql)) { + List> result = new ArrayList<>(); + int columnCount = resultSet.getMetaData().getColumnCount(); + while (resultSet.next()) { + ArrayList objects = new ArrayList<>(); + for (int i = 1; i <= columnCount; i++) { + objects.add(resultSet.getString(i)); + } + result.add(objects); + log.debug(String.format("Print query, sql: %s, data: %s", sql, objects)); + } + return result; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private void clearSinkTables() throws SQLException { + for (String table : SINK_TABLES) { + String sql = "truncate table " + table; + try (Statement statement = connection.createStatement()) { + statement.execute(sql); + } + } + } + + private static Pair> generateTestDataset() { + String[] fieldNames = + new String[] { + "VARCHAR_10_COL", + "CHAR_10_COL", + "CLOB_COL", + "NUMBER_1", + "NUMBER_6", + "NUMBER_10", + "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" + }; + List rows = new ArrayList<>(); + for (int i = 0; i < 2000; i++) { + SeaTunnelRow row = + new SeaTunnelRow( + new Object[] { + String.format("f%s", i), + String.format("f%s", i), + String.format("f%s", i), + 1, + i * 10, + i * 1000, + BigDecimal.valueOf(1.1), + BigDecimal.valueOf(2400), + i, + Float.parseFloat("2.2"), + Float.parseFloat("2.2"), + Float.parseFloat("22.2"), + Double.parseDouble("2.2"), + Date.valueOf(LocalDate.now()), + Timestamp.valueOf(LocalDateTime.now()), + Timestamp.valueOf(LocalDateTime.now()), + "SeaTunnel : E2E : Connector V2 : Oracle XMLType" + }); + rows.add(row); + } + + return Pair.of(fieldNames, rows); + } +} 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.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink.conf index d956894c340..4df8c7b9934 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink.conf @@ -30,7 +30,7 @@ source { url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" user = testUser password = testPassword - 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" + query = "SELECT VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" } @@ -46,7 +46,7 @@ sink { 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(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" } 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 index 8a0c8310443..1988b488721 100644 --- 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 @@ -31,7 +31,7 @@ source { 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" + query = "SELECT VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" } @@ -47,7 +47,7 @@ sink { 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(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" } 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 index ebebdb55051..4d01da5c72a 100644 --- 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 @@ -47,7 +47,7 @@ sink { 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(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" } 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 index d00ce9b6434..94a850fdd01 100644 --- 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 @@ -48,7 +48,7 @@ sink { 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(?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + query = "INSERT INTO E2E_TABLE_SINK (VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" } 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_without_decimal_type_narrowing.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_without_decimal_type_narrowing.conf new file mode 100644 index 00000000000..58e98f5deff --- /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_without_decimal_type_narrowing.conf @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + driver = oracle.jdbc.driver.OracleDriver + url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" + user = testUser + password = testPassword + decimal_type_narrowing = false + query = "SELECT NUMBER_1,NUMBER_6,NUMBER_10 FROM E2E_TABLE_SOURCE" + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 20000 + }, + { + rule_type = MIN_ROW + rule_value = 20000 + } + ], + field_rules = [ + { + field_name = NUMBER_1 + field_type = "decimal(1, 0)" + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = NUMBER_6 + field_type = "decimal(6, 0)" + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = NUMBER_10 + field_type = "decimal(10, 0)" + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } + +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf new file mode 100644 index 00000000000..22c66493134 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### 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_list = [ + { + table_path = "TESTUSER.TABLE1" + }, + { + table_path = "TESTUSER.TABLE2" + } + ] + 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 + database = XE + table = "TESTUSER.SINK_${table_name}" + generate_sink_sql = true + properties { + database.oracle.jdbc.timezoneAsRegion = "false" + } + } + +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java index 1d41c480c34..e34557b7394 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.commons.lang3.tuple.Pair; @@ -105,6 +106,7 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) + .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index f183c0c1932..dfa128f8134 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.ExceptionUtils; @@ -43,7 +44,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { private static final String HIVE_DATABASE = "default"; - private static final String HIVE_SOURCE = "e2e_table_source"; + private static final String HIVE_SOURCE = "hive_e2e_source_table"; private static final String HIVE_USERNAME = "root"; private static final String HIVE_PASSWORD = null; private static final int HIVE_PORT = 10000; @@ -94,6 +95,7 @@ JdbcCase getJdbcCase() { .sourceTable(HIVE_SOURCE) .createSql(CREATE_SQL) .configFile(CONFIG_FILE) + .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java index de5621899a4..d36b864215c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java @@ -180,6 +180,7 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) + .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java index f2b9097ffa7..cc6af23d6ef 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java @@ -127,6 +127,7 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) + .tablePathFullName(String.format("%s.%s", DM_DATABASE, DM_SOURCE)) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java index 388cf67ae9c..c90d91693f4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java @@ -114,7 +114,7 @@ void compareResult(String executeKey) {} @Override String driverUrl() { - return "https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar"; + return "https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar"; } @Override diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java index 9c98c29a7a7..1ea4e2b693f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java @@ -84,6 +84,7 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) + .tablePathFullName(GREENPLUM_SOURCE) .build(); } 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/JdbcXuguIT.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/JdbcXuguIT.java index 5fdae0ad939..186e23530a0 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/JdbcXuguIT.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/JdbcXuguIT.java @@ -152,6 +152,7 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) + .tablePathFullName(XUGU_DATABASE + "." + XUGU_SCHEMA + "." + XUGU_SOURCE) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml index fa2e1930cce..668747b9db1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml @@ -43,6 +43,12 @@ connector-cdc-mysql ${project.version} test + + + org.apache.kafka + kafka-clients + + org.apache.seatunnel 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 6e67aa021d1..ffc97f4dd33 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 @@ -19,9 +19,15 @@ 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.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.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.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -40,6 +46,7 @@ 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.protobuf.ProtobufDeserializationSchema; import org.apache.seatunnel.format.text.TextSerializationSchema; import org.apache.kafka.clients.admin.AdminClient; @@ -71,8 +78,13 @@ import lombok.extern.slf4j.Slf4j; +import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.math.BigDecimal; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Paths; import java.time.Duration; import java.time.LocalDate; import java.time.LocalDateTime; @@ -85,6 +97,7 @@ import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; import java.util.stream.Stream; @Slf4j @@ -126,7 +139,8 @@ public void startUp() throws Exception { "test_topic_source", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER); + DEFAULT_FIELD_DELIMITER, + null); generateTestData(serializer::serializeRow, 0, 100); } @@ -259,7 +273,8 @@ public void testSourceKafkaJsonToConsole(TestContainer container) "test_topic_json", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER); + DEFAULT_FIELD_DELIMITER, + null); generateTestData(row -> serializer.serializeRow(row), 0, 100); Container.ExecResult execResult = container.executeJob("/jsonFormatIT/kafka_source_json_to_console.conf"); @@ -274,7 +289,8 @@ public void testSourceKafkaJsonFormatErrorHandleWaySkipToConsole(TestContainer c "test_topic_error_message", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER); + DEFAULT_FIELD_DELIMITER, + null); generateTestData(row -> serializer.serializeRow(row), 0, 100); Container.ExecResult execResult = container.executeJob( @@ -290,7 +306,8 @@ public void testSourceKafkaJsonFormatErrorHandleWayFailToConsole(TestContainer c "test_topic_error_message", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER); + DEFAULT_FIELD_DELIMITER, + null); generateTestData(row -> serializer.serializeRow(row), 0, 100); Container.ExecResult execResult = container.executeJob( @@ -314,7 +331,8 @@ public void testSourceKafkaStartConfig(TestContainer container) "test_topic_group", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER); + DEFAULT_FIELD_DELIMITER, + null); generateTestData(row -> serializer.serializeRow(row), 100, 150); testKafkaGroupOffsetsToConsole(container); } @@ -331,7 +349,8 @@ public void testSourceKafkaStartConfigWithCommitOffset(TestContainer container) "test_topic_group_with_commit_offset", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER); + DEFAULT_FIELD_DELIMITER, + null); generateTestData(row -> serializer.serializeRow(row), 0, 100); testKafkaGroupOffsetsToConsoleWithCommitOffset(container); } @@ -456,7 +475,8 @@ public void testKafkaAvroToAssert(TestContainer container) "test_avro_topic", SEATUNNEL_ROW_TYPE, MessageFormat.AVRO, - DEFAULT_FIELD_DELIMITER); + DEFAULT_FIELD_DELIMITER, + null); int start = 0; int end = 100; generateTestData(row -> serializer.serializeRow(row), start, end); @@ -504,6 +524,260 @@ public void testKafkaAvroToAssert(TestContainer container) }); } + @TestTemplate + public void testFakeSourceToKafkaProtobufFormat(TestContainer container) + throws IOException, InterruptedException, URISyntaxException { + + // Execute the job and verify the exit code + Container.ExecResult execResult = + container.executeJob("/protobuf/fake_to_kafka_protobuf.conf"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + + // Define the SeaTunnelRowType for the address field + SeaTunnelRowType addressType = + new SeaTunnelRowType( + new String[] {"city", "state", "street"}, + new SeaTunnelDataType[] { + BasicType.STRING_TYPE, BasicType.STRING_TYPE, BasicType.STRING_TYPE + }); + + // Define the SeaTunnelRowType for the main schema + SeaTunnelRowType seaTunnelRowType = + new SeaTunnelRowType( + new String[] { + "c_int32", + "c_int64", + "c_float", + "c_double", + "c_bool", + "c_string", + "c_bytes", + "Address", + "attributes", + "phone_numbers" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.LONG_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.STRING_TYPE, + PrimitiveByteArrayType.INSTANCE, + addressType, + new MapType<>(BasicType.STRING_TYPE, BasicType.FLOAT_TYPE), + ArrayType.STRING_ARRAY_TYPE + }); + + // Parse the configuration file + String path = getTestConfigFile("/protobuf/fake_to_kafka_protobuf.conf"); + Config config = ConfigFactory.parseFile(new File(path)); + Config sinkConfig = config.getConfigList("sink").get(0); + + // Prepare the schema properties + Map schemaProperties = new HashMap<>(); + schemaProperties.put( + "protobuf_message_name", sinkConfig.getString("protobuf_message_name")); + schemaProperties.put("protobuf_schema", sinkConfig.getString("protobuf_schema")); + + // Build the table schema based on SeaTunnelRowType + TableSchema schema = + TableSchema.builder() + .columns( + Arrays.asList( + IntStream.range(0, seaTunnelRowType.getTotalFields()) + .mapToObj( + i -> + PhysicalColumn.of( + seaTunnelRowType + .getFieldName(i), + seaTunnelRowType + .getFieldType(i), + 0, + true, + null, + null)) + .toArray(PhysicalColumn[]::new))) + .build(); + + // Create the catalog table + CatalogTable catalogTable = + CatalogTable.of( + TableIdentifier.of("", "", "", "test"), + schema, + schemaProperties, + Collections.emptyList(), + "It is converted from RowType and only has column information."); + + // Initialize the Protobuf deserialization schema + ProtobufDeserializationSchema deserializationSchema = + new ProtobufDeserializationSchema(catalogTable); + + // Retrieve and verify Kafka rows + List kafkaRows = + getKafkaSTRow( + "test_protobuf_topic_fake_source", + value -> { + try { + return deserializationSchema.deserialize(value); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + Assertions.assertEquals(16, kafkaRows.size()); + + // Validate the contents of each row + kafkaRows.forEach( + row -> { + Assertions.assertInstanceOf(Integer.class, row.getField(0)); + Assertions.assertInstanceOf(Long.class, row.getField(1)); + Assertions.assertInstanceOf(Float.class, row.getField(2)); + Assertions.assertInstanceOf(Double.class, row.getField(3)); + Assertions.assertInstanceOf(Boolean.class, row.getField(4)); + Assertions.assertInstanceOf(String.class, row.getField(5)); + Assertions.assertInstanceOf(byte[].class, row.getField(6)); + Assertions.assertInstanceOf(SeaTunnelRow.class, row.getField(7)); + Assertions.assertInstanceOf(Map.class, row.getField(8)); + Assertions.assertInstanceOf(String[].class, row.getField(9)); + }); + } + + @TestTemplate + public void testKafkaProtobufToAssert(TestContainer container) + throws IOException, InterruptedException, URISyntaxException { + + String confFile = "/protobuf/kafka_protobuf_to_assert.conf"; + String path = getTestConfigFile(confFile); + Config config = ConfigFactory.parseFile(new File(path)); + Config sinkConfig = config.getConfigList("source").get(0); + ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(sinkConfig); + SeaTunnelRowType seaTunnelRowType = buildSeaTunnelRowType(); + + // Prepare schema properties + Map schemaProperties = new HashMap<>(); + schemaProperties.put( + "protobuf_message_name", sinkConfig.getString("protobuf_message_name")); + schemaProperties.put("protobuf_schema", sinkConfig.getString("protobuf_schema")); + + // Build the table schema + TableSchema schema = + TableSchema.builder() + .columns( + Arrays.asList( + IntStream.range(0, seaTunnelRowType.getTotalFields()) + .mapToObj( + i -> + PhysicalColumn.of( + seaTunnelRowType + .getFieldName(i), + seaTunnelRowType + .getFieldType(i), + 0, + true, + null, + null)) + .toArray(PhysicalColumn[]::new))) + .build(); + + // Create catalog table + CatalogTable catalogTable = + CatalogTable.of( + TableIdentifier.of("", "", "", "test"), + schema, + schemaProperties, + Collections.emptyList(), + "It is converted from RowType and only has column information."); + + // Initialize the Protobuf deserialization schema + ProtobufDeserializationSchema deserializationSchema = + new ProtobufDeserializationSchema(catalogTable); + + // Create serializer + DefaultSeaTunnelRowSerializer serializer = + DefaultSeaTunnelRowSerializer.create( + "test_protobuf_topic_fake_source", + seaTunnelRowType, + MessageFormat.PROTOBUF, + DEFAULT_FIELD_DELIMITER, + readonlyConfig); + + // Produce records to Kafka + IntStream.range(0, 20) + .forEach( + i -> { + try { + SeaTunnelRow originalRow = buildSeaTunnelRow(); + ProducerRecord producerRecord = + serializer.serializeRow(originalRow); + producer.send(producerRecord).get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Error sending Kafka message", e); + } + }); + + producer.flush(); + + // Execute the job and validate + Container.ExecResult execResult = container.executeJob(confFile); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + + // Retrieve and verify Kafka rows + List kafkaSTRow = + getKafkaSTRow( + "test_protobuf_topic_fake_source", + value -> { + try { + return deserializationSchema.deserialize(value); + } catch (IOException e) { + throw new RuntimeException("Error deserializing Kafka message", e); + } + }); + + // Prepare expected values for assertions + SeaTunnelRow expectedAddress = new SeaTunnelRow(3); + expectedAddress.setField(0, "city_value"); + expectedAddress.setField(1, "state_value"); + expectedAddress.setField(2, "street_value"); + + Map expectedAttributesMap = new HashMap<>(); + expectedAttributesMap.put("k1", 0.1F); + expectedAttributesMap.put("k2", 2.3F); + + String[] expectedPhoneNumbers = {"1", "2"}; + + // Assertions + Assertions.assertEquals(20, kafkaSTRow.size()); + kafkaSTRow.forEach( + row -> { + Assertions.assertAll( + "Verify row fields", + () -> Assertions.assertEquals(123, (int) row.getField(0)), + () -> Assertions.assertEquals(123123123123L, (long) row.getField(1)), + () -> Assertions.assertEquals(0.123f, (float) row.getField(2)), + () -> Assertions.assertEquals(0.123d, (double) row.getField(3)), + () -> Assertions.assertFalse((boolean) row.getField(4)), + () -> Assertions.assertEquals("test data", row.getField(5).toString()), + () -> + Assertions.assertArrayEquals( + new byte[] {1, 2, 3}, (byte[]) row.getField(6)), + () -> Assertions.assertEquals(expectedAddress, row.getField(7)), + () -> Assertions.assertEquals(expectedAttributesMap, row.getField(8)), + () -> + Assertions.assertArrayEquals( + expectedPhoneNumbers, (String[]) row.getField(9))); + }); + } + + public static String getTestConfigFile(String configFile) + throws FileNotFoundException, URISyntaxException { + URL resource = KafkaIT.class.getResource(configFile); + if (resource == null) { + throw new FileNotFoundException("Can't find config file: " + configFile); + } + return Paths.get(resource.toURI()).toString(); + } + public void testKafkaLatestToConsole(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = @@ -759,4 +1033,68 @@ interface ProducerRecordConverter { interface ConsumerRecordConverter { SeaTunnelRow convert(byte[] value); } + + private SeaTunnelRow buildSeaTunnelRow() { + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(10); + + Map attributesMap = new HashMap<>(); + attributesMap.put("k1", 0.1F); + attributesMap.put("k2", 2.3F); + + String[] phoneNumbers = {"1", "2"}; + byte[] byteVal = {1, 2, 3}; + + SeaTunnelRow address = new SeaTunnelRow(3); + address.setField(0, "city_value"); + address.setField(1, "state_value"); + address.setField(2, "street_value"); + + seaTunnelRow.setField(0, 123); + seaTunnelRow.setField(1, 123123123123L); + seaTunnelRow.setField(2, 0.123f); + seaTunnelRow.setField(3, 0.123d); + seaTunnelRow.setField(4, false); + seaTunnelRow.setField(5, "test data"); + seaTunnelRow.setField(6, byteVal); + seaTunnelRow.setField(7, address); + seaTunnelRow.setField(8, attributesMap); + seaTunnelRow.setField(9, phoneNumbers); + + return seaTunnelRow; + } + + private SeaTunnelRowType buildSeaTunnelRowType() { + SeaTunnelRowType addressType = + new SeaTunnelRowType( + new String[] {"city", "state", "street"}, + new SeaTunnelDataType[] { + BasicType.STRING_TYPE, BasicType.STRING_TYPE, BasicType.STRING_TYPE + }); + + return new SeaTunnelRowType( + new String[] { + "c_int32", + "c_int64", + "c_float", + "c_double", + "c_bool", + "c_string", + "c_bytes", + "Address", + "attributes", + "phone_numbers" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.LONG_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.STRING_TYPE, + PrimitiveByteArrayType.INSTANCE, + addressType, + new MapType<>(BasicType.STRING_TYPE, BasicType.FLOAT_TYPE), + ArrayType.STRING_ARRAY_TYPE + }); + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf index 8bc6d41cd3f..6ef14eed66c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf @@ -26,11 +26,11 @@ env { source { Kafka { bootstrap.servers = "kafka_e2e:9092" + consumer.group = "ogg_multi_group" table_list = [ { topic = "^test-ogg-sou.*" pattern = "true" - consumer.group = "ogg_multi_group" start_mode = earliest schema = { fields { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf new file mode 100644 index 00000000000..29a9985629b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf @@ -0,0 +1,100 @@ +# +# 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" + + # spark config + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + +} +source { + FakeSource { + parallelism = 1 + result_table_name = "fake" + row.num = 16 + schema = { + fields { + c_int32 = int + c_int64 = long + c_float = float + c_double = double + c_bool = boolean + c_string = string + c_bytes = bytes + + Address { + city = string + state = string + street = string + } + attributes = "map" + phone_numbers = "array" + } + } + } +} + +sink { + kafka { + topic = "test_protobuf_topic_fake_source" + bootstrap.servers = "kafkaCluster:9092" + format = protobuf + kafka.request.timeout.ms = 60000 +# semantics = EXACTLY_ONCE + kafka.config = { + acks = "all" + request.timeout.ms = 60000 + buffer.memory = 33554432 + } + protobuf_message_name = Person + protobuf_schema = """ + syntax = "proto3"; + + package org.apache.seatunnel.format.protobuf; + + option java_outer_classname = "ProtobufE2E"; + + message Person { + int32 c_int32 = 1; + int64 c_int64 = 2; + float c_float = 3; + double c_double = 4; + bool c_bool = 5; + string c_string = 6; + bytes c_bytes = 7; + + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip = 4; + } + + Address address = 8; + + map attributes = 9; + + repeated string phone_numbers = 10; + } + """ + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf new file mode 100644 index 00000000000..3375cd11d77 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf @@ -0,0 +1,177 @@ +# +# 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" + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + Kafka { + topic = "test_protobuf_topic_fake_source" + format = protobuf + protobuf_message_name = Person + protobuf_schema = """ + syntax = "proto3"; + + package org.apache.seatunnel.format.protobuf; + + option java_outer_classname = "ProtobufE2E"; + + message Person { + int32 c_int32 = 1; + int64 c_int64 = 2; + float c_float = 3; + double c_double = 4; + bool c_bool = 5; + string c_string = 6; + bytes c_bytes = 7; + + message Address { + string street = 1; + string city = 2; + string state = 3; + string zip = 4; + } + + Address address = 8; + + map attributes = 9; + + repeated string phone_numbers = 10; + } + """ + schema = { + fields { + c_int32 = int + c_int64 = long + c_float = float + c_double = double + c_bool = boolean + c_string = string + c_bytes = bytes + + Address { + city = string + state = string + street = string + } + attributes = "map" + phone_numbers = "array" + } + } + bootstrap.servers = "kafkaCluster:9092" + start_mode = "earliest" + result_table_name = "kafka_table" + } +} + +sink { + Assert { + source_table_name = "kafka_table" + rules = { + field_rules = [ + { + field_name = c_int32 + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_int64 + field_type = long + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_float + field_type = float + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_bool + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_bytes + field_type = bytes + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = attributes + field_type = "map" + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = phone_numbers + field_type = array + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java index 13dcd3d675e..700bf25f510 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java @@ -32,6 +32,7 @@ public class PaimonRecord { public Long pkId; public String name; + public Integer score; public String dt; public Timestamp oneTime; public Timestamp twoTime; 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 4b1d7dd86ce..0168cc8f534 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 @@ -508,6 +508,43 @@ public void testFakeSinkPaimonWithFullTypeAndReadWithFilter(TestContainer contai Assertions.assertEquals(0, readResult4.getExitCode()); } + @TestTemplate + public void testSinkPaimonTruncateTable(TestContainer container) throws Exception { + Container.ExecResult writeResult = + container.executeJob("/fake_sink_paimon_truncate_with_local_case1.conf"); + Assertions.assertEquals(0, writeResult.getExitCode()); + Container.ExecResult readResult = + container.executeJob("/fake_sink_paimon_truncate_with_local_case2.conf"); + Assertions.assertEquals(0, readResult.getExitCode()); + given().ignoreExceptions() + .await() + .atLeast(100L, TimeUnit.MILLISECONDS) + .atMost(30L, TimeUnit.SECONDS) + .untilAsserted( + () -> { + // copy paimon to local + container.executeExtraCommands(containerExtendedFactory); + List paimonRecords = + loadPaimonData("seatunnel_namespace10", TARGET_TABLE); + Assertions.assertEquals(2, paimonRecords.size()); + paimonRecords.forEach( + paimonRecord -> { + if (paimonRecord.getPkId() == 1) { + Assertions.assertEquals("Aa", paimonRecord.getName()); + } + if (paimonRecord.getPkId() == 2) { + Assertions.assertEquals("Bb", paimonRecord.getName()); + } + Assertions.assertEquals(200, paimonRecord.getScore()); + }); + List ids = + paimonRecords.stream() + .map(PaimonRecord::getPkId) + .collect(Collectors.toList()); + Assertions.assertFalse(ids.contains(3L)); + }); + } + protected final ContainerExtendedFactory containerExtendedFactory = container -> { if (isWindows) { @@ -568,7 +605,7 @@ private void extractFilesWin() { } private List loadPaimonData(String dbName, String tbName) throws Exception { - Table table = getTable(dbName, tbName); + FileStoreTable table = (FileStoreTable) getTable(dbName, tbName); ReadBuilder readBuilder = table.newReadBuilder(); TableScan.Plan plan = readBuilder.newScan().plan(); TableRead tableRead = readBuilder.newRead(); @@ -582,7 +619,12 @@ private List loadPaimonData(String dbName, String tbName) throws E try (RecordReader reader = tableRead.createReader(plan)) { reader.forEachRemaining( row -> { - result.add(new PaimonRecord(row.getLong(0), row.getString(1).toString())); + PaimonRecord paimonRecord = + new PaimonRecord(row.getLong(0), row.getString(1).toString()); + if (table.schema().fieldNames().contains("score")) { + paimonRecord.setScore(row.getInt(2)); + } + result.add(paimonRecord); log.info("key_id:" + row.getLong(0) + ", name:" + row.getString(1)); }); } @@ -611,7 +653,7 @@ private Identifier getIdentifier(String dbName, String tbName) { private Catalog getCatalog() { Options options = new Options(); if (isWindows) { - options.set("warehouse", "file://" + CATALOG_DIR_WIN); + options.set("warehouse", CATALOG_DIR_WIN); } else { options.set("warehouse", "file://" + CATALOG_DIR); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java index 259bc0128a5..e93f45d9139 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java @@ -34,6 +34,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.InternalRow; import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.table.source.TableRead; @@ -50,6 +51,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import static org.awaitility.Awaitility.given; @@ -204,4 +206,131 @@ public void testFakeCDCSinkPaimonWithHiveCatalogAndRead(TestContainer container) container.executeJob("/paimon_to_assert_with_hivecatalog.conf"); Assertions.assertEquals(0, readResult.getExitCode()); } + + @TestTemplate + public void testSinkPaimonHdfsTruncateTable(TestContainer container) throws Exception { + Container.ExecResult writeResult = + container.executeJob("/fake_sink_paimon_truncate_with_hdfs_case1.conf"); + Assertions.assertEquals(0, writeResult.getExitCode()); + Container.ExecResult readResult = + container.executeJob("/fake_sink_paimon_truncate_with_hdfs_case2.conf"); + Assertions.assertEquals(0, readResult.getExitCode()); + given().ignoreExceptions() + .await() + .atLeast(100L, TimeUnit.MILLISECONDS) + .atMost(180L, TimeUnit.SECONDS) + .untilAsserted( + () -> { + PaimonSinkConfig paimonSinkConfig = + new PaimonSinkConfig( + ReadonlyConfig.fromMap(PAIMON_SINK_PROPERTIES)); + PaimonCatalogLoader paimonCatalogLoader = + new PaimonCatalogLoader(paimonSinkConfig); + Catalog catalog = paimonCatalogLoader.loadCatalog(); + List paimonRecords = + loadPaimonData(catalog, "seatunnel_namespace11", "st_test"); + Assertions.assertEquals(2, paimonRecords.size()); + paimonRecords.forEach( + paimonRecord -> { + if (paimonRecord.getPkId() == 1) { + Assertions.assertEquals("Aa", paimonRecord.getName()); + } + if (paimonRecord.getPkId() == 2) { + Assertions.assertEquals("Bb", paimonRecord.getName()); + } + Assertions.assertEquals(200, paimonRecord.getScore()); + }); + List ids = + paimonRecords.stream() + .map(PaimonRecord::getPkId) + .collect(Collectors.toList()); + Assertions.assertFalse(ids.contains(3L)); + }); + } + + @TestTemplate + public void testSinkPaimonHiveTruncateTable(TestContainer container) throws Exception { + Container.ExecResult writeResult = + container.executeJob("/fake_sink_paimon_truncate_with_hive_case1.conf"); + Assertions.assertEquals(0, writeResult.getExitCode()); + Container.ExecResult readResult = + container.executeJob("/fake_sink_paimon_truncate_with_hive_case2.conf"); + Assertions.assertEquals(0, readResult.getExitCode()); + given().ignoreExceptions() + .await() + .atLeast(100L, TimeUnit.MILLISECONDS) + .atMost(180L, TimeUnit.SECONDS) + .untilAsserted( + () -> { + PaimonSinkConfig paimonSinkConfig = + new PaimonSinkConfig( + ReadonlyConfig.fromMap(PAIMON_SINK_PROPERTIES)); + PaimonCatalogLoader paimonCatalogLoader = + new PaimonCatalogLoader(paimonSinkConfig); + Catalog catalog = paimonCatalogLoader.loadCatalog(); + List paimonRecords = + loadPaimonData(catalog, "seatunnel_namespace12", "st_test"); + Assertions.assertEquals(2, paimonRecords.size()); + paimonRecords.forEach( + paimonRecord -> { + if (paimonRecord.getPkId() == 1) { + Assertions.assertEquals("Aa", paimonRecord.getName()); + } + if (paimonRecord.getPkId() == 2) { + Assertions.assertEquals("Bb", paimonRecord.getName()); + } + Assertions.assertEquals(200, paimonRecord.getScore()); + }); + List ids = + paimonRecords.stream() + .map(PaimonRecord::getPkId) + .collect(Collectors.toList()); + Assertions.assertFalse(ids.contains(3L)); + }); + } + + @TestTemplate + public void testSinkPaimonHiveTruncateTable1(TestContainer container) throws Exception { + PaimonSinkConfig paimonSinkConfig = + new PaimonSinkConfig(ReadonlyConfig.fromMap(PAIMON_SINK_PROPERTIES)); + PaimonCatalogLoader paimonCatalogLoader = new PaimonCatalogLoader(paimonSinkConfig); + Catalog catalog = paimonCatalogLoader.loadCatalog(); + List paimonRecords = + loadPaimonData(catalog, "seatunnel_namespace11", "st_test"); + Assertions.assertEquals(2, paimonRecords.size()); + paimonRecords.forEach( + paimonRecord -> { + if (paimonRecord.getPkId() == 1) { + Assertions.assertEquals("Aa", paimonRecord.getName()); + } + if (paimonRecord.getPkId() == 2) { + Assertions.assertEquals("Bb", paimonRecord.getName()); + } + Assertions.assertEquals(200, paimonRecord.getScore()); + }); + List ids = + paimonRecords.stream().map(PaimonRecord::getPkId).collect(Collectors.toList()); + Assertions.assertFalse(ids.contains(3L)); + } + + private List loadPaimonData(Catalog catalog, String dbName, String tbName) + throws Exception { + FileStoreTable table = (FileStoreTable) catalog.getTable(Identifier.create(dbName, tbName)); + ReadBuilder readBuilder = table.newReadBuilder(); + TableScan.Plan plan = readBuilder.newScan().plan(); + TableRead tableRead = readBuilder.newRead(); + List result = new ArrayList<>(); + try (RecordReader reader = tableRead.createReader(plan)) { + reader.forEachRemaining( + row -> { + PaimonRecord paimonRecord = + new PaimonRecord(row.getLong(0), row.getString(1).toString()); + if (table.schema().fieldNames().contains("score")) { + paimonRecord.setScore(row.getInt(2)); + } + result.add(paimonRecord); + }); + } + return result; + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf new file mode 100644 index 00000000000..92f6f5c6de2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf @@ -0,0 +1,80 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + } + ] + } +} + +sink { + Paimon { + warehouse = "hdfs:///tmp/paimon" + database = "seatunnel_namespace11" + table = "st_test" + paimon.hadoop.conf = { + fs.defaultFS = "hdfs://nameservice1" + dfs.nameservices = "nameservice1" + dfs.ha.namenodes.nameservice1 = "nn1,nn2" + dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" + dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" + dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + dfs.client.use.datanode.hostname = "true" + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf new file mode 100644 index 00000000000..1a5eac73229 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "Aa", 200] + }, + { + kind = INSERT + fields = [2, "Bb", 200] + } + ] + } +} + +sink { + Paimon { + warehouse = "hdfs:///tmp/paimon" + database = "seatunnel_namespace11" + table = "st_test" + data_save_mode=DROP_DATA + paimon.hadoop.conf = { + fs.defaultFS = "hdfs://nameservice1" + dfs.nameservices = "nameservice1" + dfs.ha.namenodes.nameservice1 = "nn1,nn2" + dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" + dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" + dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + dfs.client.use.datanode.hostname = "true" + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf new file mode 100644 index 00000000000..26e95870e3d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + } + ] + } +} + +sink { + Paimon { + warehouse = "hdfs:///tmp/paimon" + catalog_type="hive" + catalog_uri="thrift://hadoop04:9083" + database = "seatunnel_namespace12" + table = "st_test" + paimon.hadoop.conf = { + fs.defaultFS = "hdfs://nameservice1" + dfs.nameservices = "nameservice1" + dfs.ha.namenodes.nameservice1 = "nn1,nn2" + dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" + dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" + dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + dfs.client.use.datanode.hostname = "true" + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf new file mode 100644 index 00000000000..ef1e79b86e2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf @@ -0,0 +1,67 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "Aa", 200] + }, + { + kind = INSERT + fields = [2, "Bb", 200] + } + ] + } +} + +sink { + Paimon { + warehouse = "hdfs:///tmp/paimon" + catalog_type="hive" + catalog_uri="thrift://hadoop04:9083" + database = "seatunnel_namespace12" + table = "st_test" + data_save_mode=DROP_DATA + paimon.hadoop.conf = { + fs.defaultFS = "hdfs://nameservice1" + dfs.nameservices = "nameservice1" + dfs.ha.namenodes.nameservice1 = "nn1,nn2" + dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" + dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" + dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" + dfs.client.use.datanode.hostname = "true" + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf new file mode 100644 index 00000000000..e22474a06d5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf @@ -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. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + } + ] + } +} + +sink { + Paimon { + warehouse = "file:///tmp/paimon" + database = "seatunnel_namespace10" + table = "st_test" + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf new file mode 100644 index 00000000000..64cb24bc8ef --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf @@ -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. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = [1, "Aa", 200] + }, + { + kind = INSERT + fields = [2, "Bb", 200] + } + ] + } +} + +sink { + Paimon { + warehouse = "file:///tmp/paimon" + database = "seatunnel_namespace10" + table = "st_test" + data_save_mode=DROP_DATA + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml new file mode 100644 index 00000000000..042e528c5e2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml @@ -0,0 +1,68 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-qdrant-e2e + SeaTunnel : E2E : Connector V2 : Qdrant + + + + org.apache.seatunnel + seatunnel-guava + ${project.version} + optional + + + + io.qdrant + client + 1.11.0 + test + + + org.apache.seatunnel + connector-qdrant + ${project.version} + test + + + org.testcontainers + qdrant + 1.20.1 + test + + + + org.apache.seatunnel + connector-assert + ${project.version} + test + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java new file mode 100644 index 00000000000..21854a12cad --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java @@ -0,0 +1,145 @@ +/* + * 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.v2.qdrant; + +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.junit.DisabledOnContainer; + +import org.awaitility.Awaitility; +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.lifecycle.Startables; +import org.testcontainers.qdrant.QdrantContainer; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; + +import io.qdrant.client.QdrantClient; +import io.qdrant.client.QdrantGrpcClient; +import io.qdrant.client.grpc.Collections; +import io.qdrant.client.grpc.Points; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static io.qdrant.client.PointIdFactory.id; +import static io.qdrant.client.ValueFactory.value; +import static io.qdrant.client.VectorFactory.vector; +import static io.qdrant.client.VectorsFactory.namedVectors; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "SPARK and FLINK do not support vector types yet") +public class QdrantIT extends TestSuiteBase implements TestResource { + + private static final String ALIAS = "qdrante2e"; + private static final String SOURCE_COLLECTION = "source_collection"; + private static final String SINK_COLLECTION = "sink_collection"; + private static final String IMAGE = "qdrant/qdrant:latest"; + private QdrantContainer container; + private QdrantClient qdrantClient; + + @BeforeAll + @Override + public void startUp() throws Exception { + this.container = new QdrantContainer(IMAGE).withNetwork(NETWORK).withNetworkAliases(ALIAS); + Startables.deepStart(Stream.of(this.container)).join(); + Awaitility.given().ignoreExceptions().await().atMost(10L, TimeUnit.SECONDS); + this.initQdrant(); + this.initSourceData(); + } + + private void initQdrant() { + qdrantClient = + new QdrantClient( + QdrantGrpcClient.newBuilder( + container.getHost(), container.getGrpcPort(), false) + .build()); + } + + private void initSourceData() throws Exception { + qdrantClient + .createCollectionAsync( + SOURCE_COLLECTION, + ImmutableMap.of( + "my_vector", + Collections.VectorParams.newBuilder() + .setSize(4) + .setDistance(Collections.Distance.Cosine) + .build())) + .get(); + + qdrantClient + .createCollectionAsync( + SINK_COLLECTION, + ImmutableMap.of( + "my_vector", + Collections.VectorParams.newBuilder() + .setSize(4) + .setDistance(Collections.Distance.Cosine) + .build())) + .get(); + + List points = new ArrayList<>(); + for (int i = 1; i <= 10; i++) { + Points.PointStruct.Builder pointStruct = Points.PointStruct.newBuilder(); + pointStruct.setId(id(i)); + List floats = Arrays.asList((float) i, (float) i, (float) i, (float) i); + pointStruct.setVectors(namedVectors(ImmutableMap.of("my_vector", vector(floats)))); + + pointStruct.putPayload("file_size", value(i)); + pointStruct.putPayload("file_name", value("file-name-" + i)); + + points.add(pointStruct.build()); + } + + qdrantClient + .upsertAsync( + Points.UpsertPoints.newBuilder() + .setCollectionName(SOURCE_COLLECTION) + .addAllPoints(points) + .build()) + .get(); + } + + @AfterAll + @Override + public void tearDown() { + this.qdrantClient.close(); + } + + @TestTemplate + public void testQdrant(TestContainer container) + throws IOException, InterruptedException, ExecutionException { + Container.ExecResult execResult = container.executeJob("/qdrant-to-qdrant.conf"); + Assertions.assertEquals(execResult.getExitCode(), 0); + Assertions.assertEquals(qdrantClient.countAsync(SINK_COLLECTION).get(), 10); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf new file mode 100644 index 00000000000..8fa4c4f1da0 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Qdrant { + collection_name = "source_collection" + host = "qdrante2e" + schema = { + columns = [ + { + name = file_name + type = string + } + { + name = file_size + type = int + } + { + name = my_vector + type = float_vector + } + ] + } + } +} + +sink { + Qdrant { + collection_name = "sink_collection" + host = "qdrante2e" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java index 7052aa9bef8..a846949d857 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java @@ -75,6 +75,9 @@ public class RabbitmqIT extends TestSuiteBase implements TestResource { private static final String SINK_QUEUE_NAME = "test1"; private static final String USERNAME = "guest"; private static final String PASSWORD = "guest"; + private static final Boolean DURABLE = true; + private static final Boolean EXCLUSIVE = false; + private static final Boolean AUTO_DELETE = false; private static final Pair> TEST_DATASET = generateTestDataSet(); @@ -185,6 +188,9 @@ private void initRabbitMQ() { config.setVirtualHost("/"); config.setUsername(USERNAME); config.setPassword(PASSWORD); + config.setDurable(DURABLE); + config.setExclusive(EXCLUSIVE); + config.setAutoDelete(AUTO_DELETE); rabbitmqClient = new RabbitmqClient(config); } catch (Exception e) { throw new RuntimeException("init Rabbitmq error", e); @@ -201,6 +207,9 @@ private RabbitmqClient initSinkRabbitMQ() { config.setVirtualHost("/"); config.setUsername(USERNAME); config.setPassword(PASSWORD); + config.setDurable(DURABLE); + config.setExclusive(EXCLUSIVE); + config.setAutoDelete(AUTO_DELETE); return new RabbitmqClient(config); } catch (Exception e) { throw new RuntimeException("init Rabbitmq error", e); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf index b3a834bdc2f..61267a3adce 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf @@ -28,6 +28,9 @@ source { username = "guest" password = "guest" queue_name = "test" + durable = "true" + exclusive = "false" + auto_delete = "false" for_e2e_testing = true schema = { fields { @@ -61,6 +64,9 @@ sink { virtual_host = "/" username = "guest" password = "guest" + durable = "true" + exclusive = "false" + auto_delete = "false" queue_name = "test1" } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml new file mode 100644 index 00000000000..0a7243ed6a3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml @@ -0,0 +1,51 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-typesense-e2e + SeaTunnel : E2E : Connector V2 : Typesense + + + 8 + 8 + UTF-8 + + + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-typesense + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java new file mode 100644 index 00000000000..afff4972e4b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.typesense; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.apache.commons.lang3.RandomUtils; + +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; +import org.typesense.api.FieldTypes; +import org.typesense.model.Field; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class TypesenseIT extends TestSuiteBase implements TestResource { + + private static final String TYPESENSE_DOCKER_IMAGE = "typesense/typesense:26.0"; + + private static final String HOST = "e2e_typesense"; + + private static final int PORT = 8108; + + private GenericContainer typesenseServer; + + private TypesenseClient typesenseClient; + + private static final String sinkCollection = "typesense_test_collection"; + + private static final String sourceCollection = "typesense_test_collection_for_source"; + + @BeforeEach + @Override + public void startUp() throws Exception { + typesenseServer = + new GenericContainer<>(TYPESENSE_DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withPrivilegedMode(true) + .withStartupAttempts(5) + .withCommand("--data-dir=/", "--api-key=xyz") + .withStartupTimeout(Duration.ofMinutes(5)) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(TYPESENSE_DOCKER_IMAGE))); + typesenseServer.setPortBindings(Lists.newArrayList(String.format("%s:%s", PORT, PORT))); + Startables.deepStart(Stream.of(typesenseServer)).join(); + log.info("Typesense container started"); + Awaitility.given() + .ignoreExceptions() + .atLeast(1L, TimeUnit.SECONDS) + .pollInterval(1L, TimeUnit.SECONDS) + .atMost(120L, TimeUnit.SECONDS) + .untilAsserted(this::initConnection); + } + + private void initConnection() { + String host = typesenseServer.getContainerIpAddress(); + typesenseClient = + TypesenseClient.createInstance(Lists.newArrayList(host + ":8108"), "xyz", "http"); + } + + /** Test setting primary_keys parameter write Typesense */ + @TestTemplate + public void testFakeToTypesenseWithPrimaryKeys(TestContainer container) throws Exception { + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_primary_keys.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + } + + @TestTemplate + public void testFakeToTypesenseWithRecreateSchema(TestContainer container) throws Exception { + List fields = new ArrayList<>(); + fields.add(new Field().name("T").type(FieldTypes.BOOL)); + Assertions.assertTrue(typesenseClient.createCollection(sinkCollection, fields)); + Map field = typesenseClient.getField(sinkCollection); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_recreate_schema.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + Assertions.assertNotEquals(field, typesenseClient.getField(sinkCollection)); + } + + @TestTemplate + public void testFakeToTypesenseWithErrorWhenNotExists(TestContainer container) + throws Exception { + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_error_when_not_exists.conf"); + Assertions.assertEquals(1, execResult.getExitCode()); + } + + @TestTemplate + public void testFakeToTypesenseWithCreateWhenNotExists(TestContainer container) + throws Exception { + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_create_when_not_exists.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + } + + @TestTemplate + public void testFakeToTypesenseWithDropData(TestContainer container) throws Exception { + String initData = "{\"name\":\"Han\",\"age\":12}"; + typesenseClient.createCollection(sinkCollection); + typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_drop_data.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); + } + + @TestTemplate + public void testFakeToTypesenseWithAppendData(TestContainer container) throws Exception { + String initData = "{\"name\":\"Han\",\"age\":12}"; + typesenseClient.createCollection(sinkCollection); + typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_append_data.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 6); + } + + @TestTemplate + public void testFakeToTypesenseWithErrorWhenDataExists(TestContainer container) + throws Exception { + String initData = "{\"name\":\"Han\",\"age\":12}"; + typesenseClient.createCollection(sinkCollection); + typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); + Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); + Container.ExecResult execResult = + container.executeJob("/fake_to_typesense_with_error_when_data_exists.conf"); + Assertions.assertEquals(1, execResult.getExitCode()); + } + + public List genTestData(int recordNum) { + ArrayList testDataList = new ArrayList<>(); + ObjectMapper objectMapper = new ObjectMapper(); + HashMap doc = new HashMap<>(); + for (int i = 0; i < recordNum; i++) { + try { + doc.put("num_employees", RandomUtils.nextInt()); + doc.put("flag", RandomUtils.nextBoolean()); + doc.put("num", RandomUtils.nextLong()); + doc.put("company_name", "A" + RandomUtils.nextInt(1, 100)); + testDataList.add(objectMapper.writeValueAsString(doc)); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + return testDataList; + } + + @TestTemplate + public void testTypesenseSourceAndSink(TestContainer container) throws Exception { + int recordNum = 100; + List testData = genTestData(recordNum); + typesenseClient.createCollection(sourceCollection); + typesenseClient.insert(sourceCollection, testData); + Assertions.assertEquals( + typesenseClient.search(sourceCollection, null, 0).getFound(), recordNum); + Container.ExecResult execResult = container.executeJob("/typesense_source_and_sink.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals( + typesenseClient.search(sinkCollection, null, 0).getFound(), recordNum); + } + + @TestTemplate + public void testTypesenseToTypesense(TestContainer container) throws Exception { + String typesenseToTypesenseSource = "typesense_to_typesense_source"; + String typesenseToTypesenseSink = "typesense_to_typesense_sink"; + List testData = new ArrayList<>(); + testData.add( + "{\"c_row\":{\"c_array_int\":[12,45,96,8],\"c_int\":91,\"c_string\":\"String_412\"},\"company_name\":\"Company_9986\",\"company_name_list\":[\"Company_9986_Alias_1\",\"Company_9986_Alias_2\"],\"country\":\"Country_181\",\"id\":\"9986\",\"num_employees\":1914}"); + testData.add( + "{\"c_row\":{\"c_array_int\":[60],\"c_int\":9,\"c_string\":\"String_371\"},\"company_name\":\"Company_9988\",\"company_name_list\":[\"Company_9988_Alias_1\",\"Company_9988_Alias_2\",\"Company_9988_Alias_3\"],\"country\":\"Country_86\",\"id\":\"9988\",\"num_employees\":7366}"); + typesenseClient.createCollection(typesenseToTypesenseSource); + typesenseClient.insert(typesenseToTypesenseSource, testData); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSource, null, 0).getFound(), 2); + Container.ExecResult execResult = container.executeJob("/typesense_to_typesense.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSink, null, 0).getFound(), 2); + ObjectMapper objectMapper = new ObjectMapper(); + Map sourceData = objectMapper.readValue(testData.get(0), Map.class); + Map sinkData = + typesenseClient + .search(typesenseToTypesenseSink, null, 0) + .getHits() + .get(0) + .getDocument(); + Assertions.assertNotEquals(sourceData.remove("id"), sinkData.remove("id")); + Assertions.assertEquals(sourceData, sinkData); + } + + @TestTemplate + public void testTypesenseToTypesenseWithQuery(TestContainer container) throws Exception { + String typesenseToTypesenseSource = "typesense_to_typesense_source_with_query"; + String typesenseToTypesenseSink = "typesense_to_typesense_sink_with_query"; + List testData = new ArrayList<>(); + testData.add( + "{\"c_row\":{\"c_array_int\":[12,45,96,8],\"c_int\":91,\"c_string\":\"String_412\"},\"company_name\":\"Company_9986\",\"company_name_list\":[\"Company_9986_Alias_1\",\"Company_9986_Alias_2\"],\"country\":\"Country_181\",\"id\":\"9986\",\"num_employees\":1914}"); + testData.add( + "{\"c_row\":{\"c_array_int\":[60],\"c_int\":9,\"c_string\":\"String_371\"},\"company_name\":\"Company_9988\",\"company_name_list\":[\"Company_9988_Alias_1\",\"Company_9988_Alias_2\",\"Company_9988_Alias_3\"],\"country\":\"Country_86\",\"id\":\"9988\",\"num_employees\":7366}"); + testData.add( + "{\"c_row\":{\"c_array_int\":[18,97],\"c_int\":32,\"c_string\":\"String_48\"},\"company_name\":\"Company_9880\",\"company_name_list\":[\"Company_9880_Alias_1\",\"Company_9880_Alias_2\",\"Company_9880_Alias_3\",\"Company_9880_Alias_4\"],\"country\":\"Country_159\",\"id\":\"9880\",\"num_employees\":141}"); + typesenseClient.createCollection(typesenseToTypesenseSource); + typesenseClient.insert(typesenseToTypesenseSource, testData); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSource, null, 0).getFound(), 3); + Container.ExecResult execResult = + container.executeJob("/typesense_to_typesense_with_query.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + Assertions.assertEquals( + typesenseClient.search(typesenseToTypesenseSink, null, 0).getFound(), 2); + } + + @AfterEach + @Override + public void tearDown() { + typesenseServer.close(); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf new file mode 100644 index 00000000000..ab1c7b171dc --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf new file mode 100644 index 00000000000..78870438e0a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf new file mode 100644 index 00000000000..01094580a62 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + api_key = "xyz" + max_retry_count = 3 + max_batch_size = 10 + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "DROP_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf new file mode 100644 index 00000000000..6496ceb979c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "ERROR_WHEN_DATA_EXISTS" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf new file mode 100644 index 00000000000..c3538c846a8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "ERROR_WHEN_SCHEMA_NOT_EXIST" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf new file mode 100644 index 00000000000..2a767db02d2 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf new file mode 100644 index 00000000000..ee7acce8a4b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf @@ -0,0 +1,51 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + FakeSource { + row.num = 5 + result_table_name = "typesense_test_table" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "RECREATE_SCHEMA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf new file mode 100644 index 00000000000..25e63cf2443 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" +} + +source { + Typesense { + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection_for_source" + api_key = "xyz" + schema { + fields { + company_name = string + num = long + id = string + num_employees = int + flag = boolean + } + } + result_table_name = "typesense_test_table" + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_test_collection" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","num"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "DROP_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf new file mode 100644 index 00000000000..f8c148a7afa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf @@ -0,0 +1,64 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" + flink.execution.checkpointing.interval=5000 + flink.execution.restart.strategy = failure-rate + flink.execution.restart.failureInterval = 60000 + flink.execution.restart.failureRate = 100 + flink.execution.restart.delayInterval = 10000 + +} +source { + Typesense { + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_source" + api_key = "xyz" + result_table_name = "typesense_test_table" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_sink" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf new file mode 100644 index 00000000000..7b069c90793 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + parallelism = 1 + job.mode = "BATCH" + flink.execution.checkpointing.interval=5000 + flink.execution.restart.strategy = failure-rate + flink.execution.restart.failureInterval = 60000 + flink.execution.restart.failureRate = 100 + flink.execution.restart.delayInterval = 10000 + +} +source { + Typesense { + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_source_with_query" + api_key = "xyz" + query = "q=*&filter_by=c_row.c_int:>10" + result_table_name = "typesense_test_table" + schema = { + fields { + company_name_list = array + company_name = string + num_employees = long + country = string + id = string + c_row = { + c_int = int + c_string = string + c_array_int = array + } + } + } + } +} + +sink { + Typesense { + source_table_name = "typesense_test_table" + hosts = ["e2e_typesense:8108"] + collection = "typesense_to_typesense_sink_with_query" + max_retry_count = 3 + max_batch_size = 10 + api_key = "xyz" + primary_keys = ["num_employees","id"] + key_delimiter = "=" + schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" + data_save_mode = "APPEND_DATA" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 4933ab02057..28be63f3cf0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -76,8 +76,11 @@ connector-hudi-e2e connector-milvus-e2e connector-activemq-e2e + connector-qdrant-e2e connector-sls-e2e + connector-typesense-e2e connector-email-e2e + connector-cdc-opengauss-e2e diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties b/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties index 51ace38473a..c3681980c72 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties @@ -29,7 +29,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -39,7 +39,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java index 8389cb3c058..8e45bbf9de5 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java @@ -109,7 +109,7 @@ public void testEventReport() throws IOException, InterruptedException { arrayNode.elements().forEachRemaining(jsonNode -> events.add(jsonNode)); } } - Assertions.assertEquals(8, events.size()); + Assertions.assertEquals(10, events.size()); Set eventTypes = events.stream().map(e -> e.get("eventType").asText()).collect(Collectors.toSet()); Assertions.assertTrue( diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml index 973fe6434c7..6fbdfc826bf 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml @@ -107,6 +107,16 @@ hadoop-aliyun ${hadoop-aliyun.version} test + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java index ac29b4bf355..f48ca3f1817 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java @@ -992,7 +992,6 @@ public void testStreamJobRestoreFromOssInAllNodeDown() throws Exception { + " fs.oss.endpoint: " + OSS_ENDPOINT + "\n" - + " fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider\n" + " properties:\n" + " hazelcast.invocation.max.retry.count: 200\n" + " hazelcast.tcp.join.port.try.count: 30\n" diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java index bf16cab75d2..894e901596f 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.engine.server.rest.RestConstant; import org.awaitility.Awaitility; +import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -32,15 +33,21 @@ import org.testcontainers.utility.DockerLoggerFactory; import org.testcontainers.utility.MountableFile; +import com.hazelcast.jet.json.JsonUtil; import io.restassured.response.Response; import java.io.IOException; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; import static io.restassured.RestAssured.given; import static org.apache.seatunnel.e2e.common.util.ContainerUtil.PROJECT_ROOT_PATH; @@ -113,6 +120,24 @@ public void testSubmitJobWithCustomJobId() { i, paramJobName + "&jobId=" + CUSTOM_JOB_ID, true)); + + String serverLogs = server.getLogs(); + String secondServerLogs = secondServer.getLogs(); + Stream.of( + // [862969647010611201] 2024-08-24 16:01:21,155 INFO + // org.apache.seatunnel.connectors.seatunnel.fake.source.FakeSourceSplitEnumerator - Starting to calculate splits. + "\\[862969647010611201\\].* INFO org\\.apache\\.seatunnel\\.connectors\\.seatunnel\\.fake\\.source\\.FakeSourceSplitEnumerator", + // [862969647010611201] 2024-08-24 16:01:21,278 INFO + // org.apache.seatunnel.connectors.seatunnel.console.sink.ConsoleSinkWriter + // - subtaskIndex=0 rowIndex=63: SeaTunnelRow#tableId=fake + // SeaTunnelRow#kind=INSERT : qOWCd, 1033892054, 671516661 + "\\[862969647010611201\\].* INFO org\\.apache\\.seatunnel\\.connectors\\.seatunnel\\.console\\.sink\\.ConsoleSinkWriter") + .map( + regex -> { + Assertions.assertTrue( + serverLogs.matches(regex) || secondServerLogs.matches(regex)); + return regex; + }); } @Test @@ -269,6 +294,167 @@ private Response submitJob( return submitJob(jobMode, container, false, jobName, paramJobName); } + @Test + public void testStopJobs() { + Arrays.asList(server) + .forEach( + container -> { + try { + submitJobs("STREAMING", container, false, CUSTOM_JOB_ID); + + String parameters = + "[{\"jobId\":" + + CUSTOM_JOB_ID + + ",\"isStopWithSavePoint\":false},{\"jobId\":" + + (CUSTOM_JOB_ID - 1) + + ",\"isStopWithSavePoint\":false}]"; + + given().body(parameters) + .post( + http + + container.getHost() + + colon + + container.getFirstMappedPort() + + RestConstant.STOP_JOBS_URL) + .then() + .statusCode(200) + .body("[0].jobId", equalTo(CUSTOM_JOB_ID)) + .body("[1].jobId", equalTo(CUSTOM_JOB_ID - 1)); + + Awaitility.await() + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> + given().get( + http + + container + .getHost() + + colon + + container + .getFirstMappedPort() + + RestConstant + .FINISHED_JOBS_INFO + + "/CANCELED") + .then() + .statusCode(200) + .body( + "[0].jobId", + equalTo( + String.valueOf( + CUSTOM_JOB_ID))) + .body( + "[0].jobId", + equalTo( + String.valueOf( + CUSTOM_JOB_ID + - 1)))); + + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testSubmitJobs() { + AtomicInteger i = new AtomicInteger(); + Arrays.asList(server, secondServer) + .forEach( + container -> { + try { + submitJobs("BATCH", container, false, CUSTOM_JOB_ID); + submitJobs("BATCH", container, true, CUSTOM_JOB_ID); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private void submitJobs( + String jobMode, GenericContainer container, boolean isStartWithSavePoint, Long jobId) + throws IOException { + + String requestBody = getJobJson(jobMode, isStartWithSavePoint, jobId); + + Response response = + given().body(requestBody) + .header("Content-Type", "application/json; charset=utf-8") + .post( + http + + container.getHost() + + colon + + container.getFirstMappedPort() + + RestConstant.SUBMIT_JOBS_URL); + + response.then() + .statusCode(200) + .body("[0].jobId", equalTo(String.valueOf(jobId))) + .body("[1].jobId", equalTo(String.valueOf(jobId - 1))); + + Response jobInfoResponse = + given().header("Content-Type", "application/json; charset=utf-8") + .get( + http + + container.getHost() + + colon + + container.getFirstMappedPort() + + RestConstant.JOB_INFO_URL + + "/" + + jobId); + jobInfoResponse.then().statusCode(200).body("jobStatus", equalTo("RUNNING")); + } + + private static @NotNull String getJobJson( + String jobMode, boolean isStartWithSavePoint, Long jobId) throws IOException { + List> jobList = new ArrayList<>(); + for (int i = 0; i < 2; i++) { + Map job = new HashMap<>(); + Map params = new HashMap<>(); + params.put("jobId", String.valueOf(jobId - i)); + if (isStartWithSavePoint) { + params.put("isStartWithSavePoint", "true"); + } + job.put("params", params); + + Map env = new HashMap<>(); + env.put("job.mode", jobMode); + job.put("env", env); + + List> sourceList = new ArrayList<>(); + Map source = new HashMap<>(); + source.put("plugin_name", "FakeSource"); + source.put("result_table_name", "fake"); + source.put("row.num", 1000); + + Map schema = new HashMap<>(); + Map fields = new HashMap<>(); + fields.put("name", "string"); + fields.put("age", "int"); + fields.put("card", "int"); + schema.put("fields", fields); + source.put("schema", schema); + + sourceList.add(source); + job.put("source", sourceList); + + List> transformList = new ArrayList<>(); + job.put("transform", transformList); + + List> sinkList = new ArrayList<>(); + Map sink = new HashMap<>(); + sink.put("plugin_name", "Console"); + List sourceTableName = new ArrayList<>(); + sourceTableName.add("fake"); + sink.put("source_table_name", sourceTableName); + + sinkList.add(sink); + job.put("sink", sinkList); + + jobList.add(job); + } + return JsonUtil.toJson(jobList); + } + private Response submitJob( String jobMode, GenericContainer container, diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java new file mode 100644 index 00000000000..63736a90ae5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java @@ -0,0 +1,161 @@ +/* + * 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.resourceIsolation; + +import org.apache.seatunnel.engine.common.config.ConfigProvider; +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.e2e.TestUtils; +import org.apache.seatunnel.engine.server.SeaTunnelServer; +import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; +import org.apache.seatunnel.engine.server.resourcemanager.ResourceManager; + +import org.awaitility.Awaitility; +import org.awaitility.core.ThrowingRunnable; +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.config.Config; +import com.hazelcast.instance.impl.HazelcastInstanceImpl; +import com.hazelcast.spi.impl.NodeEngineImpl; +import lombok.extern.slf4j.Slf4j; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +@Slf4j +public class WorkerTagClusterTest { + + HazelcastInstanceImpl masterNode1 = null; + HazelcastInstanceImpl workerNode1 = null; + String testClusterName = "WorkerTagClusterTest"; + + @BeforeEach + public void before() { + SeaTunnelConfig masterNode1Config = getSeaTunnelConfig(testClusterName); + SeaTunnelConfig workerNode1Config = getSeaTunnelConfig(testClusterName); + masterNode1 = SeaTunnelServerStarter.createMasterHazelcastInstance(masterNode1Config); + workerNode1 = SeaTunnelServerStarter.createWorkerHazelcastInstance(workerNode1Config); + } + + @AfterEach + void afterClass() { + if (masterNode1 != null) { + masterNode1.shutdown(); + } + if (workerNode1 != null) { + workerNode1.shutdown(); + } + } + + @Test + public void testTagMatch() throws Exception { + Map tag = new HashMap<>(); + tag.put("group", "platform"); + tag.put("team", "team1"); + testTagFilter(tag, 1); + } + + @Test + public void testTagMatch2() throws Exception { + testTagFilter(null, 1); + } + + @Test + public void testTagNotMatch() throws Exception { + Map tag = new HashMap<>(); + tag.put("group", "platform"); + tag.put("team", "team1111111"); + testTagFilter(tag, 0); + } + + @Test + public void testTagNotMatch2() throws Exception { + testTagFilter(new HashMap<>(), 1); + } + + public void testTagFilter(Map tagFilter, int expectedWorkerCount) + throws Exception { + // waiting all node added to cluster + Awaitility.await() + .atMost(10000, TimeUnit.MILLISECONDS) + .untilAsserted( + new ThrowingRunnable() { + @Override + public void run() throws Throwable { + Thread.sleep(2000); + // check master and worker node + Assertions.assertEquals( + 2, masterNode1.getCluster().getMembers().size()); + NodeEngineImpl nodeEngine = masterNode1.node.nodeEngine; + SeaTunnelServer server = + nodeEngine.getService(SeaTunnelServer.SERVICE_NAME); + ResourceManager resourceManager = + server.getCoordinatorService().getResourceManager(); + // if tag matched, then worker count is 1 else 0 + int workerCount = resourceManager.workerCount(tagFilter); + Assertions.assertEquals(expectedWorkerCount, workerCount); + } + }); + } + + private static SeaTunnelConfig getSeaTunnelConfig(String testClusterName) { + Config hazelcastConfig = Config.loadFromString(getHazelcastConfig()); + hazelcastConfig.setClusterName(TestUtils.getClusterName(testClusterName)); + SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); + seaTunnelConfig.setHazelcastConfig(hazelcastConfig); + return seaTunnelConfig; + } + + protected static String getHazelcastConfig() { + return "hazelcast:\n" + + " cluster-name: seatunnel\n" + + " network:\n" + + " rest-api:\n" + + " enabled: true\n" + + " endpoint-groups:\n" + + " CLUSTER_WRITE:\n" + + " enabled: true\n" + + " join:\n" + + " tcp-ip:\n" + + " enabled: true\n" + + " member-list:\n" + + " - localhost\n" + + " port:\n" + + " auto-increment: true\n" + + " port-count: 100\n" + + " port: 5801\n" + + "\n" + + " properties:\n" + + " hazelcast.invocation.max.retry.count: 200\n" + + " hazelcast.tcp.join.port.try.count: 30\n" + + " hazelcast.invocation.retry.pause.millis: 2000\n" + + " hazelcast.slow.operation.detector.stacktrace.logging.enabled: true\n" + + " hazelcast.logging.type: log4j2\n" + + " hazelcast.operation.generic.thread.count: 200\n" + + " member-attributes:\n" + + " group:\n" + + " type: string\n" + + " value: platform\n" + + " team:\n" + + " type: string\n" + + " value: team1"; + } +} diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties index f0090af0248..05ffd76520a 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties index fc0c2063b7a..030095bda72 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties @@ -33,7 +33,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -43,7 +43,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties index f0090af0248..05ffd76520a 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties index fc0c2063b7a..030095bda72 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties @@ -33,7 +33,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -43,7 +43,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties index f0090af0248..05ffd76520a 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties index fc0c2063b7a..030095bda72 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties @@ -33,7 +33,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -43,7 +43,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties index f0090af0248..05ffd76520a 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties index 3b1f20dd67e..ffc08447233 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties @@ -44,7 +44,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -54,7 +54,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java new file mode 100644 index 00000000000..0107e8b6170 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.transform; + +import org.apache.seatunnel.e2e.common.TestResource; +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.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; + +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK not support adapt") +public class TestEmbeddingIT 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("/mock-embedding.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 + "/mock-embedding.json") + .withEnv( + "MOCKSERVER_INITIALIZATION_JSON_PATH", + TMP_DIR + "/mock-embedding.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 testEmbedding(TestContainer container) throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/embedding_transform.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + @TestTemplate + public void testEmbeddingWithCustomModel(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/embedding_transform_custom.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } +} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java index 6f17c5a94f7..5a4fa640e96 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java @@ -87,4 +87,11 @@ public void testLLMWithOpenAI(TestContainer container) Container.ExecResult execResult = container.executeJob("/llm_openai_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } + + @TestTemplate + public void testLLMWithCustomModel(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/llm_transform_custom.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf new file mode 100644 index 00000000000..aa0288a705b --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf @@ -0,0 +1,295 @@ +# +# 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 { + book_id = "int" + book_name = "string" + book_intro = "string" + author_biography = "string" + } + } + rows = [ + {fields = [1, "To Kill a Mockingbird", + "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", + "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." + ], kind = INSERT} + {fields = [2, "1984", + "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", + "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." + ], kind = INSERT} + {fields = [3, "Pride and Prejudice", + "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", + "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." + ], kind = INSERT} + {fields = [4, "The Great GatsbyThe Great Gatsby", + "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", + "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." + ], kind = INSERT} + {fields = [5, "Moby-Dick", + "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", + "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." + ], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + Embedding { + source_table_name = "fake" + model_provider = QIANFAN + model = bge_large_en + api_key = xxxxxxxx + secret_key = xxxxxxxx + api_path = "http://mockserver:1080/v1/qianfan/embedding" + oauth_path = "http://mockserver:1080/v1/qianfan/token" + single_vectorized_input_number = 2 + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + result_table_name = "embedding_output_1" + } + + Embedding { + source_table_name = "fake" + model_provider = DOUBAO + model = ep-20240830113341-wwwqd + api_key = xxxxxxxx + api_path = "http://mockserver:1080/v1/doubao/embedding" + single_vectorized_input_number = 2 + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + result_table_name = "embedding_output_2" + } + + + Embedding { + source_table_name = "fake" + model_provider = OPENAI + model = text-embedding-3-small + api_key = xxxxxxxx + api_path = "http://mockserver:1080/v1/openai/embedding" + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + result_table_name = "embedding_output_3" + } +} + +sink { + Assert { + source_table_name = "embedding_output_1" + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } + Assert { + source_table_name = "embedding_output_2" + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } + Assert { + source_table_name = "embedding_output_3" + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + 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/embedding_transform_custom.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform_custom.conf new file mode 100644 index 00000000000..09611d44fc8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform_custom.conf @@ -0,0 +1,152 @@ +# +# 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 { + book_id = "int" + book_name = "string" + book_intro = "string" + author_biography = "string" + } + } + rows = [ + {fields = [1, "To Kill a Mockingbird", + "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", + "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." + ], kind = INSERT} + {fields = [2, "1984", + "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", + "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." + ], kind = INSERT} + {fields = [3, "Pride and Prejudice", + "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", + "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." + ], kind = INSERT} + {fields = [4, "The Great GatsbyThe Great Gatsby", + "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", + "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." + ], kind = INSERT} + {fields = [5, "Moby-Dick", + "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", + "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." + ], kind = INSERT} + ] + result_table_name = "fake" + } +} + +transform { + Embedding { + source_table_name = "fake" + model_provider = CUSTOM + model = text-embedding-3-small + api_key = xxxxxxxx + api_path = "http://mockserver:1080/v1/custom/embedding" + single_vectorized_input_number = 2 + vectorization_fields { + book_intro_vector = book_intro + author_biography_vector = author_biography + } + custom_config={ + custom_response_parse = "$.data[*].embedding" + custom_request_headers = { + # refer to mockserver config + Authorization = "Bearer xxxxxxxx" + } + custom_request_body ={ + modelx = "${model}" + inputx = ["${input}"] + } + } + result_table_name = "embedding_output_1" + } +} + +sink { + Assert { + source_table_name = "embedding_output_1" + rules = + { + field_rules = [ + { + field_name = book_id + field_type = int + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = book_intro_vector + field_type = float_vector + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = author_biography_vector + field_type = float_vector + 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/llm_transform_custom.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_transform_custom.conf new file mode 100644 index 00000000000..8f23fa9c1b1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_transform_custom.conf @@ -0,0 +1,94 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### 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 = CUSTOM + 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" + custom_config={ + custom_response_parse = "$.choices[*].message.content" + custom_request_headers = { + Content-Type = "application/json" + Authorization = "Bearer b2e66711-10ed-495c-9f27-f233a8db09c2" + } + custom_request_body ={ + model = "${model}" + messages = [ + { + role = "system" + content = "${prompt}" + }, + { + role = "user" + content = "${input}" + }] + } + } + 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/mock-embedding.json b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mock-embedding.json new file mode 100644 index 00000000000..1688d3b2e27 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mock-embedding.json @@ -0,0 +1,75 @@ +// https://www.mock-server.com/mock_server/getting_started.html#request_matchers + +[ + { + "httpRequest": { + "method": "POST", + "path": "/v1/qianfan/embedding/.*", + "queryStringParameters": { + "access_token": ["^.*$"] + } + }, + "httpResponse": { + "body":{"id":"as-7bd1tetvey","object":"embedding_list","created":1724948271,"data":[{"object":"embedding","embedding":[-0.00021059511345811188,-0.027767932042479515,-0.01854688860476017,0.032603006809949875,-0.00508118188008666,-0.0278035756200552,-0.009797265753149986,0.0006566192023456097,0.002196578774601221,0.03303470090031624,-0.0032474317122250795,0.0023599115666002035,0.045467741787433624,-0.009151561185717583,-0.016853950917720795,0.006868097465485334,0.007051482331007719,-0.03534781560301781,-0.039046160876750946,-0.009137873537838459,0.0004795161366928369,0.04441152513027191,0.018654968589544296,-0.0032995922956615686,-0.021618107333779335,-0.0176478773355484,0.05218972638249397,0.02979690209031105,0.04109659045934677,-0.024078253656625748,-0.027630683034658432,0.03802218288183212,0.018793299794197083,0.0413341224193573,0.00588208669796586,-0.020419621840119362,-0.00002904470056819264,0.01946319453418255,-0.008217964321374893,0.002777740126475692,-0.007581755518913269,-0.011274664662778378,-0.007225516252219677,-0.02712559886276722,-0.04745253548026085,-0.033444084227085114,-0.03344576060771942,-0.02444254420697689,-0.002873974619433284,-0.004047377035021782,0.003027654252946377,-0.030405033379793167,-0.034476716071367264,0.011209231801331043,-0.033511288464069366,-0.007611188106238842,0.0029583752620965242,0.02062322199344635,-0.02413185127079487,-0.03221965208649635,-0.006494476459920406,0.18443112075328827,-0.023148996755480766,0.009271507151424885,-0.020391836762428284,0.014609504491090775,-0.006966262124478817,-0.0026493698824197054,-0.05946456268429756,-0.006900311913341284,-0.00634230999276042,0.040444329380989075,0.014873902313411236,-0.011657400988042355,-0.0280571561306715,-0.019210409373044968,-0.0011373738525435328,0.004437817260622978,0.02734195999801159,-0.01763341575860977,0.0029308348894119263,0.020157339051365852,0.024618560448288918,0.005358884576708078,0.014323997311294079,0.0029505817219614983,-0.005110694095492363,0.6496252417564392,-0.053306080400943756,-0.0058554974384605885,-0.01981886848807335,-0.008071924559772015,-0.006869315169751644,0.011915079317986965,0.01261670608073473,-0.019067028537392616,-0.004722272977232933,-0.05077458918094635,-0.046526990830898285,0.020257316529750824,0.013736839406192303,0.002608766546472907,-0.0034607655834406614,-0.018692856654524803,-0.014540831558406353,0.044487159699201584,-0.056308597326278687,-0.03237638995051384,0.004933377727866173,0.018158087506890297,0.021088099107146263,-0.04402942582964897,-0.029466865584254265,0.0105899628251791,0.05357479676604271,-0.05160846561193466,-0.016380205750465393,-0.010790468193590641,0.024171648547053337,0.013854079879820347,0.03144305944442749,0.02881663851439953,0.031199457123875618,-0.028395550325512886,0.01117578987032175,0.019122391939163208,-0.009040268138051033,0.022094037383794785,-0.012739963829517365,0.00017399451462551951,-0.028970466926693916,0.01716982014477253,-0.004101032391190529,-0.03627597913146019,-0.01965869776904583,0.02041907235980034,-0.015489906072616577,-0.012333795428276062,0.019006699323654175,0.005117892753332853,0.016836678609251976,-0.02589852176606655,0.01113149244338274,-0.02863095887005329,0.004759848117828369,-0.05533807352185249,-0.013764551840722561,-0.029555510729551315,0.043936941772699356,-0.026340026408433914,-0.008651661686599255,-0.019796498119831085,-0.02661082334816456,-0.0677989050745964,-0.00811520405113697,0.04509684816002846,-0.0015024503227323294,0.015820348635315895,0.023989910259842873,-0.030045341700315475,0.02508663572371006,0.009965837001800537,0.003464705077931285,-0.011840038001537323,0.0198811162263155,-0.03810553252696991,0.01861695945262909,-0.032312821596860886,0.019130485132336617,0.04861173406243324,-0.033806804567575455,0.013468412682414055,-0.00406223488971591,0.033424291759729385,0.04291091859340668,-0.01149976346641779,0.016552245244383812,0.0069789523258805275,0.025855500251054764,-0.009393014945089817,-0.007661312352865934,0.05473153665661812,0.025690428912639618,-0.036358196288347244,0.010988211259245872,0.010048598982393742,-0.0058916471898555756,-0.005367740988731384,0.048110589385032654,-0.0007429873221553862,0.023407895117998123,-0.020141126587986946,-0.014089311473071575,-0.014083907008171082,-0.015959562733769417,-0.02766922116279602,-0.015350828878581524,0.0020960834808647633,-0.0032107033766806126,0.0037848034407943487,-0.011974534951150417,0.035355035215616226,0.037306368350982666,-0.0077398004941642284,-0.005957275163382292,0.000970572407823056,0.016372399404644966,0.013819240964949131,0.010282308794558048,0.01615927368402481,-0.008819358423352242,-0.026662444695830345,-0.017539484426379204,0.012908521108329296,0.004405930172652006,0.008539012633264065,-0.000672093010507524,-0.028431832790374756,0.006341465283185244,-0.01010969653725624,0.01827351562678814,-0.02010517567396164,-0.0162906926125288,-0.014747546054422855,-0.00887925736606121,0.04651108384132385,-0.003932764753699303,0.009381836280226707,-0.024245109409093857,-0.017034199088811874,-0.03038940392434597,-0.0013200266985222697,-0.004807321820408106,-0.020380636677145958,-0.018323810771107674,-0.008138819597661495,-0.015519209206104279,-0.024748897179961205,0.007331625558435917,-0.007141429465264082,-0.013838447630405426,-0.019334133714437485,-0.005408428609371185,0.03597620502114296,-0.010720993392169476,-0.0034316780511289835,-0.02497507818043232,-0.001224928768351674,0.02395603433251381,-0.03572462499141693,-0.03453918173909187,-0.033510107547044754,-0.006008330732584,0.006822410970926285,0.03696754574775696,0.015473157167434692,0.01460866630077362,-0.0007411232218146324,0.07114441692829132,-0.0010755411349236965,-0.015285762026906013,0.01705975830554962,0.004958299919962883,-0.04508613049983978,0.011662159115076065,0.0025561910588294268,-0.005702427122741938,0.013844580389559269,-0.01892123930156231,-0.011532346718013287,-0.008036554791033268,-0.0011347347171977162,-0.01549521740525961,-0.03018491342663765,0.024034442380070686,0.024171337485313416,-0.0015363524435088038,-0.004312901291996241,-0.041096772998571396,-0.004297324921935797,0.02075061947107315,0.028171395882964134,-0.030296403914690018,0.02048708125948906,0.012012072838842869,0.03347966820001602,-0.020060598850250244,-0.014894988387823105,0.02002022974193096,-0.02158072404563427,-0.033770836889743805,0.021192878484725952,-0.0018208179390057921,0.0009991482365876436,-0.044341955333948135,-0.011937808245420456,0.006897172890603542,-0.04503057897090912,-0.03995048627257347,-0.004204429220408201,0.021579941734671593,-0.028874646872282028,0.03665200620889664,0.03300997614860535,-0.006059388630092144,-0.0018951641395688057,-0.010933570563793182,0.010337389074265957,-0.03216280788183212,0.019717233255505562,0.017239345237612724,-0.025499241426587105,0.030989864841103554,-0.008808952756226063,0.015054614283144474,-0.008688508532941341,0.018388424068689346,-0.021100683137774467,0.01686527207493782,-0.011277221143245697,0.012690097093582153,-0.005680461414158344,0.01244433131068945,-0.02932148240506649,0.013947572559118271,0.008459887467324734,-0.0000801066926214844,0.002725276630371809,-0.00300898146815598,0.038368478417396545,-0.02121691033244133,0.008453425951302052,-0.0574941411614418,-0.0030004887375980616,-0.017873194068670273,-0.011487414129078388,0.021161673590540886,0.002342820866033435,-0.005630783271044493,0.017525827512145042,0.012779763899743557,-0.008308114483952522,0.007321945857256651,-0.02929837629199028,0.007987956516444683,-0.003551504109054804,0.04633054509758949,0.011856377124786377,-0.008557242341339588,-0.01718260534107685,0.019947312772274017,0.023221932351589203,0.013549587689340115,0.01817863993346691,0.04516642913222313,0.0033165987115353346,-0.0007992511964403093,-0.025754224509000778,0.02426086738705635,0.007627000566571951,0.018178611993789673,0.012401783838868141,-0.004206185694783926,0.008747300133109093,0.004693929571658373,-0.021600430831313133,-0.02287721261382103,0.01350963395088911,0.0053276680409908295,0.004475089721381664,0.02381257340312004,-0.019157743081450462,-0.005368090234696865,-0.03884730115532875,0.0007509654387831688,-0.003929459489881992,0.0033258332405239344,-0.01829037442803383,-0.00557937566190958,-0.03142822906374931,-0.027745844796299934,-0.028555698692798615,0.027369597926735878,-0.015915514901280403,0.02619199827313423,-0.006784170866012573,0.037110622972249985,0.010802475735545158,0.038187120109796524,-0.009386979043483734,-0.005884665530174971,-0.04339227080345154,0.02942570112645626,-0.015048210509121418,-0.0017648611683398485,-0.02051001787185669,0.0004942170926369727,0.01565675437450409,-0.043645285069942474,0.03105166181921959,0.01577756553888321,-0.03641166910529137,0.0015384277794510126,-0.18682853877544403,0.03271225094795227,0.006441325880587101,-0.008044271729886532,0.005846887361258268,-0.014891608618199825,0.017346085980534554,-0.01583673618733883,-0.028799831867218018,-0.011949403211474419,0.005177765619009733,0.009200219996273518,0.024670500308275223,0.004799164365977049,-0.010118584148585796,-0.007180654443800449,0.009807412512600422,-0.001643523690290749,0.025886712595820427,-0.02271057665348053,0.0013511634897440672,0.017349032685160637,-0.012280421331524849,0.01876072958111763,-0.004262855276465416,-0.010785785503685474,0.012419067323207855,0.015993289649486542,0.006141404155641794,-0.0486750565469265,0.01631086878478527,0.009667158126831055,0.007043206598609686,-0.030202830210328102,-0.05439183861017227,-0.007263322826474905,-0.011104041710495949,0.04070024937391281,0.009039278142154217,-0.005261984653770924,-0.0017805563984438777,-0.003699537366628647,-0.024146024137735367,-0.012978706508874893,-0.016917143017053604,0.010574422776699066,-0.0032952926121652126,-0.030544349923729897,0.003406661795452237,0.006680595222860575,-0.015289555303752422,0.003602110082283616,-0.029202651232481003,0.00042448146268725395,-0.004458544310182333,-0.006447230000048876,0.004317210055887699,0.005357617978006601,0.0019395265262573957,0.0026867224369198084,-0.0009431689977645874,-0.006399401463568211,0.010073689743876457,0.030944395810365677,0.002550299745053053,0.025951610878109932,0.003800789127126336,-0.04651292786002159,-0.048458099365234375,-0.014975139871239662,-0.002749948063865304,0.004808521363884211,0.03070972114801407,0.006722541525959969,-0.007859279401600361,0.013682885095477104,0.00957096740603447,-0.013668203726410866,-0.00016980688087642193,-0.024517782032489777,0.020281005650758743,0.03860524669289589,-0.01655668392777443,-0.05017245560884476,-0.005908954422920942,0.01387692242860794,-0.01276292186230421,-0.03422152251005173,-0.0023568233009427786,0.016376817598938942,-0.041153550148010254,-0.014757751487195492,0.03825400024652481,-0.035122163593769073,0.03596165031194687,0.038283392786979675,-0.01915409415960312,-0.014336580410599709,-0.015385901555418968,-0.029441365972161293,0.03101111762225628,0.005073732230812311,-0.006437203381210566,-0.01908678002655506,-0.002977523487061262,-0.002094009891152382,0.04122431203722954,0.004754438530653715,0.02652120590209961,0.010309034027159214,0.012997240759432316,-0.019439255818724632,0.003306824015453458,0.006698875222355127,0.005795760545879602,-0.010780135169625282,-0.023200908675789833,0.01796633005142212,0.009287238121032715,-0.003895723959431052,0.013337905518710613,0.025569044053554535,0.022346174344420433,-0.011740101501345634,0.03706115111708641,0.01642177440226078,-0.005734262056648731,-0.002154782647266984,-0.025524543598294258,-0.041741956025362015,0.005545818246901035,0.013133159838616848,-0.014538787305355072,-0.01684839278459549,0.05526084452867508,-0.010182134807109833,0.029960233718156815,0.004521056544035673,0.000838306441437453,0.022687237709760666,-0.029925795271992683,-0.009856436401605606,-0.02253301441669464,-0.009113472886383533,-0.00920250453054905,0.014004389755427837,0.029789825901389122,-0.007926435209810734,-0.021544726565480232,-0.0133292768150568,-0.008220207877457142,-0.02275729365646839,-0.052783627063035965,-0.03071376122534275,-0.0022968738339841366,0.017426472157239914,0.0038121037650853395,-0.0407782718539238,-0.00009582042548572645,0.009582215920090675,-0.030289215967059135,0.0013146387645974755,-0.035006701946258545,0.03376166895031929,-0.008873547427356243,-0.005690729711204767,-0.02085866592824459,0.023660162463784218,0.02535366639494896,-0.006353122182190418,0.0007649947656318545,-0.006274092476814985,-0.04827839881181717,-0.01785298064351082,0.010698039084672928,0.0014962840359658003,0.03554944321513176,0.012286324985325336,-0.039143819361925125,0.004069188609719276,0.018372230231761932,0.008820582181215286,-0.009328721091151237,-0.004874794743955135,-0.014218525029718876,0.019161565229296684,0.01462504081428051,0.019836289808154106,0.025463195517659187,0.009707238525152206,-0.009576020762324333,-0.0055716028437018394,-0.011893569491803646,-0.008480378426611423,0.004252410028129816,0.013349207118153572,-0.01655896194279194,-0.039842694997787476,-0.016235416755080223,-0.003012096043676138,-0.0040341513231396675,0.01641716994345188,-0.0019341664155945182,0.02344946376979351,0.015591164119541645,-0.0017627474153414369,0.01982186734676361,0.014068963937461376,-0.016694217920303345,0.021293651312589645,0.004868016578257084,-0.021520724520087242,-0.018620682880282402,-0.01744752563536167,0.01813305914402008,-0.008659204468131065,-0.009580496698617935,0.005942412186414003,-0.00136253098025918,-0.01846194826066494,0.0020596617832779884,-0.039835125207901,-0.002534691244363785,-0.032292310148477554,0.013045907020568848,0.024678610265254974,-0.023161306977272034,0.04890305921435356,-0.004793909378349781,0.0038888126146048307,-0.02832169272005558,-0.02261134423315525,-0.06501864641904831,-0.004058612510561943,0.03268272429704666,0.010768147185444832,-0.008082466199994087,-0.0017423891695216298,-0.041671812534332275,0.009975595399737358,-0.00382527569308877,-0.025365419685840607,0.009978887625038624,-0.04618224874138832,-0.009343093261122704,-0.03074515052139759,-0.021165281534194946,-0.001912703737616539,-0.010987833142280579,0.015884140506386757,0.036444295197725296,-0.015434290282428265,0.014077482745051384,-0.011205187067389488,0.030951518565416336,0.03389952704310417,-0.029726112261414528,-0.0016963136149570346,-0.02061229571700096,-0.0139729343354702,0.04218011349439621,-0.011218545027077198,-0.004003942012786865,0.01652691140770912,-0.007812418509274721,0.03990553691983223,-0.03662434220314026,-0.010878518223762512,-0.023101497441530228,0.024742165580391884,0.013572390191257,0.01166819129139185,-0.006664498709142208,0.04057473689317703,0.02378370426595211,0.009193843230605125,0.009010028094053268,-0.0010899485787376761,-0.029528630897402763,0.05595232546329498,-0.0066804904490709305,-0.0021086069755256176,-0.03645811975002289,-0.02671816200017929,0.0075734639540314674,0.022725099697709084,0.014151963405311108,-0.006069105118513107,-0.027776548638939857,-0.0017719474853947759,0.022220611572265625,0.01029113307595253,0.03194112330675125,0.02617032639682293,-0.013851424679160118,-0.0008156702970154583,0.038775257766246796,-0.013746884651482105,-0.007707295008003712,-0.03796851634979248,-0.004138441290706396,0.028397297486662865,-0.0010368649382144213,-0.008697853423655033,-0.00862084235996008,0.012430219911038876,0.00206479849293828,-0.006794648244976997,-0.009484518319368362,-0.014684796333312988,-0.025407856330275536,0.021318815648555756,0.018594937399029732,-0.015070969238877296,-0.021831439808011055,-0.019772548228502274,0.031087186187505722,-0.025306066498160362,-0.02949276752769947,-0.0328788198530674,-0.0320521742105484,-0.011163540184497833,0.02418673224747181,-0.0032977017108350992,-0.014361183159053326,-0.026074668392539024,-0.007526756729930639,0.014066735282540321,0.005600559059530497,0.029861778020858765,-0.013118032366037369,-0.01834111101925373,0.002682792954146862,-0.0009973339037969708,-0.030452396720647812,-0.003517824225127697,-0.01721714250743389,0.012537650763988495,0.03137042373418808,-0.02563992515206337,-0.014541576616466045,-0.029647422954440117,-0.03872761130332947,0.0124127846211195,0.017298279330134392,-0.01542678289115429,0.023507999256253242,0.009598481468856335,0.01405521109700203,-0.021878115832805634,-0.014565207064151764,0.009759897366166115,-0.00893075205385685,0.009577925316989422,0.020548827946186066,0.0009061423479579389,0.0004889803822152317,0.020842552185058594,0.0028179590590298176,-0.028958793729543686,-0.03639044985175133,-0.01715780794620514,0.0032596688251942396,-0.02279387041926384,0.013053672388195992,0.026006784290075302,-0.007702230010181665,0.032267000526189804,-0.0017689288360998034,-0.04426606371998787,0.027218639850616455,-0.035268381237983704,-0.03934603929519653,0.01837988942861557,-0.03718952834606171,0.029364554211497307,0.00939704105257988,-0.01175576914101839,-0.011473660357296467,-0.028851402923464775,-0.023649366572499275,-0.032468169927597046,0.01531847845762968,-0.012807006016373634,0.007838662713766098,-0.015825774520635605,0.004559505730867386,-0.01605060324072838,-0.006479652598500252,0.0012571568368002772,0.0008885100251063704,-0.01844359003007412,0.012507060542702675,0.014665930531919003,0.013433980755507946,-0.02287564054131508,-0.05416969209909439,-0.03761182725429535,0.022247064858675003,0.00992788840085268,-0.025322325527668,0.00988700706511736,-0.005227380432188511,-0.026791758835315704,0.023695098236203194,-0.000014328586075862404,-0.013037673197686672,-0.01907210424542427,0.016273565590381622,-0.021701635792851448,-0.0021066220942884684,0.03337745741009712,-0.013645888306200504,0.0018892678199335933,-0.005045998375862837,0.042833685874938965,-0.01615271344780922,0.04580359905958176,-0.0223141610622406,0.014279637485742569,0.02616218850016594,-0.015080823563039303,0.01668858341872692,-0.013901581056416035,0.009553187526762486,0.025403399020433426,0.0052113644778728485,-0.014745769090950489,-0.0009014360257424414,0.004632167983800173,0.024682113900780678,0.03319165110588074,-0.033311035484075546,-0.006520306225866079,0.01641225256025791,0.05087489262223244,0.011769518256187439,-0.0068300398997962475,0.0040258122608065605,-0.012988467700779438,0.034983839839696884,-0.017945125699043274,-0.013408321887254715,-0.02442520298063755,0.04357581213116646,-0.056211747229099274,0.009304540231823921,-0.005600585136562586,-0.03652577102184296,0.02592485398054123,-0.008577843196690083,-0.014812407083809376,0.0018684475217014551,0.005596611183136702,-0.02698114700615406,0.008817252703011036,-0.009420663118362427,-0.016018936410546303,-0.007171689998358488,-0.023529503494501114,0.04592137411236763,-0.025283947587013245,0.018765371292829514,-0.03229653462767601,0.009904840029776096,0.017309457063674927,0.0005982531001791358,-0.017608163878321648,0.010085015557706356,0.009459641762077808,-0.014238577336072922,0.015561285428702831,0.01512686163187027,-0.016820671036839485,0.009932256303727627,0.006123207975178957,-0.019477643072605133,-0.014218680560588837,0.0034777051769196987,0.02694743499159813,0.014948295429348946,0.007310180924832821,-0.01140064187347889,0.02155054546892643,-0.0031535422895103693,0.02321101352572441,0.03501196205615997,0.016104355454444885,-0.011443550698459148,0.020494714379310608,0.000037473870179383084,0.030133651569485664,0.013036134652793407,-0.010131465271115303,-0.03145812451839447,0.006029221694916487,-0.018738742917776108,-0.0026733726263046265,-0.0069665201008319855,0.019513335078954697,0.006418284960091114,0.011988451704382896,-0.020809510722756386,0.004837896674871445,0.006817515939474106,-0.007561174221336842,-0.01496717520058155,-0.005628278013318777,-0.0151112275198102,-0.02160394750535488,0.021545330062508583,-0.011424845084547997,0.04252980276942253,-0.028489435091614723,0.021204529330134392,-0.061839692294597626,0.014390102587640285,-0.008755280636250973,0.026408012956380844,-0.006401140242815018,0.017197787761688232,-0.0370585173368454,0.034825924783945084,-0.006772985681891441,-0.02114637941122055,-0.02247908152639866,0.007700352463871241,-0.03821778669953346,0.00017269796808250248,-0.03410832956433296,0.019855795428156853,0.0009135074215009809,0.035855021327733994,-0.016612650826573372,-0.040418609976768494,0.0028196251951158047,0.0040102992206811905,-0.04908296465873718,0.012688972987234592,-0.003799975384026766,0.006731805857270956,-0.03011994995176792,-0.03930181637406349,0.006181399803608656,-0.007794621866196394,-0.017922749742865562,-0.0012883433373644948,-0.04030757397413254,-0.01805681362748146,0.005565496627241373,-0.0047272671945393085,-0.014569416642189026,-0.008235592395067215,-0.003890547202900052,-0.020433317869901657,0.02066405490040779,-0.005242255982011557,-0.00019520313071552664,-0.03202907368540764,-0.029797418043017387,-0.00018843963334802538,0.016573864966630936,-0.04733728989958763,-0.0244253259152174,-0.0031971873249858618,0.014172783121466637,-0.002092082519084215,-0.005944475065916777,0.014103066176176071,0.027422163635492325,0.009740452282130718,-0.055597104132175446,-0.024621648713946342,-0.023668251931667328,-0.0015227218391373754,0.00628467695787549,0.02427695132791996,-0.029232285916805267,-0.005822507664561272,0.005340541712939739,0.017792735248804092,-0.0404917411506176,0.01850849948823452,-0.02137850783765316,0.032680269330739975,-0.04013790190219879,-0.009750901721417904,0.011691272258758545,-0.10531952232122421,0.008833343163132668,0.02102864719927311,0.01059207133948803,-0.005922437179833651,-0.000013196819054428488,-0.022282278165221214,-0.00016967281408142298,-0.026948388665914536,-0.027980580925941467,0.021381018683314323,-0.02278803288936615,-0.011419698596000671,-0.006655302830040455,-0.01766994222998619,-0.016247810795903206,0.0005969391786493361,0.030986133962869644,0.026341672986745834,-0.008760204538702965,0.0017293060664087534,-0.003207669360563159,-0.035950031131505966,0.01429867185652256,0.005979999899864197,-0.012309896759688854,-0.01963735744357109,-0.008175088092684746,0.006148855201900005,0.01289496198296547,0.04852227866649628,-0.012619220651686192,-0.0026282796170562506],"index":0},{"object":"embedding","embedding":[-0.004285297356545925,-0.014564486220479012,-0.016580479219555855,0.016443807631731033,0.004217916633933783,-0.015437845140695572,-0.006158899050205946,-0.019124578684568405,-0.006950558628886938,0.012497621588408947,0.0351058691740036,0.008919301442801952,0.035629112273454666,-0.0023913895711302757,0.007267395965754986,0.011561121791601181,-0.01238621398806572,-0.011214325204491615,-0.015068083070218563,-0.022773437201976776,0.010673204436898232,0.044169772416353226,-0.0039693983271718025,-0.015642477199435234,-0.021421419456601143,0.012200327590107918,0.008789672516286373,0.019884992390871048,0.03242874518036842,-0.012119706720113754,-0.009400391951203346,0.03798016160726547,0.015466639772057533,0.006101908162236214,0.014775496907532215,0.02007640339434147,-0.016790490597486496,0.004852014128118753,0.013239431194961071,0.011806187219917774,-0.017249641939997673,0.0033052500803023577,0.03879188746213913,-0.02303340472280979,-0.0277368426322937,-0.03020656481385231,-0.02082614041864872,-0.03701325133442879,0.004458844196051359,0.021700605750083923,-0.003344543045386672,-0.0331253781914711,-0.020582376047968864,0.05624222010374069,-0.035694669932127,0.013974400237202644,0.003263092366978526,-0.005511644762009382,-0.014846364967525005,-0.02936599962413311,-0.009299523197114468,0.19663512706756592,-0.019456950947642326,0.011017006821930408,-0.012154284864664078,0.031545598059892654,-0.006233204156160355,-0.020257527008652687,-0.02846020832657814,-0.013629027642309666,-0.011972113512456417,0.005676695145666599,0.008949648588895798,-0.01061919890344143,0.0032657296396791935,-0.020064225420355797,-0.02201222814619541,-0.02192266285419464,0.050561849027872086,0.0028951717540621758,-0.023434070870280266,0.02147388458251953,0.014437002129852772,-0.02084781974554062,-0.01056731678545475,-0.0031143685337156057,0.01706208847463131,0.6640109419822693,-0.06142793968319893,-0.009754650294780731,-0.013139267452061176,-0.028979729861021042,-0.011413425207138062,0.0024580468889325857,-0.0210629403591156,-0.009800789877772331,-0.010361839085817337,-0.039458002895116806,-0.04414813965559006,-0.009862912818789482,0.039889171719551086,-0.00729469396173954,-0.03144644573330879,-0.024813562631607056,-0.004071842413395643,0.02081509307026863,-0.03137628734111786,-0.014005686156451702,-0.007677961140871048,0.018851477652788162,-0.006114735268056393,-0.056475453078746796,0.011689091101288795,-0.011373203247785568,0.007448793854564428,-0.039242036640644073,0.007213531527668238,-0.014041850343346596,0.024084540084004402,-0.008465197868645191,0.023254621773958206,0.008521183393895626,0.00899258442223072,-0.03143690526485443,0.013184809125959873,0.025586312636733055,-0.022307634353637695,0.050704192370176315,-0.0009345149737782776,-0.0012701196828857064,-0.002420233329758048,0.017340997233986855,-0.007981940172612667,-0.01048630103468895,-0.014014068059623241,-0.013617749325931072,0.028746429830789566,0.00424461392685771,0.0015173522988334298,-0.0012624065857380629,0.0006192005821503699,-0.006423024460673332,0.013799447566270828,-0.025131532922387123,0.01045960746705532,-0.012492459267377853,-0.02260611020028591,0.021308323368430138,0.0036427201703190804,-0.03241097927093506,0.004548081196844578,0.0031649810262024403,0.0028049550019204617,-0.0182278361171484,0.007720143999904394,0.054627809673547745,0.0007530491566285491,0.01669897697865963,0.005791725590825081,-0.021497447043657303,-0.010770023800432682,-0.022960832342505455,-0.03926060348749161,-0.0028847239445894957,0.0018401116831228137,-0.003583054058253765,0.013903351500630379,-0.04713357985019684,0.0013615776551887393,0.002380107529461384,0.009916814044117928,0.01951373554766178,-0.013420704752206802,0.015164395794272423,0.02784680761396885,-0.005289891269057989,0.002555672312155366,0.009907773695886135,0.02710774540901184,0.01775195077061653,0.007064831908792257,0.048922691494226456,0.0004531689337454736,-0.02901897020637989,0.022401420399546623,-0.022551216185092926,-0.000843644724227488,-0.004428130574524403,0.03575357049703598,0.0009244136745110154,0.042618829756975174,-0.009692701511085033,0.01043030433356762,0.0038911611773073673,-0.018474670127034187,-0.02080972120165825,-0.019145945087075233,0.025530647486448288,-0.004237101413309574,0.018756091594696045,-0.01180137600749731,0.02230081334710121,0.023273218423128128,-0.0391106940805912,-0.017901204526424408,0.020478615537285805,0.008464116603136063,0.009504350833594799,-0.012657145038247108,0.02349432185292244,-0.021555786952376366,-0.0019459519535303116,-0.030501462519168854,-0.0017687676008790731,0.015922917053103447,0.04266855865716934,0.010693078860640526,-0.017475955188274384,-0.021369535475969315,0.009963973425328732,0.04677752032876015,0.0024647170212119818,-0.014198452234268188,-0.00824415497481823,0.002477082656696439,0.024890942499041557,0.006369042210280895,0.00020600203424692154,-0.010448234155774117,-0.0023461419623345137,0.007621110882610083,0.03437066450715065,-0.03733120858669281,-0.03150142729282379,0.002128505613654852,0.004461865406483412,0.021113652735948563,-0.009577988646924496,-0.034292206168174744,-0.003404168179258704,0.00008232207619585097,-0.018383057788014412,-0.012374766170978546,-0.010677404701709747,-0.028470903635025024,-0.021290645003318787,-0.0031482786871492863,-0.006115273106843233,0.006543636322021484,-0.0007899189949966967,-0.00857260636985302,-0.03981125354766846,-0.0280560664832592,0.003977597691118717,0.0579090379178524,-0.015463583171367645,0.02602853812277317,-0.018383540213108063,0.014274277724325657,0.02344614639878273,-0.006901269778609276,0.0043588485568761826,0.025912975892424583,-0.03577928990125656,0.047188080847263336,-0.002287505427375436,-0.013656373135745525,-0.0022315282840281725,-0.037634190171957016,-0.012077958323061466,0.03787172958254814,-0.000589891045819968,0.006841442547738552,-0.05457588657736778,-0.013828745111823082,0.03929060697555542,-0.010475962422788143,-0.003952181898057461,-0.012839345261454582,-0.013511652126908302,-0.014495810493826866,0.009007317945361137,-0.04348411038517952,0.009792719967663288,-0.010241426527500153,0.009871255606412888,0.007920924574136734,-0.030454762279987335,0.002537030028179288,-0.009922289289534092,-0.03304130211472511,0.010475628077983856,-0.008054936304688454,0.037432774901390076,-0.02610715478658676,-0.012445738539099693,0.02444445714354515,-0.03828082233667374,-0.03207232803106308,-0.012661836110055447,0.0030418329406529665,-0.018977487459778786,-0.021577944979071617,0.024432986974716187,-0.007854047231376171,0.005464739631861448,-0.016096334904432297,0.00376148265786469,-0.01685277186334133,0.006472278852015734,-0.013275789096951485,-0.02851736731827259,0.012032454833388329,-0.0019194848136976361,0.045898307114839554,-0.028387319296598434,0.016325538977980614,-0.005413290578871965,-0.005501685664057732,-0.003401385620236397,-0.009151612408459187,0.005875143222510815,0.02995760180056095,-0.009996180422604084,0.004154312424361706,0.012755542062222958,0.0041709179058671,0.0008314028382301331,-0.02778591401875019,-0.010996638797223568,-0.011812896467745304,0.016545087099075317,-0.033632829785346985,-0.017562076449394226,-0.028860215097665787,0.011606150306761265,-0.0010538806673139334,-0.02621433138847351,-0.018384141847491264,0.050438541918992996,-0.005153534468263388,-0.015172770246863365,-0.03160208463668823,0.004491359461098909,0.010390433482825756,-0.024569563567638397,0.018246639519929886,0.03956255316734314,-0.01111405435949564,-0.0005722676869481802,0.04335317015647888,0.015209430828690529,0.028309915214776993,0.006345074158161879,0.00763111375272274,0.004354933276772499,-0.007664976641535759,-0.03524396941065788,0.006746530067175627,-0.030275514349341393,0.03031207248568535,0.027718152850866318,0.04550035297870636,-0.009212451986968517,-0.007164254318922758,-0.009993841871619225,-0.02746083214879036,0.019127987325191498,-0.012546233832836151,-0.023727668449282646,0.006587502546608448,-0.024809980764985085,0.0141473188996315,-0.013625837862491608,-0.012030841782689095,0.001633543404750526,0.008806606754660606,-0.0029283168260008097,0.002919907448813319,-0.009159025736153126,-0.01866384781897068,-0.04996678978204727,0.020794417709112167,0.013871634379029274,0.013637245632708073,0.014808045700192451,0.011251740157604218,0.026380857452750206,0.03816097974777222,0.00008325099770445377,-0.00430482579395175,-0.03483356162905693,0.055521171540021896,-0.011507807299494743,-0.021945221349596977,-0.02403912879526615,-0.024212120100855827,0.008661230094730854,-0.01638766936957836,0.028347833082079887,-0.008635062724351883,-0.016870172694325447,-0.03763468191027641,-0.20208165049552917,0.027341028675436974,-0.002945993561297655,0.0035014082677662373,0.004758790601044893,-0.01142621785402298,0.035212136805057526,-0.003012634813785553,-0.02983098104596138,0.012239260599017143,-0.011561859399080276,-0.013696473091840744,0.006615175865590572,0.0007685653981752694,0.023931700736284256,-0.04886975511908531,0.015155188739299774,-0.039185937494039536,0.026939528062939644,-0.0030653858557343483,-0.008654006756842136,0.011049889959394932,0.0007772246026434004,0.017968233674764633,-0.002294074511155486,-0.03704323247075081,-0.009562411345541477,0.0013991565210744739,0.01862112060189247,-0.009166751056909561,0.028347564861178398,0.005998472683131695,0.007365141995251179,-0.012594997882843018,-0.05549926310777664,0.010497687384486198,-0.03980746865272522,0.01720789261162281,0.024997714906930923,-0.025926917791366577,0.004801096394658089,-0.033253345638513565,0.00344124436378479,-0.00523914210498333,-0.004632190335541964,-0.031936343759298325,-0.013963598757982254,-0.00831675436347723,0.008605812676250935,0.008931395597755909,-0.04545494541525841,0.00015152715786825866,-0.04069611802697182,-0.0008555970271117985,0.004400073550641537,0.012143315747380257,0.0070645990781486034,0.0047237626276910305,0.0017272194381803274,0.011750890873372555,0.006575545761734247,-0.018963251262903214,0.0007710521458648145,0.02249409630894661,0.008462783880531788,0.00477238604798913,0.0016029777470976114,-0.03513152524828911,-0.02871382236480713,0.0023913830518722534,-0.017433786764740944,0.00119790097232908,0.028445323929190636,-0.013209926895797253,0.012617296539247036,0.028333215042948723,0.01438191533088684,-0.013831940479576588,-0.017087753862142563,-0.013826519250869751,0.016261164098978043,0.015808813273906708,-0.0033666789531707764,-0.016928475350141525,-0.025441525503993034,0.017038721591234207,-0.014180978760123253,0.006150401197373867,-0.011126064695417881,0.0289481021463871,-0.02729945257306099,-0.009256374090909958,0.014649285934865475,0.010429946705698967,0.009939033538103104,0.054632559418678284,-0.023171652108430862,0.018928486853837967,-0.0033703099470585585,-0.005358698777854443,0.039696455001831055,0.019350869581103325,-0.019738400354981422,-0.040208470076322556,0.008610597811639309,0.005552174989134073,0.04277408495545387,0.03762585669755936,-0.004798519425094128,-0.005145879462361336,0.013238409534096718,-0.023252611979842186,-0.007956072688102722,-0.007360454648733139,-0.006025645416229963,0.0019996999762952328,-0.023733634501695633,0.03507727384567261,0.024928174912929535,-0.0094306580722332,0.03118046000599861,0.033407676964998245,-0.009490770287811756,-0.016119850799441338,0.015381553210318089,-0.008191749453544617,0.0008451060857623816,0.0026274705305695534,0.004227403085678816,-0.0269999448210001,-0.012398885563015938,0.007053732872009277,0.01273365318775177,0.025427738204598427,0.0011117614340037107,0.009466675110161304,0.00901126954704523,-0.01754252426326275,-0.00019764728494919837,0.024138011038303375,-0.032829221338033676,-0.03675679489970207,-0.012212643399834633,-0.007770919241011143,0.008708507753908634,0.00558779202401638,0.039335306733846664,0.00006601445784326643,0.01122430618852377,-0.02178182266652584,-0.015776459127664566,0.0054691568948328495,-0.01675081066787243,-0.03336552157998085,0.0166954156011343,0.0024392888881266117,0.003997897729277611,-0.002229025587439537,0.015642855316400528,0.001834203489124775,-0.033062949776649475,0.03803792595863342,-0.01253955066204071,-0.008650057017803192,-0.011906759813427925,-0.009489190764725208,0.014507987536489964,-0.029505277052521706,0.02621149830520153,-0.015256255865097046,0.007301995065063238,0.01432863064110279,-0.036672890186309814,-0.026238076388835907,0.030241230502724648,-0.015589285641908646,0.00874402653425932,0.023671485483646393,-0.03277534991502762,0.007715675979852676,0.02306244894862175,0.011130577884614468,0.03420516848564148,0.0025111325085163116,0.02822883613407612,0.02705836109817028,0.030841918662190437,0.024715416133403778,0.023459019139409065,0.026125172153115273,0.0022017727605998516,0.00509035587310791,-0.0043801055289804935,-0.019084477797150612,-0.03443866968154907,-0.012868187390267849,-0.005454624071717262,-0.013477494940161705,-0.01139871310442686,0.004025402013212442,-0.02896619401872158,0.013030430302023888,-0.0314362533390522,0.017526881769299507,0.00688193691894412,0.0015910121146589518,-0.004511791281402111,-0.0047766980715096,0.010657932609319687,0.025815216824412346,0.00406211894005537,-0.010275744833052158,0.014373905956745148,0.016398504376411438,0.0132300378754735,-0.003038227092474699,-0.020517529919743538,-0.0011860469821840525,-0.01791244186460972,-0.029571333900094032,-0.026921171694993973,-0.024518650025129318,-0.003963025286793709,-0.03323286399245262,0.029462462291121483,0.023246020078659058,-0.03247830644249916,0.032463159412145615,0.006981177255511284,-0.013332989998161793,-0.035614851862192154,-0.028454722836613655,-0.048840075731277466,-0.002592537784948945,0.03852313756942749,0.03448646515607834,-0.000010741460755525623,-0.010983542539179325,-0.02426745370030403,0.03015029802918434,-0.03640568256378174,-0.015703972429037094,-0.0120193837210536,-0.03578052297234535,-0.004227074328809977,-0.04714713990688324,-0.008670351468026638,-0.023414140567183495,-0.00647988636046648,0.023887991905212402,-0.017443330958485603,-0.009883265011012554,0.004583965055644512,0.0045952401123940945,0.004000450484454632,0.020097874104976654,-0.03839832916855812,0.004807732533663511,-0.013137241825461388,0.006424373481422663,0.013701317831873894,-0.02419520914554596,-0.000354949472239241,-0.0010745523031800985,-0.012133711017668247,0.06865239888429642,-0.03662195801734924,-0.004433418624103069,-0.0320836678147316,-0.006161244120448828,-0.007129179313778877,0.019383039325475693,0.018517648801207542,0.03318283334374428,0.01993001624941826,-0.03151266276836395,0.01657181605696678,-0.04769771173596382,-0.023217162117362022,0.03115389496088028,0.0037344854790717363,0.019596990197896957,-0.026819203048944473,-0.010630406439304352,-0.01917910948395729,0.007786108180880547,-0.008129295893013477,-0.013014078140258789,-0.025056708604097366,-0.001359487883746624,0.02083597704768181,-0.0024170821998268366,0.026222048327326775,0.003635784611105919,-0.00789398979395628,0.0015712741296738386,0.0362250842154026,-0.031276918947696686,-0.006678259000182152,-0.03371664509177208,-0.008746802806854248,0.015244794078171253,0.0018131999531760812,0.005407710559666157,-0.0005441837129183114,-0.00586351752281189,-0.006155264098197222,-0.014328676275908947,0.003661924507468939,0.002441331511363387,-0.025601623579859734,-0.012678084895014763,-0.006185202859342098,-0.03537547588348389,-0.0033379066735506058,-0.009049040265381336,0.011929714120924473,-0.036004748195409775,-0.02195044234395027,-0.03921817988157272,0.0024661284405738115,0.015271213836967945,-0.01674790307879448,-0.010218728333711624,-0.0052257198840379715,0.009074671193957329,0.03286176919937134,-0.02335970848798752,0.01016333419829607,0.0442512147128582,-0.020692570134997368,-0.013103711418807507,0.0056952862069010735,-0.021073592826724052,-0.039438072592020035,-0.006213201675564051,-0.042385492473840714,0.007222500629723072,0.015634113922715187,-0.041070736944675446,0.0009582927450537682,-0.03664640709757805,-0.04938074201345444,0.001490876660682261,0.01197650283575058,-0.001951630343683064,0.015785934403538704,0.01437116228044033,0.025310048833489418,-0.03669779375195503,-0.019031353294849396,0.009278659708797932,0.007139967288821936,-0.008264263160526752,0.03240935504436493,-0.0036554262042045593,0.0037953818682581186,0.011005212552845478,0.007403810974210501,-0.01439606212079525,-0.02725701965391636,0.014846962876617908,-0.007086945232003927,-0.014345903880894184,-0.009519629180431366,-0.015709487721323967,0.001809190260246396,0.03050282783806324,-0.0015988610684871674,-0.03703836724162102,0.021602381020784378,-0.026087500154972076,-0.026468560099601746,0.02036127820611,-0.013567350804805756,0.019908210262656212,-0.016476161777973175,-0.0070074936375021935,-0.004955696873366833,-0.03437802195549011,0.0014351793797686696,0.0015881286235526204,0.009430079720914364,0.014634589664638042,0.02186407335102558,-0.03837528079748154,0.010351221077144146,-0.016911588609218597,-0.014886453747749329,0.013385550118982792,0.010058059357106686,-0.010260870680212975,0.020600154995918274,0.0007443947251886129,0.007274840027093887,-0.02829049341380596,-0.03897042199969292,-0.04904303327202797,0.012711732648313046,0.007413977757096291,-0.03721877560019493,0.012386798858642578,-0.021901436150074005,-0.01706158183515072,-0.006582002621144056,-0.03286066651344299,0.003577388823032379,-0.0116258654743433,0.033767685294151306,-0.009672567248344421,0.01853187382221222,0.017301080748438835,-0.012196091935038567,0.0007180116372182965,-0.013477363623678684,0.025062477216124535,-0.030618542805314064,0.058702465146780014,0.0029667892958968878,0.015435351058840752,0.015869298949837685,-0.017837613821029663,-0.0013025450753048062,0.0006691031157970428,0.0012306577991694212,-0.005210723262280226,0.01562783122062683,-0.026647359132766724,-0.033227283507585526,0.0034284016583114862,0.041636619716882706,-0.007551092654466629,-0.015632979571819305,-0.022543368861079216,0.021474895998835564,0.015658073127269745,0.023935707286000252,-0.000008678483936819248,0.01143769733607769,0.022784100845456123,0.0193592831492424,0.0005993618979118764,0.015519926324486732,-0.021937448531389236,0.016499070450663567,-0.03145519644021988,0.003398893168196082,-0.03416622057557106,-0.03869443014264107,0.01505317073315382,-0.0025696989614516497,0.008020944893360138,0.013248646631836891,-0.001916668494231999,-0.02654431201517582,0.0005295672453939915,0.014294037595391273,-0.021158931776881218,-0.021527625620365143,-0.02215154841542244,0.035656727850437164,0.0029146927408874035,-0.0024859781842678785,-0.020434454083442688,-0.02422427013516426,0.03750944137573242,0.006173830013722181,-0.02485613524913788,0.03312735632061958,0.026846840977668762,-0.0203867107629776,-0.009074349887669086,0.016166184097528458,-0.02515099197626114,0.01875888556241989,0.0024134088307619095,-0.0027398590464144945,-0.02792530134320259,0.011878864839673042,0.02505829744040966,0.00039012328488752246,-0.006761571858078241,-0.01600414514541626,-0.0015294201439246535,-0.023948417976498604,-0.01289227046072483,0.04074708744883537,0.023160865530371666,-0.024107523262500763,0.011518262326717377,-0.01088558230549097,0.03635071963071823,0.010532603599131107,-0.008299595676362514,-0.0685909166932106,0.010029605589807034,0.0004576777864713222,0.016736051067709923,-0.00915433932095766,0.028094463050365448,-0.01615842990577221,0.0001378582528559491,0.0037299012765288353,0.0008524635341018438,0.009157671593129635,0.021297553554177284,-0.016759026795625687,-0.0007924129022285342,0.002374124713242054,-0.020575031638145447,0.01547573134303093,0.004493643995374441,0.053134579211473465,-0.02813638001680374,0.02432250790297985,-0.04440009221434593,-0.0316280797123909,0.008527335710823536,0.021472688764333725,0.018138103187084198,0.00021756731439381838,-0.04883526638150215,0.0011839086655527353,0.009020301513373852,-0.02158375456929207,-0.023044917732477188,0.028649644926190376,-0.03312399983406067,-0.009188404306769371,-0.027762670069932938,-0.024221809580922127,0.00405800249427557,0.05258597433567047,-0.04477847367525101,-0.04022582247853279,0.020190922543406487,0.01400039717555046,-0.025313694030046463,-0.001916348235681653,0.005557131487876177,-0.015018303878605366,-0.025281036272644997,-0.006455676630139351,0.01639048010110855,-0.025934258475899696,-0.010504878126084805,0.010294954292476177,-0.029208291321992874,0.009740525856614113,-0.003083680523559451,0.006991423666477203,-0.0048240674659609795,0.0033262157812714577,-0.015392951667308807,-0.012937430292367935,0.015343974344432354,-0.02448674850165844,-0.029327604919672012,-0.017669834196567535,-0.04447150230407715,0.017003554850816727,0.010389355011284351,-0.06664980947971344,-0.0012785647995769978,-0.02246190421283245,0.005143077112734318,0.006811458617448807,-0.02333330176770687,-0.017299499362707138,0.028497187420725822,0.012595298700034618,-0.06986308097839355,0.009842248633503914,-0.04555542394518852,-0.008612144738435745,0.024109914898872375,0.020183617249131203,-0.020054178312420845,0.01033328752964735,0.01785575971007347,0.023270340636372566,-0.02451990731060505,0.014299891889095306,-0.008909483440220356,0.022880008444190025,-0.0392623096704483,-0.0018133302219212055,0.03437480330467224,-0.0891222432255745,0.007946223020553589,0.0067281522788107395,0.015900803729891777,-0.003918438218533993,0.0033753952011466026,0.0021835961379110813,-0.0026962822303175926,-0.0050804796628654,-0.03597750514745712,0.0003162282519042492,-0.04856378957629204,0.0006047003553248942,0.015648940578103065,0.009548251517117023,-0.009370777755975723,0.006808259058743715,0.0010861477348953485,0.001406832248903811,0.03751282021403313,0.00678780535236001,0.022363783791661263,-0.020217739045619965,0.00324359443038702,0.0008289300021715462,-0.024063166230916977,-0.029920633882284164,0.002625570399686694,0.02280960977077484,0.01854970119893551,-0.002961813472211361,0.01971779577434063,-0.009154127910733223],"index":1}],"usage":{"prompt_tokens":84,"total_tokens":84}}, + "headers": { + "Content-Type": "application/json" + } + } + }, + { + "httpRequest": { + "method": "GET", + "path": "/v1/qianfan/token/*" + }, + "httpResponse": { + "body": {"refresh_token":"25.739889c11bd3da5314feb915f7a2fbb9.315360000.2040272418.282335-111798018","expires_in":2592000,"session_key":"9mzdDAf85Y5boZ09O03b02PJ4A8E3TsMKj7vdOtV37xs\/jC6CN9MpEIvHlgjfpybyPOM7xNj86XLqFkQPLxBDxQ7LRXFm0c=","access_token":"24.7214280a4e58a8bc2550e8527fe93058.2592000.1727504418.282335-111798018","scope":"public ai_custom_qianfan_bloomz_7b_compressed ai_custom_yiyan_com ai_custom_yiyan_com_128k ai_custom_yiyan_com_adv_pro ai_custom_yiyan_com_ai_apaas ai_custom_yiyan_com_ai_apaas_lite ai_custom_yiyan_com_aquilachat_7b ai_custom_yiyan_com_bce_reranker_base ai_custom_yiyan_com_bloomz7b1 ai_custom_yiyan_com_chatglm2_6b_32k ai_custom_yiyan_com_chatlaw ai_custom_yiyan_com_codellama_7b_ins ai_custom_yiyan_com_eb_instant ai_custom_yiyan_com_eb_pro ai_custom_yiyan_com_eb_turbo_pro ai_custom_yiyan_com_eb_turbo_pro_128k ai_custom_yiyan_com_emb_bge_large_en ai_custom_yiyan_com_emb_bge_large_zh ai_custom_yiyan_com_emb_tao_8k ai_custom_yiyan_com_emb_text ai_custom_yiyan_com_ernie_3.5_8k_0701 ai_custom_yiyan_com_ernie_35_8k_0329 ai_custom_yiyan_com_ernie_35_8k_0613 ai_custom_yiyan_com_ernie_35_8k_preview ai_custom_yiyan_com_ernie_40_8k_0329 ai_custom_yiyan_com_ernie_40_8k_0613 ai_custom_yiyan_com_ernie_40_8k_beta ai_custom_yiyan_com_ernie_40_8k_preview ai_custom_yiyan_com_ernie_40_turbo_8k(2) ai_custom_yiyan_com_ernie_40_turbo_8k_preview ai_custom_yiyan_com_ernie_char_8k ai_custom_yiyan_com_ernie_char_fiction_8k ai_custom_yiyan_com_ernie_func_8k ai_custom_yiyan_com_ernie_lite_8k ai_custom_yiyan_com_ernie_novel_8k ai_custom_yiyan_com_ernie_tiny_8k ai_custom_yiyan_com_fuyu_8b ai_custom_yiyan_com_gemma_7b_it ai_custom_yiyan_com_llama2_13b ai_custom_yiyan_com_llama2_70b ai_custom_yiyan_com_llama2_7b ai_custom_yiyan_com_llama3_70b ai_custom_yiyan_com_llama3_8b ai_custom_yiyan_com_mixtral_8x7b ai_custom_yiyan_com_qf_chinese_llama_2_13b ai_custom_yiyan_com_qf_chinese_llama_2_70b ai_custom_yiyan_com_qianfan_chinese_llama_2_7b ai_custom_yiyan_com_qianfan_dynamic_8k ai_custom_yiyan_com_sd_xl ai_custom_yiyan_com_sqlcoder_7b ai_custom_yiyan_com_tokenizer_eb ai_custom_yiyan_com_xuanyuan_70b_chat ai_custom_yiyan_com_yi_34b brain_all_scope wenxinworkshop_mgr wise_adapt lebo_resource_base lightservice_public hetu_basic lightcms_map_poi kaidian_kaidian ApsMisTest_Test\u6743\u9650 vis-classify_flower lpq_\u5f00\u653e cop_helloScope ApsMis_fangdi_permission smartapp_snsapi_base smartapp_mapp_dev_manage iop_autocar oauth_tp_app smartapp_smart_game_openapi oauth_sessionkey smartapp_swanid_verify smartapp_opensource_openapi smartapp_opensource_recapi fake_face_detect_\u5f00\u653eScope vis-ocr_\u865a\u62df\u4eba\u7269\u52a9\u7406 idl-video_\u865a\u62df\u4eba\u7269\u52a9\u7406 smartapp_component smartapp_search_plugin avatar_video_test b2b_tp_openapi b2b_tp_openapi_online smartapp_gov_aladin_to_xcx","session_secret":"ea141799c04c6cdbbdf4b7ab2b04f358"}, + "headers": { + "Content-Type": "application/json" + } + } + }, + { + "httpRequest": { + "method": "POST", + "path": "/v1/doubao/embedding" + }, + "httpResponse": { + "body": {"created":1725001256,"id":"02172500125677376580aba8475a41c550bbf05104842f0405ef5","data":[{"embedding":[-1.625,0.07958984375,-1.5703125,-3.03125,-1.4609375,3.46875,-0.73046875,-2.578125,-0.66796875,1.71875,0.361328125,2,5.125,2.25,4.6875,1.4921875,-0.77734375,-0.466796875,0.0439453125,-2.46875,3.59375,4.96875,2.34375,-5.34375,0.11083984375,-5.875,3.0625,4.09375,3.4375,0.2265625,9,-1.9296875,2.25,0.765625,3.671875,-2.484375,-1.171875,-1.6171875,4.1875,2.390625,-6.90625,0.369140625,0.259765625,3.671875,-2.9375,-1.9140625,-0.71875,-1.6640625,0.29296875,0.396484375,-4.625,-1.9921875,5.15625,-1.3984375,3.015625,-3.203125,-1.453125,4,-8.75,-5.625,1.0546875,-3.28125,-1.2265625,0.287109375,2.09375,4.6875,0.1572265625,0.42578125,0.79296875,3.234375,-0.169921875,0.9296875,7.40625,-3.296875,5.53125,3.890625,0.62109375,1.1171875,-0.373046875,4.125,-2.78125,0.333984375,3.9375,4.59375,6,1.53125,-0.373046875,1.109375,-4.0625,1.96875,1.421875,0.57421875,-0.56640625,-2.390625,0.734375,1.1875,-2.859375,-6.09375,-5.96875,1.8359375,-3,0.80859375,-0.130859375,-5.3125,-2.859375,1.484375,-4.53125,-6.90625,-2.25,0.7734375,-1.2734375,1.1484375,3.421875,-3.484375,2.65625,1.3359375,1.1484375,-4.09375,-5.625,2.625,-0.283203125,-3.46875,2.3125,-0.220703125,4.21875,3.75,-0.37109375,0.9609375,7.25,-0.87890625,7.03125,2.34375,4.5,-1.609375,-6.46875,-6.125,-2.59375,2.234375,3.78125,1.3046875,-5.5,1.953125,-3.421875,-5.9375,3.25,-3.4375,-8.3125,-2.546875,3.640625,0.267578125,-0.220703125,0.294921875,-0.4140625,2.515625,-1.0546875,-5.21875,6.6875,3.640625,0.2314453125,-2.5,1,1.6640625,0.59765625,2.75,1.1328125,1.1328125,-4.96875,4.53125,-0.349609375,3.390625,-0.193359375,7.625,2.921875,-3.484375,4.1875,8.5,-1.9140625,6.3125,2.5625,3.0625,0.40234375,0.76953125,-4.78125,3.53125,-2.765625,0.1591796875,-0.1025390625,-3.875,2.203125,0.03076171875,1.765625,1.859375,2.15625,-1.2578125,-4.40625,-0.62890625,4.4375,-1.78125,2.671875,2.765625,-1.7890625,-8.3125,-0.02197265625,1.640625,-3.96875,-3.15625,2.796875,1.1875,2,1.15625,2.359375,1.3984375,4.21875,-2.953125,8.5,3.46875,3.578125,0.90625,-1.8828125,2.15625,3.921875,4.125,-0.9609375,-2.171875,2.328125,2.921875,1.9765625,1.0703125,4.03125,6.28125,-3.59375,-0.94921875,5.6875,-1.9140625,-5.1875,-4.25,-7.71875,1.7109375,-1.59375,3.765625,-5.3125,-3.9375,-3.796875,2.90625,2.859375,-2.203125,-1.78125,-3.796875,0.1708984375,-5.15625,0.298828125,1.828125,-7.1875,1.6953125,6.125,2.671875,0.1728515625,3.375,0.609375,-4.78125,0.40625,-3.875,-6.4375,0.6953125,1.171875,-2.140625,5.8125,-1.640625,5.90625,-0.1650390625,4.9375,-2.421875,1.609375,-3.171875,-4.71875,7.6875,-1.09375,-1.9296875,0.033447265625,-3.46875,-2.671875,-8.875,2.4375,-1.1015625,4.40625,-3.53125,1.546875,2.359375,-3.15625,10.625,7.46875,-3.0625,-0.044677734375,0.90234375,-5.28125,-3,-1.2890625,0.59375,-6.34375,-1.8203125,5.40625,5.78125,-1.578125,2.46875,-2.171875,-1.71875,-0.38671875,-2.21875,-0.150390625,4.65625,-3.46875,1.5625,4.4375,-2.609375,1.6875,-2.828125,-6.03125,5.15625,-2.296875,-1.65625,-2.3125,-4.75,-3.3125,-3.703125,-1.9296875,-6.59375,3.640625,-0.62890625,4.8125,0.11279296875,2.515625,0.9921875,-3.03125,-5.40625,7.5625,-1.765625,4.4375,4.25,-0.140625,3.671875,-2.984375,-2.734375,2.203125,-6.96875,-1.1640625,2.390625,1.3515625,-1.75,2.96875,-3.75,-0.109375,2.5,0.796875,5.21875,7.8125,-4,1.171875,0.435546875,1.2734375,-3.015625,1.90625,-1.21875,5.9375,-0.31640625,-4.03125,-3.59375,1.09375,4.65625,-0.81640625,-2.046875,0.80859375,-5.375,2,-2.265625,5.34375,-0.46875,-1.3359375,-2.953125,-4.875,-0.53515625,-3,1.8203125,-2.59375,-1.4765625,6.28125,2.09375,0.1318359375,2.40625,-0.09130859375,-2.421875,-1.78125,1.59375,0.48828125,-0.310546875,-0.2353515625,0.1748046875,0.4453125,2.078125,-1.046875,1.46875,0.6953125,-0.52734375,-0.19140625,-2.28125,-0.515625,0.47265625,-1.2421875,-8.3125,1.1875,2.015625,-4.46875,3.734375,1.453125,-2.8125,-2.78125,5.875,-0.38671875,1.171875,-6.5,1.8046875,-2.15625,4,3.375,-0.78125,0.87890625,-1.796875,-1.265625,2.59375,3.96875,1.7421875,2.296875,2.78125,-5.8125,-2.046875,-0.1201171875,-4.1875,3.96875,-3.484375,-4.125,1.21875,3.484375,0.3828125,3.8125,1.90625,-8.3125,-2.15625,2.578125,2.578125,-1.34375,-3.359375,4.71875,-1.640625,-3.484375,2.046875,3.0625,-1.03125,-2.96875,6.96875,3.703125,-0.29296875,-0.423828125,2.640625,-1.265625,3.9375,-0.314453125,-4.15625,-2.171875,0.2734375,6.375,-6.21875,-6.3125,4.6875,-0.053466796875,0.045166015625,2.765625,2.953125,1.078125,-0.453125,1.96875,-6.71875,-3.375,-4.1875,2.515625,-0.5390625,-1.9296875,-4.03125,1.1953125,8.1875,1.0078125,0.80859375,-1.15625,-1.53125,2.875,-3.921875,1.953125,4.09375,6.59375,-4.5625,-1.2109375,3.5,-8.1875,0.294921875,-3.453125,-0.9921875,-2.015625,1.5,0.6328125,-4.90625,-2.765625,1.0546875,4.25,-2.390625,-5.96875,7.15625,-5.4375,-3.953125,1.15625,-0.017822265625,2.90625,2.78125,-2.21875,3.390625,1.9453125,2.171875,1.8671875,-1.125,-3.65625,-1.359375,0.96484375,2.5625,-2.9375,1.2734375,4.15625,-6,-0.2021484375,-1.8515625,-0.56640625,-1.671875,1.546875,5.8125,-0.640625,1.140625,-5.71875,-0.40625,0.5390625,-1.6640625,0.3203125,-2.375,4.9375,-2.453125,-1.59375,0.1669921875,1.6796875,-0.81640625,1.765625,-3.125,-1.234375,0.84375,-0.96484375,0.232421875,-0.01300048828125,-6.03125,4.25,5.625,0.65625,-1.6015625,1.390625,5.65625,3.0625,0.287109375,-0.08203125,4.15625,-1.5703125,-0.609375,-6.34375,2.203125,-3.84375,-2.53125,-3.390625,3.15625,4.59375,-4.46875,5.0625,-3.09375,3.328125,-0.65625,1.8515625,-9.375,1.609375,-1.515625,-2.5625,-2.953125,0.734375,2.375,1.3515625,0.390625,1.8671875,0.07080078125,1.328125,3.6875,0.2421875,0.73828125,3.1875,1.65625,2.75,2.859375,-2.8125,-7.75,1.53125,-1.1015625,-1.6875,6.3125,3.03125,-2.46875,0.77734375,-0.34765625,-1.78125,-1.4453125,3.40625,3.140625,-3.96875,3.984375,-3.21875,5.375,-2.890625,2.90625,-2.375,-6.1875,-2.4375,0.34375,-4.46875,-2.421875,3.40625,-1.2578125,4.59375,4.125,-6,0.003936767578125,1.1484375,2.359375,4.09375,0.5703125,-1.328125,-6.03125,4.5,3.234375,-2.140625,5.03125,-2.640625,0.041748046875,-0.90234375,4.375,-2.125,-0.1630859375,2.421875,-2.078125,1.1328125,-3.53125,1.0234375,-0.2734375,-9.125,-6.03125,0.73828125,-0.87109375,6.59375,-0.65625,-2.109375,-3.359375,2.40625,-0.0157470703125,5.96875,2.390625,3.078125,5.65625,5.09375,-1.5859375,1.78125,-0.921875,-8.0625,7.0625,-5.71875,-2.375,2.359375,2.65625,-1.453125,-1.2265625,1.984375,-2.125,-5.46875,-5.25,-1.78125,-4.28125,3.375,-2.09375,1.984375,-0.75,-5.0625,1.46875,-1.8671875,-2.875,-1.859375,2.609375,-5.5,2.484375,5.65625,1.875,-0.94921875,3.890625,4.125,0.8984375,-2.796875,0.95703125,-7.9375,1.7890625,3.453125,-1.9296875,-0.69140625,-5.84375,2.171875,-3.4375,2.921875,0.890625,-2.203125,-2.375,-1.6328125,-2.65625,0.8515625,-7.28125,2.484375,1.6484375,-0.8359375,-0.859375,0.232421875,1.921875,0.73046875,-0.30078125,1.515625,4.9375,0.7109375,-0.43359375,-3.140625,-2.796875,-0.2431640625,2.265625,-2.53125,6.875,-0.54296875,-1.5625,3.96875,0.44921875,-3.640625,-4.25,4.375,-1.875,0.45703125,-1.2265625,5.65625,0.298828125,3.921875,-1.703125,-2.8125,-3.328125,1.7578125,3.3125,-1.6875,-3.234375,2.09375,2.375,5.40625,-3.234375,-7.09375,1.984375,4.125,-0.8046875,-2.71875,8.6875,-1.296875,-2.625,-3,-3.78125,1.359375,1.515625,2.875,0.11279296875,-1.5859375,1.078125,3.46875,-1.390625,0.6328125,0.24609375,-3.765625,3.515625,0.380859375,2.609375,-0.80078125,-2.484375,-2.15625,-1.3203125,0.02490234375,4.03125,8.25,-1.5234375,-1.1953125,1.2109375,0.3125,-1.7421875,5.625,-0.76953125,5.90625,1.15625,0.1640625,1.390625,0.82421875,-0.322265625,3.21875,-4.65625,-4.5,-1.765625,3.171875,-4.3125,-1.4375,-2.546875,-0.9140625,4.28125,0.609375,-3.171875,3.671875,0.48046875,-0.9140625,-4,-2.4375,-5.34375,-1.96875,0.828125,1.953125,-2.140625,-2.59375,-0.353515625,4.78125,-4.09375,-3.921875,0.03173828125,1.8359375,1.3984375,-0.65234375,-1.15625,0.1611328125,0.50390625,2.90625,-1.875,-3.40625,0.498046875,8.75,3.90625,-4.53125,0.67578125,-0.765625,1.8359375,-5.3125,-2.15625,-0.6796875,-1.8984375,-3.046875,-1.7734375,-1.390625,-2.71875,-2.015625,5.84375,-3.28125,0.55859375,0.8046875,3.984375,0.99609375,3.015625,0.458984375,5.3125,3.1875,-1.2421875,-5.84375,-1.3828125,-0.04052734375,-5.75,-1.8828125,3.234375,6,3.171875,1.5703125,-2.828125,0.033203125,-0.953125,0.640625,5.3125,-5.75,-3.78125,-1.984375,-7.9375,-6.84375,-3.859375,-2.65625,-3.15625,-6.84375,-0.9765625,-1.375,-7.1875,-1.1328125,-2.109375,-1.546875,-1,0.640625,4.625,-4.65625,2.3125,3.703125,2.6875,3.0625,-2.28125,3.34375,0.474609375,-1.46875,0.34765625,-2.03125,5.25,-1.4609375,5.875,3.984375,-0.87890625,-3.8125,4.46875,4.40625,5.90625,-4.875,-3.53125,-2.53125,-1.8125,-0.39453125,-1.2421875,2.203125,-3.828125,-3.59375,-1.0859375,-3.453125,0.1845703125,5.625,0.421875,5.3125,-1.3671875,0.30859375,1.5234375,2.953125,0.1064453125,2.59375,1.5546875,-4.46875,3.609375,-0.81640625,1.390625,0.8359375,-2.78125,2.125,-1.6875,0.365234375,2.234375,3.875,10.4375,1.15625,2.328125,-0.09326171875,-0.76171875,-2.609375,-2.96875,2.40625,1.6796875,1.4921875,-3.65625,0.74609375,-0.8828125,2.03125,-0.306640625,-16.875,-3.328125,-5.53125,-2.109375,4.625,-1.0546875,-1.984375,1.0625,3.6875,2.671875,7.09375,-1.484375,4.03125,-1.078125,-0.7265625,2.515625,-4.3125,1.578125,3.6875,1.890625,4.625,1.7734375,-1.8125,-2.828125,6.9375,5.0625,-4.5,0.193359375,5.09375,-1.3515625,-1.140625,4.40625,-2.96875,2.078125,-4.75,3.078125,7.09375,2.75,-2.953125,-4.125,-2.375,-2.0625,1.0234375,3.046875,-2.578125,1.578125,2.921875,-5.65625,2.28125,2.28125,-0.259765625,-3.484375,-0.37109375,2.71875,1.625,-0.158203125,-4.5,2.5625,0.98828125,3.84375,4.8125,-2.796875,-2.140625,2.34375,2.90625,2.1875,1.5546875,2.578125,2.8125,-1.8515625,-2.984375,0.310546875,-1.328125,-0.0234375,-1.9765625,0.83984375,3.65625,2.046875,-4.5625,2.171875,2.234375,-2.109375,-0.0439453125,-4.0625,-3.5,2.09375,-2.21875,-2.5,0.703125,-2.953125,-1.28125,3.234375,-4.6875,4.1875,-2.484375,8.75,-0.53125,-1.8203125,1.171875,-3.0625,4.78125,-2.484375,-3.453125,3.765625,-2.6875,1.5625,-3.828125,1.9296875,-1.765625,1.2421875,5.0625,-4.65625,-2.0625,0.53125,3.265625,-2.875,-2.296875,0.29296875,3.859375,0.123046875,-4.46875,4.09375,-2.796875,3.96875,-3.890625,1.875,-4.46875,-0.5078125,2.140625,0.3203125,4.84375,5.03125,-5.34375,-4.96875,-1.3203125,-5.03125,-4.875,-4.5625,5.03125,-2.625,-0.75,1.046875,2.109375,-0.130859375,1.890625,-1.8125,2.53125,6.53125,-2.09375,0.87890625,-0.41015625,-0.412109375,-4.09375,-2.421875,-4.46875,6.40625,0.43359375,1.2578125,3.734375,-1.7109375,2.953125,1.8125,-1.1171875,-1.7109375,2.15625,1.859375,-2.015625,-2.25,1.7734375,-3.578125,4.15625,-3.328125,-3.28125,-4.71875,2.953125,1.40625,-0.287109375,1.5703125,3.53125,1.578125,3.171875,-4.34375,-3.125,5.78125,3.453125,-2.046875,4.3125,-1.2265625,-1.84375,0.640625,2.625,0.12890625,-3.25,-4.6875,5.28125,2.65625,2.015625,-4.4375,-5.75,-3.625,4.0625,4.59375,-0.78125,-2.484375,-2.03125,-3.75,1.6875,-4.15625,2.734375,-1.65625,-3.453125,-0.89453125,3.71875,2.453125,-4.15625,2.09375,0.82421875,-2.03125,0.052978515625,4.4375,1.734375,-3.71875,1.375,-0.349609375,-1.75,-7,3.59375,-2.625,-0.427734375,-4.40625,-3.84375,-3.265625,-3.796875,0.74609375,2.65625,1.6171875,3.609375,-0.7890625,3.890625,2.796875,-0.8671875,-0.43359375,2.15625,-1.7578125,-3.640625,2.375,-4.65625,-3.5,1.3984375,-7.1875,-1.5,5.0625,-2.625,4.0625,-1.171875,3.796875,-1.453125,-2.9375,-4,-1.3046875,0.91796875,6.59375,0.64453125,-0.91796875,0.64453125,2.703125,2.1875,-2.296875,-1.015625,-1.9921875,5,-0.298828125,2.953125,-5.125,-5.03125,5.375,-1.1328125,-4.46875,-0.5546875,-3.09375,1.5703125,5.34375,0.765625,-4.46875,-2.421875,-6.75,2.8125,-1.6171875,3.109375,-5.59375,0.87109375,-4.875,2.53125,4.46875,-7.21875,-3.1875,2.4375,3,5.1875,1.84375,-2.625,-6.21875,2.21875,0.306640625,-1.90625,-4.09375,-2.34375,-1.3046875,-3.875,4.4375,-2.328125,2.546875,-3.875,-2.40625,0.80078125,0.34765625,1,0.828125,1.4453125,-0.859375,3.03125,1.109375,5.15625,1.1953125,-3.8125,2.734375,4.21875,0.345703125,-1.2109375,2.0625,-0.79296875,-2.8125,2.109375,2.96875,-2.90625,5.15625,3.359375,4.3125,-5.53125,-2.875,1.515625,3.515625,-2.75,1.7109375,-4.9375,0.7265625,3.71875,-0.4765625,1.34375,0.049560546875,2.796875,-1.421875,-1.7890625,1.5,2.3125,4.21875,1.6875,3.015625,3.3125,-1.1640625,3.546875,-0.375,-1.2265625,-1.59375,3.609375,-3.015625,-2.546875,-4.625,1.046875,-1.796875,4.75,2.515625,1.1484375,0.8984375,-1.4140625,-2.328125,0.037841796875,-5.78125,-1.5859375,0.11669921875,3.015625,-0.83984375,0.84375,-0.82421875,0.96484375,4.0625,0.0400390625,4.25,-2.28125,1.3515625,1,1.5625,-2.8125,3.15625,-2.609375,-0.142578125,1.578125,-2.875,3.75,-4.3125,-1.359375,-2.578125,-0.69140625,2.84375,3.75,-4.75,-5.5625,0.84765625,0.380859375,5.125,3.0625,-3.140625,-0.93359375,0.73046875,0.0303955078125,4.3125,0.85546875,2.703125,-4.28125,5.625,5.90625,0.4296875,0.76953125,-0.9140625,-1.71875,-4.5,3.828125,-0.4609375,2.21875,-1.9453125,2.5,4.15625,1.8984375,3.984375,-5.75,2.953125,0.2734375,3.890625,-0.76171875,-3.90625,0.337890625,1.96875,0.69140625,-0.70703125,3.578125,0.046142578125,0.765625,-2.734375,4.28125,4.3125,2.578125,-4.40625,1.921875,-2.90625,1.7734375,-3.90625,1.1484375,-5.625,1.65625,2.703125,5.34375,-1.9296875,-6.1875,4.5,1.5625,-0.9140625,-3.953125,4.65625,4.5625,2.484375,-5.15625,-2.375,1.625,-1.328125,-0.26171875,-5.25,3.328125,2.0625,-3.609375,-3.71875,1.6171875,1.046875,-3.1875,-3.71875,-3.34375,1.9609375,2.5625,3.609375,1.59375,-2.484375,4.125,-0.80078125,1.9140625,4.78125,-1.09375,0.140625,3.171875,-3.578125,2.640625,-0.6640625,-2.65625,-1.4375,0.47265625,-2.46875,2.6875,-2.515625,-1.0234375,-2.09375,-0.138671875,-0.5078125,1.5,4.15625,-3.09375,0.158203125,4.4375,-1.96875,-3,-1.9609375,2.09375,-1.7734375,-1.09375,-1.8984375,3.3125,1.9765625,0.8671875,0.2890625,0.66796875,-1.9765625,-3.640625,-4.90625,2.0625,-4.0625,3.59375,-0.84765625,-6.21875,1.515625,3.890625,3.640625,-0.2734375,-2.046875,0.875,3.78125,0.07470703125,-1.078125,-1.4921875,3.671875,-2.796875,-3.6875,2.75,2.78125,-5.40625,1.7890625,-4.28125,-2.265625,-0.98046875,4.46875,0.173828125,-2.25,-2.875,-3.84375,-1.7421875,-1.6171875,-3.21875,1.9140625,1.7421875,2.671875,1.09375,1.4375,-3.5,2.59375,19.125,0.0101318359375,-8.4375,1.3515625,-3.625,4.4375,4.65625,1.8125,0.423828125,-1.5,0.62890625,4.21875,0.609375,0.5390625,-2.390625,0.984375,-0.79296875,2.078125,-3.703125,-3.109375,-2.265625,-1.0234375,-0.328125,1.9765625,2.5,2.375,0.8046875,-2.265625,1.2734375,-3.390625,-4.375,-4.71875,3.765625,-2.921875,3.125,-3.171875,4.65625,-0.7890625,-3.3125,-2.984375,-3.296875,-2.796875,2.375,-0.12255859375,-3.21875,5.21875,0.1982421875,0.2138671875,-1.1796875,-0.130859375,-4.34375,-1.4453125,-2.5,6.3125,1.0625,-6.15625,-0.5703125,-3.203125,-3.546875,-1.375,2.9375,-0.53515625,1.7578125,2.71875,-1.9453125,-2.640625,-3.046875,0.49609375,1.0078125,-3,-4.84375,0.2119140625,1.2265625,1.3515625,1.609375,-4.84375,2.46875,2.140625,2.171875,1.75,0.67578125,-0.60546875,-2.46875,-2.234375,-0.9453125,1.2421875,-3.15625,0.006744384765625,3.359375,-1.765625,8.375,-8.3125,5.8125,5.15625,-2.0625,-0.470703125,1.5,-0.30859375,-2.421875,-0.2294921875,0.95703125,1.8828125,4.84375,-0.68359375,4.625,1.359375,0.373046875,0.83203125,2.640625,4.34375,0.7578125,3.109375,-0.412109375,-2,2.15625,-0.08349609375,-3.140625,-3,-3.703125,-2.5625,3.6875,1.7890625,-3.296875,0.89453125,-7.5,-5.40625,-2.25,-7.625,4.34375,-1.34375,-0.14453125,3.515625,-2.46875,-1.2109375,-2.46875,-3.921875,1.265625,3.65625,1.4375,-1.46875,-5.03125,2.59375,3.890625,-2.765625,-2.4375,0.353515625,-4.21875,4.4375,-0.376953125,3.9375,-2.09375,3.96875,3.234375,-2.203125,-6.875,5.15625,-3.6875,-4.34375,-6.625,-2.90625,-4.9375,-3.34375,0.412109375,-0.9453125,-0.5703125,-1.3046875,3.21875,-0.65234375,-1.6796875,3.171875,3.453125,-4.4375,-1.2578125,0.828125,1.1796875,-4.375,0.1787109375,4,0.53515625,1.328125,-0.546875,0.271484375,-0.5546875,-3.859375,-0.2216796875,0.86328125,-4.53125,-1.3828125,-0.60546875,-5.46875,-1.3515625,-1.2890625,-3.734375,2.9375,2.40625,-3.984375,0.875,-2.953125,-0.9765625,-1.6328125,-1.25,3.96875,1.6953125,0.0072021484375,5.875,-0.921875,-3.46875,-3.140625,-0.26953125,0.2265625,-2.09375,7.0625,-1.09375,0.30078125,-6.03125,5.34375,2.359375,1.6640625,-0.99609375,4.625,4.25,-2.484375,-4,0.89453125,3.0625,4.1875,-4.28125,3.953125,0.6328125,-0.74609375,-1.53125,2.015625,-1.1796875,1.03125,-1.6484375,-5.4375,0.3671875,1.8125,-0.326171875,1.546875,4.03125,-3.34375,0.484375,2.5,-1.4140625,3.34375,4.25,-1.7890625,1.09375,2.171875,5.34375,-1.5625,0.98828125,-5.09375,-3.625,-2.640625,-2.46875,3.109375,-2.515625,0.09033203125,0.21484375,-3.921875,3.125,-4.1875,1.2109375,1.3671875,1.1875,-5.4375,4.59375,3.890625,-2.8125,3.328125,-5.125,-1.9765625,-1.4296875,2.34375,-2.71875,-5.875,3.125,3.453125,-1.515625,3.546875,2.265625,-0.52734375,1.9375,-2.859375,2.703125,-3.359375,4.75,1.2734375,3.09375,3.65625,-0.255859375,-0.1044921875,-5.75,-0.3359375,-0.77734375,-2.234375,6.1875,-3.84375,0.19921875,4.25,6.4375,-10.5,-1.5078125,0.7265625,0.2890625,3.921875,5.0625,0.09814453125,0.68359375,3.109375,1.015625,2.671875,0.0257568359375,-0.4765625,-4,5.15625,0.2314453125,-4.6875,3.1875,3.984375,-2.609375,3.4375,-2.375,-3.734375,-0.07568359375,2.75,-5.3125,1.9296875,4.625,-1.6484375,2.875,3.734375,-1.34375,3.875,-1.9921875,-11.3125,-1.53125,3.296875,5.71875,0.80859375,1.7578125,0.48046875,-2.015625,1.4765625,-0.5546875,0.71484375,-0.7578125,-11.1875,0.9765625,-3,-0.09765625,-1.9453125,-3.8125,-2.5,4.375,1.65625,1.1015625,3.328125,2.84375,0.84375,4.5625,0.11279296875,-5.84375,1.1484375,1.7578125,-4.8125,-0.59765625,3.234375,1.125,-1.859375,-2.515625,3.78125,-1.7421875,-0.69921875,5.8125,3.765625,1.578125,-1.84375,-5.03125,0.984375,-3.375,-1.9140625,1.1953125,-0.384765625,2.8125,-2.203125,2.828125,1.1171875,-3.75,-4.15625,-2.25,-3.5625,1.5,2.671875,2.171875,-2.609375,-1.7265625,2.8125,2.5,-0.455078125,-1.546875,2.1875,-0.1884765625,-2.984375,-1.4765625,2.0625,-4.46875,-2.90625,4.0625,1.8359375,0.443359375,-0.7734375,-3.140625,2.171875,1.734375,-1.8515625,-1.84375,-1.234375,2.15625,5.34375,-2.484375,-5.6875,-1.2734375,0.1806640625,-4.375,-3.5625,0.89453125,-1.15625,0.75,3.09375,-2.25,1.1875,4.6875,-1.3359375,-3.875,3.53125,4.4375,-2.671875,-0.75,-0.458984375,-2.53125,3.8125,5,-1.2421875,-2.109375,-0.50390625,-2.734375,-4.90625,1.0234375,2.421875,-3.34375,-10.125,6.46875,3.671875,5.40625,1.546875,-2.59375,3.8125,-1.6953125,3.703125,-0.423828125,0.82421875,1.515625,-7.59375,-2.40625,-2.0625,-5.0625,0.59375,-0.345703125,-4.75,1.4921875,6.25,-2.15625,-1.8671875,-2.703125,-3.9375,4.28125,-3.484375,-5.9375,1.984375,-7.4375,1.4609375,-1.9609375,3.265625,-5.875,1.8359375,-0.017333984375,2.046875,-0.5859375,-0.671875,-2.328125,1.1953125,-2.65625,3.625,0.7890625,3.9375,-0.365234375,2.90625,-1.2421875,0.314453125,-3.265625,1.6640625,1.7109375,0.60546875,0.384765625,2.296875,-2.28125,-0.8046875,-1.0546875,1.046875,2.796875,0.61328125,-0.625,0.10693359375,4.21875,-0.6484375,2.03125,-2.3125,-0.173828125,-1.015625,-0.224609375,0.74609375,-0.86328125,0.0145263671875,0.1318359375,1.7109375,1.421875,0.486328125,-0.19921875,0.140625,1.2734375,1.015625,1.5625,-1.65625,-0.45703125,-0.435546875,-0.0206298828125,1.828125,1.734375,-2.734375,1.65625,-2.09375,-0.6875,-0.2421875,2.125,1.1015625,0.1064453125,1.59375,-1.875,1.828125,0.15234375,-1.2421875,1.25,-0.765625,-2.265625,2.34375,-2.109375,-0.921875,0.6640625,-1.2734375,-1.4765625,-0.73828125,2.21875,-0.84375,1.328125,-1.171875,-0.181640625,0.306640625,-1.171875,0.279296875,0.94140625,1.171875,-3.921875,3.15625,1.2421875,0.52734375,-0.1630859375,1.0390625,-1.46875,-0.08447265625,1.0390625,-0.37109375,0.921875,1.859375,-1.8046875,0.54296875,-0.8203125,-1.09375,1.1640625,1.515625,0.54296875,-1.65625,-1,1.5234375,1.4453125,-1.1953125,0.359375,-0.062255859375,-2.09375,3.03125,1.21875,-3.15625,-0.357421875,-0.169921875,0.546875,-0.73828125,-0.126953125,1.046875,-2.75,-0.2314453125,0.2421875,0.306640625,-1.1328125,1.8984375,0.00469970703125,3.9375,0.8515625,1.1328125,1.1875,1.3984375,2.046875,-1.3515625,0.25390625,-0.9921875,3.234375,-0.373046875,0.8828125,1.3828125,-1.921875,-0.484375,-0.81640625,0.61328125,1.4296875,-0.70703125,-0.404296875,2.53125,1.625,0.494140625,2.375,-2.03125,0.33984375,0.291015625,-0.68359375,-1.625,1.625,-0.478515625,0.349609375,-2.0625,-1.25,-0.1484375,-0.44140625,0.67578125,0.3671875,0.4921875,0.236328125,1.1953125,0.5078125,-2.375,1.3671875,-0.341796875,0.6328125,-1.7265625,-1.328125,0.84375,-0.08935546875,1.0625,0.90625,1.984375,2.828125,1.109375,-1.3671875,1.03125,1.0625,1.75,0.263671875,-1.234375,-0.09228515625,-0.13671875,0.271484375,0.58203125,-0.9375,-1.28125,0.4609375,-0.95703125,-0.1552734375,-1.5703125,3.375,-0.9609375,-1.1796875,-0.419921875,-1.5,0.58984375,-1.3125,1,-1.578125,2.484375,1.34375,3.34375,1.4296875,-0.671875,-0.984375,0.30859375,0.72265625,-0.337890625,-0.06982421875,-1.125,-0.44921875,-0.62890625,5.40625,0.263671875,1.0390625,-2.03125,3.296875,0.68359375,-0.10986328125,-1.078125,-0.2412109375,-2.078125,-0.13671875,-1.4375,-1.390625,0.29296875,-1.1484375,-4.0625,-2.703125,-0.302734375,0.77734375,-1.640625,-0.0390625,3.890625,0.375,1.2890625,1.5,2.640625,0.19140625,-1.78125,-0.5859375,1.6328125,-1.234375,2,0.8125,-1.9453125,-2.78125,-0.3671875,-2.328125,-1.9453125,-0.59375,-0.8046875,1.9921875,-0.265625,-0.03515625,-1.3125,-1.5234375,-3.03125,-0.458984375,-0.1279296875,2.375,1.53125,0.67578125,-0.55078125,-0.4296875,0.515625,-1.75,0.6640625,-1.65625,4.25,-0.326171875,-1.4296875,2.53125,0.396484375,3.140625,0.859375,-1.3671875,-1.8828125,-0.828125,0.45703125,0.7109375,3.0625,-0.2578125,0.6328125,0.57421875,-0.85546875,0.5625,1.0234375,-0.296875,-4.84375,-1.578125,-0.486328125,2.59375,-1.2109375,0.09765625,2.59375,-0.87109375,-0.7890625,-1.7421875,-2.34375,-0.2490234375,-0.82421875,0.8046875,2.078125,-0.7265625,-0.10400390625,-0.703125,-1.046875,0.46875,-1.7734375,1.09375,-0.30859375,0.0181884765625,0.2734375,-2.703125,-0.470703125,0.67578125,-1.921875,-1.0078125,1.6328125,0.2021484375,1.359375,1.6796875,-1.6015625,1.5703125,0.6484375,-2.859375,-0.63671875,-0.8359375,1.34375,0.0556640625,0.4375,1.765625,-1.1484375,-1.90625,-1.453125,0.57421875,0.84375,-0.349609375,0.251953125,-0.0927734375,0.416015625,-0.40625,-2.71875,-0.48046875,0.4140625,-0.2109375,0.96484375,1.0859375,1.453125,1.15625,1.375,-0.478515625,1.375,-1.8828125,1.6484375,0.9921875,-2.171875,0.5859375,2.03125,-2.125,0.314453125,1.1796875,-0.4921875,-0.72265625,-0.80078125,0.5546875,-0.52734375,0.58203125,-0.52734375,1.9453125,1.71875,-0.328125,1.453125,-2.203125,-2.09375,-2.625,0.2177734375,-0.82421875,0.3359375,-2.203125,1.375,-1.7578125,-0.072265625,-0.4765625,-0.38671875,-1.9453125,1.5625,1.7578125,0.4453125,0.640625,0.0255126953125,-0.5703125,3.796875,-1.0703125,-0.1201171875,0.93359375,1.15625,-2.078125,3.484375,0.5234375,2.109375,0.0037078857421875,1.3359375,-0.796875,1.25,0.1455078125,0.86328125,0.478515625,1.828125,0.31640625,-0.296875,-0.154296875,-1.53125,-1.1640625,0.6484375,1.0703125,-5.375,0.86328125,0.890625,0.48828125,0.84765625,-2.828125,1.1015625,0.4765625,3.296875,-0.00408935546875,-0.40234375,3.421875,0.61328125,-1.46875,1.1875,0.953125,0.0771484375,-2.78125,-1.171875,-0.86328125,2.9375,-1.0703125,0.1015625,-0.279296875,-0.90625,3.046875,0.6796875,-1.6640625,1.453125,0.443359375,-0.439453125,-1.453125,-3.40625,-0.1689453125,1.71875,-0.9453125,2.234375,0.158203125,0.87109375,0.66796875,-1.640625,1,0.265625,0.267578125,-0.90625,1.75,-0.2041015625,-1.59375,1.65625,-1.1484375,-1.78125,2.421875,1.6953125,-2.328125,0.027587890625,-0.494140625,-0.3203125,-0.01953125,0.58203125,-2.28125,0.546875,0.62109375,0.90625,-0.921875,-1.53125,2.484375,1.890625,2.953125,2.359375,-0.90234375,0.171875,-2.234375,0.33984375,-0.45703125,-0.87109375,0.08251953125,1.8671875,-1.0078125,1.5703125,-0.30078125,0.921875,-1.8046875,1.609375,2.703125,0.92578125,0.40625,-0.26171875,-0.322265625,-1.8671875,-0.5,-2.296875,0.62109375,0.6953125,1.1640625,0.1376953125,-1.4296875,1.5390625],"index":0,"object":"embedding"},{"embedding":[-2.28125,-0.7734375,-0.8359375,-2.3125,3.046875,4.125,-1.0390625,-2.890625,0.0103759765625,1.9296875,0.1015625,1.75,2.4375,2.015625,5.09375,1.203125,-2.140625,-2.828125,-1.328125,-4.6875,1.0078125,6.8125,0.578125,-4.71875,-0.80859375,-6.25,1.578125,4.25,4.46875,-1.0078125,8,-2.3125,2.546875,-0.00555419921875,1.5625,-1.8671875,-2.375,-2.53125,5.25,-0.69140625,-2.96875,-0.68359375,1.6171875,2.96875,-3.015625,-1.734375,0.4140625,-2.9375,2.53125,-1.6640625,-4.5625,-1.9296875,3.234375,-2.734375,2.359375,-4.125,-3.046875,4.5,-5.875,-2.984375,-1.8515625,-2.8125,-0.7734375,0.46484375,1.3984375,5.28125,0.68359375,-1.3359375,0.51171875,8.625,-0.055908203125,3.578125,6.5,-2.390625,6.34375,5.5625,0.7265625,1.578125,-2.921875,4.90625,-2.953125,-0.62890625,2.453125,3.46875,4.5625,2.671875,-1.9140625,0.859375,-3.03125,1.703125,1.96875,0.59375,-1.4140625,-3.140625,-1.2109375,1.2890625,-3.21875,-6.5625,-6.78125,2.765625,-0.78515625,-0.3515625,1.8125,-4.53125,-5.03125,2.171875,-1.8515625,-5.46875,-1.78125,0.380859375,2.640625,1.65625,3.640625,-2.140625,2.46875,1.21875,4.28125,-2.796875,-4.40625,2.796875,-2.0625,-1.9765625,4.28125,-0.6796875,4.4375,4.28125,-4.03125,-0.01416015625,5.53125,-1.4609375,7.25,3.578125,3.6875,-2.375,-8.0625,-4.71875,-1.9453125,3.71875,4.3125,4.40625,-5.03125,3.21875,-3.734375,-6.625,4.1875,-3.4375,-6.4375,-3.15625,3.859375,-1.9140625,-1.78125,1.8046875,0.5,2.3125,-1.2421875,-4.375,4.0625,3.875,0.1259765625,-1.0546875,2.015625,3.328125,1.1484375,1.7265625,1.8046875,-0.462890625,-5.625,3.6875,-1.0390625,2.5625,0.90625,10.4375,4.28125,-4.5625,1.9765625,8.625,-1.328125,8.625,1.4609375,2.203125,0.81640625,-0.640625,-2.90625,4.53125,-2.15625,1.5,0.12255859375,-5.6875,3.140625,1.2890625,1.578125,1.5625,2.71875,-1,-4.84375,-1.8671875,3.484375,-2.578125,3.4375,0.1025390625,-1.40625,-7.375,1.4921875,1.5546875,-4.71875,-3.765625,2.703125,-1.71875,3.078125,-0.380859375,2.265625,0.24609375,3.21875,-2.0625,7.65625,2.640625,2.734375,2.046875,1.8359375,2.46875,4.53125,3.484375,1.8359375,-2.078125,-0.83984375,2.03125,5.8125,0.439453125,3.75,8.6875,0.251953125,0.408203125,6.84375,-2.515625,-1.78125,-3.578125,-3.78125,1.6015625,-0.279296875,2.671875,-5.65625,-4.0625,-2.328125,2.984375,3.515625,-3.359375,-2.34375,-2.703125,-0.51171875,-6.4375,1.484375,3.671875,-9.0625,1.8828125,5.625,3.96875,1.984375,1.265625,-0.33203125,-4.125,0.333984375,-2.4375,-5.875,-0.58203125,1.890625,-2.390625,5.09375,-1.5546875,3.515625,-0.7421875,5.1875,-2.28125,-0.0927734375,-3.046875,-4.3125,8.8125,-0.232421875,-1.90625,1.0703125,-3.078125,-3.5625,-10.25,2.5,1.1171875,4.96875,-2.921875,1.40625,0.40234375,-3.640625,12.75,3.90625,-1.8203125,1.9921875,-0.63671875,-6.03125,-1.984375,-2.046875,2.046875,-5.59375,1.84375,3.6875,4.5,-1.9296875,3.4375,-1.7421875,-0.9296875,-1.109375,-4.5625,-1.9375,2.671875,-3.765625,2.34375,9.625,-4.75,2.03125,-2.109375,-6.1875,4.75,-0.03662109375,-0.11376953125,-2.140625,-5.125,-1.9921875,-2.78125,-1.4296875,-6.65625,4.96875,-0.984375,5.375,0.97265625,3,3.296875,-4.1875,-5.03125,8.4375,-1.5,3.296875,5.71875,0.55078125,0.68359375,-3.515625,-4.6875,2.46875,-5.46875,0.953125,5.71875,3.328125,-1.640625,1.0234375,-6.21875,2.40625,2.328125,-0.68359375,6.53125,6.90625,-2.265625,2.78125,1.9140625,-0.71484375,-2.28125,-0.2294921875,-1.078125,6.34375,1.1875,-3.890625,-3.796875,-0.5859375,5.03125,-2.375,0.7734375,-1.21875,-4.15625,2.59375,-1.15625,3.6875,0.91796875,0.90625,-1.8046875,-5.125,0.087890625,-2.625,0.29296875,-1.7734375,-3.28125,4.25,1.515625,-0.484375,1.59375,0.67578125,-3.53125,-0.46484375,0.59765625,-1.15625,0.65625,2.5625,-0.5703125,-0.984375,1.5546875,-0.3828125,-2.21875,1.0546875,-1.2734375,2.40625,-6.9375,-0.6484375,-0.2490234375,-2.125,-8.375,-0.4765625,1.0703125,-3.78125,2.71875,1.96875,-1.2578125,-3.0625,4.4375,1.421875,1.8671875,-6.90625,2.15625,-1.8828125,3.328125,2.140625,-1.7421875,0.59375,-1.4296875,-2.765625,4.375,3.546875,-0.69921875,3.453125,0.68359375,-3.265625,-3.625,0.1630859375,-4.90625,4.75,-0.236328125,-1.859375,5.21875,2.203125,-1.5,1.625,0.98828125,-6.28125,-4.78125,2.96875,3.171875,-3.078125,-3.96875,0.470703125,-1.4296875,-4.4375,3.078125,3.84375,-1.1171875,-2.8125,3.40625,4.375,-2.203125,0.0830078125,1.1171875,0.52734375,2.703125,-1.9375,-3.140625,-0.1103515625,0.130859375,4.71875,-5.8125,-6.84375,3.015625,-2.875,0.2001953125,1.15625,4.5625,0.46875,-1.8984375,-1.9296875,-3.0625,-3.46875,-2.828125,3.53125,-1.078125,-2.53125,-2.90625,0.29296875,8.3125,1.90625,0.369140625,-2.375,-0.11572265625,2.453125,-1.71875,0.50390625,4.4375,7.90625,-4.03125,-0.63671875,3.53125,-8.125,0.94921875,-1.375,-1.15625,-0.94921875,2.3125,2.1875,-6.25,-0.7890625,0.0115966796875,5.03125,-3.453125,-3.828125,5.15625,-4.8125,-3.09375,1.859375,-0.6875,4.0625,1.296875,-1.34375,2.875,2.984375,2.65625,1.8203125,-2.53125,-3.640625,-3.3125,1.2890625,2.265625,-2.234375,2.296875,4,-5.4375,0.90234375,-2.25,-0.6953125,-0.212890625,-0.515625,5.90625,2.125,2.25,-6.09375,1.2578125,0.50390625,-0.416015625,-0.7421875,-1.1484375,6.71875,-0.5,-0.2294921875,0.94921875,2.09375,-1.1953125,1.640625,-3.796875,-2.453125,-3.109375,-1.796875,-1.0234375,-4.03125,-5.5,4.4375,6,-1.234375,-1.6796875,2.171875,5.5,3.984375,-0.84375,1.515625,3.421875,-2.5,0.23828125,-5.40625,2.609375,-7.84375,-2.53125,-1.6875,2.921875,3.75,-4.15625,3.765625,-2.578125,2.4375,-1.4375,4.4375,-10.5625,2.046875,-2.15625,-2.796875,-2.28125,-0.57421875,3.171875,-0.44921875,2.109375,1.3671875,-0.75,3.953125,5.46875,-1.5,1.765625,2.1875,2.46875,-0.5859375,2.515625,-2.125,-8.25,1.3125,-1.1484375,1.09375,7.5625,1.9375,-1.7734375,2.46875,0.88671875,-1.5703125,-1.7265625,4.0625,3.015625,-1.546875,4.25,-3.90625,5.40625,-3.28125,1.7265625,-3.265625,-6.15625,0.279296875,1.9296875,-5.5625,-4.09375,2.859375,0.216796875,5.78125,3.421875,-5.375,1.21875,-0.41796875,1.109375,2,0.30078125,-0.03759765625,-4.75,3.921875,4.1875,-2.40625,7.03125,-1.5703125,-1.6484375,-1.1171875,2.40625,-1.7734375,0.373046875,1.84375,0.287109375,-0.78125,-3.484375,0.96484375,0.5703125,-6.625,-7.21875,1.7265625,-1.7734375,7.0625,0.73046875,-0.859375,-3.15625,2,1.5546875,6.375,3.3125,3.765625,4.5,3.765625,-2.390625,2.671875,-3.6875,-6.09375,7,-6.53125,-1.8515625,1.015625,0.859375,-0.2578125,-1.0234375,-0.3515625,-0.71484375,-3.484375,-6.09375,-2.359375,-1.875,2.015625,-1.6484375,2.203125,0.57421875,-4.09375,-0.5703125,-1.6484375,-1.6875,-1.6640625,4.15625,-5.625,1.484375,5.71875,2.046875,-1.5234375,4.15625,3.09375,-0.47265625,-4.78125,0.7109375,-6.875,1.6015625,1.46875,-0.6015625,0.50390625,-8,2.03125,-2.4375,3.5,-0.671875,-0.05078125,-1.265625,-3.296875,-1.3984375,-0.91796875,-5.40625,-0.171875,1.6953125,1.125,-1.8359375,0.671875,3.078125,-0.52734375,0.384765625,-1.125,2.046875,0.40625,2.34375,-4.78125,-2.90625,1.28125,0.9140625,-2.03125,6.53125,0.91796875,0.79296875,3.546875,1.7265625,-5.5,-5.78125,3.921875,-2.8125,-1.796875,-3.25,2.421875,-1.359375,6.53125,-2.21875,-5.53125,-3.703125,1.6484375,3.15625,-2.609375,-3.09375,4.78125,1.8359375,2.765625,-2.15625,-7.5,1.609375,0.98828125,-0.146484375,-1.140625,8.625,-1.9296875,-0.4765625,-4.4375,-3.234375,2.046875,0.875,2.046875,-0.76171875,-1.2734375,0.69921875,0.4765625,-2.34375,-0.55078125,0.6015625,-2.546875,1.75,0.07177734375,4.875,-2.53125,0.3984375,-1.2734375,-0.50390625,-0.10009765625,4.3125,8.75,-1.765625,-0.96875,0.35546875,2.984375,-3.59375,6.6875,1.3515625,7.75,-1.1640625,0.25,1.03125,0.375,-2.171875,4.59375,-5.25,-2.84375,-1.890625,1.21875,-2.5625,0.671875,-3.984375,-0.498046875,4.40625,-0.455078125,-0.007568359375,2.609375,0.79296875,-0.201171875,-3.09375,-1.3125,-4.71875,-2.515625,-0.14453125,2.03125,-3.03125,-0.4921875,-0.33984375,5.84375,-0.357421875,-1.4453125,-2.59375,1.53125,1.859375,1.171875,-0.8046875,0.255859375,0.58984375,3.3125,-1.015625,-4.34375,-0.94921875,8.4375,4.21875,-6.875,1.5703125,-0.43359375,1.4453125,-4.8125,-1.4609375,-2.15625,-1.4921875,-4.1875,1.1328125,0.419921875,-3,-0.06494140625,4.5,-1.2890625,-0.15625,3.46875,4.0625,0.478515625,2.96875,-2.125,4.375,2.21875,-2.09375,-5.96875,-1.703125,0.48046875,-2.75,-1.4140625,2.03125,6.15625,0.55859375,2.625,-1.0625,2.28125,-1.6953125,3.78125,5.125,-4.59375,-2.703125,-2.3125,-9.5625,-4.03125,-1.7421875,-2.921875,-5.34375,-4.25,-0.86328125,-1.2421875,-8,0.0966796875,-2.234375,-3.265625,1.4453125,2.953125,1.7578125,-5.75,3.125,4.125,2.578125,2.546875,0.84765625,5.46875,-0.050537109375,-2.96875,1.4453125,-3.4375,4.15625,-1.03125,3.546875,6.25,-0.453125,-4.96875,4.78125,2.96875,5.53125,-7.375,-2.625,-0.337890625,-1.671875,-0.458984375,-1.7578125,2.546875,-4.5,-5.5,1.078125,-3.203125,1.2265625,4.6875,-0.8046875,6.78125,1.6328125,0.419921875,2.140625,2.71875,0.62109375,0.169921875,1.7421875,-5.9375,3.234375,-2.171875,3.265625,-0.296875,-1.5234375,2.734375,-0.7578125,-0.310546875,2.8125,2.734375,10.3125,0.515625,4,-2.3125,0.63671875,-1.7265625,-0.2392578125,2.25,2.015625,0.79296875,-1.4765625,0.7890625,-0.44921875,0.478515625,-0.4609375,-13.25,-1.9609375,-7.25,-1.9296875,7.0625,-2.1875,-1.9921875,1.4296875,2.6875,3.484375,5.125,-0.58984375,3.375,-0.60546875,0.80859375,5.96875,-4.25,1.03125,3.359375,2.546875,5.21875,0.154296875,-0.44921875,-3.203125,8,2.25,-1.4140625,0.8359375,2.796875,-1.3046875,-2.34375,3.09375,-3.171875,2.96875,-4.9375,0.5859375,4.15625,0.65625,-3.890625,-3.4375,-2,-0.62890625,1.3828125,1.375,-2.59375,0.18359375,0.94921875,-4.1875,3.328125,-0.59375,0.140625,-5.53125,1.03125,4.65625,0.703125,-0.109375,-1.8515625,1.4453125,-0.8984375,4.3125,2.78125,-2.734375,0.2734375,2.21875,1.7421875,-0.125,1.03125,1.1328125,2.921875,-3.09375,-0.353515625,-0.44140625,-1.625,1.4765625,-3.1875,1.6640625,3.203125,1.3984375,-3.984375,2.21875,0.79296875,-0.11669921875,2.96875,-5.125,-1.9921875,-1.1015625,-0.71484375,-4.0625,-0.9140625,-4.375,-0.1455078125,5.46875,-5,3.4375,-2.515625,8.1875,0.1298828125,-1.421875,1.2890625,-2.828125,2.59375,-3.390625,-1.234375,3.484375,-0.92578125,2.125,-3.546875,1.8984375,-2.078125,-0.46484375,6.09375,-3.953125,-1.9765625,0.7421875,3.21875,-5.0625,-3.296875,0.1611328125,0.8515625,0.009765625,-1.8984375,1.4765625,-2.03125,4.4375,-4.75,3.390625,-4.65625,-3.90625,0.28125,0.07568359375,7.90625,4.25,-3.796875,-3.421875,-0.6015625,-7.0625,-3.421875,-3.859375,6.65625,-0.52734375,0.96875,2.078125,2.390625,-0.01031494140625,1.46875,-2.96875,3.203125,5.28125,0.294921875,3.046875,2.1875,-1.125,-4.40625,0.3125,-3.171875,7.0625,3.0625,0.404296875,3,-1.8984375,1.484375,-1.03125,-1.0625,-2.828125,2.171875,1.71875,-2.5,-3.28125,1.046875,-3.859375,0.72265625,-5.40625,-2.578125,-5.3125,2.765625,2.3125,-0.81640625,-0.7578125,4.4375,0.318359375,3.328125,-5.53125,-3.890625,3.8125,0.9765625,0.333984375,2.84375,-0.6796875,-5.03125,-0.9375,0.201171875,1.9140625,-4.1875,-3.609375,3.328125,2.46875,0.283203125,-3.9375,-4.40625,-3.453125,2.390625,4.1875,-0.96484375,0.353515625,0.06005859375,-1.53125,2.171875,-2.65625,4.5,-3.109375,-4.15625,-0.47265625,0.734375,3.578125,-3.203125,-1.0703125,1.4296875,-3.4375,0.7578125,1.2734375,-0.11279296875,-1.9453125,3.171875,-2,-3.65625,-5.4375,5.78125,-2.0625,0.45703125,-3.875,-2.65625,-3.1875,-1.421875,-0.6640625,1.7421875,0.0703125,5.78125,-0.63671875,2.8125,0.478515625,-0.8828125,0.0712890625,3.453125,-0.271484375,-2.90625,1.8359375,-4.59375,-4.65625,0.7578125,-8.0625,-2.0625,2.90625,-2.40625,2.671875,-2.671875,2.375,-1.1015625,-2.21875,-1.8203125,-0.8203125,0.83984375,5.375,2.171875,0.2216796875,0.38671875,1.8984375,0.859375,-1.109375,-1.8515625,-0.25,5.34375,0.62109375,2.765625,-3.359375,-2.34375,4.46875,-0.59375,-3.75,0.8984375,-0.357421875,0.6640625,4.5625,0.9609375,-3.796875,-2.9375,-6.15625,4.03125,0.73828125,1.828125,-4.625,1.5,-3.0625,0.1748046875,2.03125,-6.5625,-2.546875,3.328125,2.828125,5.46875,1.328125,-2.421875,-4.53125,2.203125,-0.396484375,-1.6171875,-2.234375,-1.7265625,-0.96875,-3.765625,4.125,-2.515625,4.25,-1.3359375,-2.8125,-0.8671875,0.61328125,-0.203125,0.47265625,-0.353515625,-0.88671875,4.0625,-0.3515625,7,2.171875,-4.0625,4.59375,2.515625,0.412109375,-1.5625,3.75,-1.109375,-2.3125,3.921875,2.890625,-4.0625,4.96875,2.125,3.375,-3.46875,-2.1875,-0.9921875,4.5625,0.287109375,1.28125,-4.34375,0.1630859375,4.0625,-0.1884765625,0.8671875,-1.765625,0.3046875,0.65234375,0.52734375,2,1.921875,3.4375,-0.52734375,1,-0.92578125,-1.2265625,2.328125,-0.1328125,-0.703125,-1.8828125,3.21875,-1.6953125,-1.875,-6,1.2421875,-3.46875,2.21875,3.1875,2.875,2.234375,-2.828125,-1.625,-2.640625,-5.25,-3.140625,1.75,1.09375,-1.75,1.875,-0.1181640625,2.546875,5.84375,0.130859375,4.6875,-3.109375,2.5,1.140625,0.875,0.046630859375,4.3125,-1.8203125,-2.21875,3.640625,-4.46875,3.71875,-4.53125,-3.078125,-0.63671875,-0.10986328125,2.640625,6.625,-4.5625,-3.953125,5.21875,1.328125,4.59375,3.78125,-2.078125,-1.484375,0.79296875,1.3515625,5.46875,0.93359375,2.953125,-2.734375,6.9375,5.65625,0.90625,2.359375,0.166015625,-2.6875,-6.4375,5.125,1.3984375,1.984375,-2.375,1.6875,3.109375,0.1533203125,3.640625,-5.5,0.8671875,1.2109375,0.90625,0.5234375,-3.15625,0.103515625,2.640625,0.33203125,-1.6875,5.84375,0.97265625,4.125,-0.72265625,3.34375,2.328125,3.703125,-2.03125,1.5234375,-3.46875,3.578125,-1.3984375,2.15625,-5.5,1.0546875,3.640625,4.3125,-1.625,-3.5625,2.21875,0.275390625,-0.5,-4.46875,4.21875,3.59375,2.5625,-6.9375,-3.328125,-0.05029296875,0.2060546875,1.234375,-3.484375,1.171875,1.6796875,-4.625,-3.265625,1.296875,1.625,-5.65625,-6.0625,-3.203125,1.65625,1.3203125,3.1875,3.21875,-0.8203125,3.40625,-0.55078125,3.046875,4.28125,-1.1328125,1.5546875,0.9375,-2.75,4.125,-0.263671875,-2.671875,1.5546875,-0.50390625,-2.140625,0.50390625,-2.296875,-1.0703125,-4.21875,-0.85546875,2.328125,-1.09375,5.125,-3.96875,0.30078125,3.609375,-1.4375,-2.28125,-2.65625,0.5703125,-2.921875,-2.578125,-1.9140625,3.609375,2.984375,2.046875,0.58203125,-0.6015625,-3.265625,-6.40625,-5.65625,3.578125,-2.515625,2.859375,0.439453125,-4.25,2.078125,2.8125,1.78125,-0.1640625,-0.55859375,2.765625,4.59375,0.455078125,-1.7265625,-0.466796875,3.609375,-4.5625,-3.78125,0.515625,1,-3.171875,2.28125,-3.125,-1.8359375,0.79296875,4.5,-0.5078125,-2.859375,-1.75,-2.40625,-2.875,-3.03125,-2.859375,2.5625,1.859375,3.296875,0.1689453125,-0.421875,-5,3.71875,16.875,0.9375,-4.71875,2.421875,-3.140625,2.65625,3.171875,4.8125,-1.7109375,-1.96875,-2.1875,1.765625,0.01031494140625,1.4140625,-2.140625,1.7421875,1.9921875,-0.48828125,-4.125,-1.9765625,-1.328125,0.84765625,-0.7578125,2.96875,0.408203125,2.265625,-0.734375,-0.259765625,0.2333984375,-3.234375,-4.46875,-4.4375,2.265625,-1.7578125,4.75,-4.25,5.375,0.1845703125,-2.9375,-2.09375,-3.296875,-3.171875,1.0234375,-0.75,-1.9453125,4.34375,-0.72265625,1.09375,0.37890625,-0.337890625,-3.546875,-3.046875,-2.6875,7.25,0.62890625,-5.71875,-1.546875,-4.84375,-4.5625,0.58984375,2.796875,-2.328125,1.6328125,1.453125,-1.828125,-2.171875,-1.953125,0.85546875,3,-5.125,-5.625,0.13671875,1.5546875,3.359375,2.796875,-4.0625,1.5703125,5.3125,2.6875,0.69140625,-0.75,1.4453125,-1.3828125,-2.5,-0.91015625,1.4609375,-4.03125,1.109375,1.4453125,-4.875,11.25,-8.625,4.8125,4.0625,-4.75,-0.1865234375,2.796875,1.796875,-1.6796875,-0.169921875,2.953125,2.453125,3.359375,-0.306640625,6.09375,1.5234375,0.388671875,0.73828125,2.9375,3.578125,2.4375,2.9375,-0.828125,-1.9609375,1.3046875,1.7734375,-2.484375,-3.46875,-1.4609375,-4.4375,6,1.6171875,-2.765625,-1.2578125,-10.5,-3.421875,-2.328125,-5.84375,4.5,-2.65625,2.46875,3.421875,-0.609375,-1.078125,-2.53125,-5,2.296875,4.0625,0.208984375,-0.3984375,-6.0625,2.84375,3.546875,-3.984375,-2.09375,1.4453125,-3.265625,3.296875,-0.1923828125,4.9375,-3.578125,3.9375,2.03125,-2.546875,-5.8125,3.171875,-3.765625,-2.234375,-5.3125,-2.453125,-2.078125,-3.328125,-0.6171875,-0.35546875,-2.078125,-1.03125,1.6171875,-0.60546875,-3.15625,2.921875,2.96875,-4.375,-2.625,0.58203125,0.73046875,-4.28125,1.1875,5.1875,-0.54296875,1.5,0.55078125,0.078125,-0.3203125,-4.34375,0.81640625,1.71875,-4.03125,-0.71875,-1.359375,-2.828125,-2.4375,-2.78125,-3.375,3.875,3.59375,-5.0625,1.9609375,-0.34765625,0.014892578125,-1.4453125,-1.546875,6.4375,2.234375,-1.6484375,5.59375,1.03125,-4.15625,-2,-2.046875,-1.1484375,-1.2734375,6.3125,1.2578125,2.375,-5.90625,7.53125,2.453125,1.7265625,-0.43359375,2.34375,1.6796875,-3.71875,-5.40625,2.46875,2.75,3.84375,-4.59375,0.6328125,0.53515625,0.53125,-4.28125,1.90625,-0.259765625,0.482421875,-3.140625,-7.59375,-0.109375,0.90625,-1.8828125,1.5234375,4.25,-2.96875,1.3828125,0.95703125,-0.58984375,3.640625,3.28125,-2.828125,1.90625,-0.1904296875,2.625,-2.34375,1.4921875,-3.71875,-4.96875,-3.109375,-1.765625,1.8828125,-2.625,0.67578125,-0.357421875,-4.1875,2.109375,-2.25,1.125,1.09375,0.2578125,-6.25,3.984375,5.1875,-4.15625,4.4375,-5.53125,-2.4375,-1.640625,2.21875,-1.9140625,-6.46875,2.0625,4.5,-3.390625,2.203125,3.546875,-1.625,-0.4453125,-2.25,5.3125,-1.015625,4.78125,-0.6953125,3.953125,3.9375,-1.28125,-0.061279296875,-5.125,0.470703125,-2.28125,-3.84375,5.53125,-1.921875,2.46875,5.21875,4.9375,-9,-1.96875,0.54296875,-0.1845703125,3.578125,3.109375,-1.3671875,1.0234375,0.028076171875,-0.30859375,4.4375,-0.9296875,-1.46875,-3.65625,4.96875,-0.1728515625,-4.0625,2.984375,2.609375,-4.15625,4.34375,-2.75,-2.6875,-0.6875,-0.1396484375,-5.625,1.8046875,2.6875,-0.92578125,3.4375,3.109375,1.203125,3.59375,-2.640625,-10.0625,0.0703125,2.75,5.3125,1.7265625,2.3125,0.0859375,-1.0625,3.640625,-4.5625,0.46875,-1.484375,-9.5,0.255859375,-4.15625,-1.609375,-3.453125,-1.4921875,-1.9453125,3.90625,1.3984375,-0.8515625,3.5,2.921875,0.453125,4.15625,-0.361328125,-3.578125,1.2734375,1.75,-5.28125,-1.90625,4.8125,3.578125,-2.203125,-2.0625,3.84375,-4.28125,-0.70703125,4.3125,4.28125,2.15625,-0.828125,-3.234375,2.84375,-2.546875,-2.828125,1.703125,-3.421875,2.453125,-1.4375,2.578125,1.296875,-2.640625,-2.03125,-4.15625,-2.71875,3.484375,0.28515625,0.9765625,-2.265625,-1.1171875,3.234375,3.5625,-2.359375,-2.109375,2.796875,-1.3515625,-4.28125,-1.0859375,1.0859375,-5.90625,-2.609375,2.734375,3.4375,-2.5625,-3.5625,-2.125,1.6171875,1.3046875,-0.8984375,-0.1318359375,-3.53125,2.65625,5.0625,-2.9375,-3.75,-1.6171875,-0.486328125,-5.03125,-3.609375,-0.1767578125,1.140625,-0.73046875,3.890625,-1.40625,0.47265625,4.4375,-3.65625,-3.21875,3.96875,3.359375,-3.203125,-1.46875,2.25,-3.375,1.03125,5.4375,-2.390625,-2.234375,0.41796875,-2.171875,-4.28125,2.34375,1.2265625,-3.734375,-7.875,5.96875,1.0703125,4.34375,4.125,-3.90625,4.0625,-4.6875,1.8828125,-1.265625,1.015625,1.3828125,-5.65625,-1.1875,-2.5,-3.5,0.5390625,-1.734375,-3.5625,0.66015625,8.0625,-1.328125,-2.59375,-2.953125,-3.515625,3.3125,-4.15625,-7.625,0.1181640625,-7.34375,1.734375,-2.1875,1.75,-5.59375,1.9140625,-1.078125,1.734375,-2.984375,0.27734375,-0.384765625,1.21875,0.54296875,4.6875,1.2109375,1.984375,-0.1484375,2.71875,0.0791015625,1.875,-1.453125,-0.4921875,1.21875,-1.234375,0.33203125,0.69921875,-2.734375,0.1708984375,-1.7578125,-0.263671875,-1.015625,1.7578125,2.9375,-0.640625,-0.291015625,-1.6875,1.703125,-4.5,1.3125,-1.796875,0.859375,-0.78515625,-1.0078125,1.9609375,-2.328125,1.6640625,1.015625,1.640625,0.01068115234375,-1.5,2.234375,2.6875,-0.031982421875,-2.328125,-1.8046875,-0.55859375,-1.7421875,1.7421875,0.55078125,-2.0625,2.9375,-1.640625,-0.41015625,0.890625,1.7265625,0.44140625,-1.6484375,2.40625,-1.8671875,1.2890625,1.0859375,-1.5234375,2.609375,0.63671875,1.03125,1.2734375,0.9765625,-2,0.64453125,0.2578125,-1.4375,-0.291015625,3.484375,-1.7265625,0.31640625,-1.078125,-0.5625,1.0859375,-0.8671875,1.2109375,0.15625,-0.396484375,-2.75,2.640625,-2.125,-1.2578125,-0.42578125,0.29296875,-0.5703125,0.8984375,0.08935546875,1.2109375,-0.29296875,2.28125,-0.73828125,2.171875,-0.020263671875,-0.2060546875,1.3359375,3.421875,-1.984375,0.7421875,-2.0625,-1.1328125,1.3203125,-0.3046875,1.15625,-0.93359375,-2,1.2421875,1.1328125,-2.984375,-0.734375,2.265625,-0.189453125,-1.1328125,-0.609375,1.2265625,-0.75390625,-0.38671875,0.419921875,-0.89453125,2,3.265625,-1.0625,2.5,-1.453125,0.396484375,0.73046875,1.046875,2.3125,0.07958984375,-2.34375,-0.9296875,2.71875,-1.4375,0.37109375,0.890625,-1.53125,-0.1396484375,1.3359375,0.5703125,1.640625,-0.06982421875,-1.859375,-0.330078125,-0.6796875,1.609375,1.65625,-1.6875,0.68359375,-1.8359375,-0.53125,-1.015625,2.765625,-1.7578125,-2.140625,-0.78515625,-1.1015625,-0.83203125,-0.498046875,0.11962890625,-0.1298828125,0.60546875,1.125,1.5,0.4296875,-0.609375,1.4375,-0.08056640625,0.68359375,-1.1875,-1.5234375,1.484375,1.2421875,2.34375,-1.359375,1.34375,0.9296875,0.8828125,-1.1796875,1.9453125,-0.5234375,0.314453125,0.010986328125,-0.1181640625,1.40625,2.21875,0.318359375,0.5859375,-0.1328125,1.40625,0.69921875,1.375,-1.3046875,-2.203125,-1.0078125,-1.4296875,-2.125,0.361328125,-0.0615234375,-1.3046875,-0.1904296875,0.034912109375,-0.86328125,1.375,1.1796875,1.5390625,-0.828125,-0.58203125,0.1787109375,-0.328125,0.25390625,0.8828125,-0.8046875,-0.78125,-1.1171875,-2.0625,1.578125,0.88671875,-1.09375,-0.2890625,2.0625,-1.5,1.0078125,-2.78125,0.55078125,-1.828125,-0.341796875,0.0859375,-3.265625,0.34765625,-0.12451171875,-2.15625,-3.078125,-1.75,-0.85546875,-2.375,-0.3203125,4,-0.81640625,-1.21875,2.03125,0.08203125,-1.0078125,-0.94921875,1.7578125,2.84375,-0.8203125,3.859375,0.349609375,-0.16015625,-1.3984375,-1.265625,0.52734375,-1.2890625,0.294921875,-0.84765625,-0.8046875,-1.6796875,-3.109375,0.05859375,-4.1875,-2.125,0.1337890625,0.90625,1.890625,-0.08447265625,-0.7421875,-0.56640625,-0.96875,2.796875,-0.267578125,0.18359375,1.4375,0.27734375,0.46875,-1.4140625,0.92578125,-0.84375,2.953125,-1.171875,-0.50390625,-2.65625,-1.5546875,-4.1875,1.453125,2.484375,0.421875,2.96875,1.3671875,-0.5546875,-2.5625,0.07421875,0.00909423828125,-4.75,-0.373046875,-0.7265625,0.07275390625,-1.4140625,-0.7109375,-0.1318359375,-0.609375,-1.328125,-0.51953125,-1.828125,-0.271484375,-2.28125,2.984375,1.7890625,1.875,2.3125,0.3125,-0.31640625,1.1875,2.359375,1.1484375,0.6953125,0.255859375,0.408203125,-1.09375,2.09375,0.337890625,0.4609375,-1.2265625,0.2275390625,1.1875,2.5625,1.734375,-0.76171875,0.85546875,0.328125,-1.9140625,-1.40625,0.31640625,0.296875,1.140625,0.333984375,1.03125,-1.2890625,0.416015625,-0.6875,0.9453125,1.7578125,-1.953125,1.109375,-0.134765625,0.1787109375,-1.5,1.203125,1.15625,1.8203125,-0.48046875,2.140625,1.1640625,0.48828125,1.8515625,2.609375,-0.361328125,1.421875,-0.86328125,1.953125,0.51953125,-2.484375,3.15625,-0.34375,-0.47265625,-0.56640625,1.2890625,1.359375,-0.60546875,-0.25,-0.38671875,2.015625,0.52734375,0.14453125,1.8828125,0.67578125,-0.546875,-0.77734375,-0.6015625,-1.09375,-2.328125,-1.0078125,-3.0625,-0.37109375,-0.9375,1.765625,-0.828125,-1.484375,-0.142578125,1.390625,-0.02099609375,1.3203125,1.6171875,-1.0859375,2.09375,0.154296875,0.1962890625,0.89453125,-0.97265625,-1.2421875,1.15625,0.82421875,-0.59765625,4.625,0.1962890625,2.28125,-0.65625,-1.0390625,-0.78515625,3.59375,-0.44921875,-0.4375,-1.6953125,1.140625,-0.296875,-1.25,-0.76953125,-1.3984375,-0.9765625,1.78125,-0.87109375,-3.234375,-2.171875,0.330078125,-1.875,0.48828125,-1.859375,-1.0390625,2.40625,1.734375,-0.63671875,0.216796875,1.125,-1.0234375,0.58984375,-0.4296875,0.3515625,1.6015625,-1.2109375,1.765625,0.5859375,2.796875,-3.921875,-0.298828125,2.171875,1.578125,-0.458984375,-1.015625,-0.51171875,2.109375,0.369140625,-0.018798828125,-0.50390625,-4.46875,0.0135498046875,-0.043212890625,-3.21875,-0.09423828125,0.4921875,1.2421875,0.6640625,-3.15625,0.73046875,-1.5078125,-1.6328125,3.46875,-0.55078125,-0.41796875,0.58203125,1.1640625,-0.83203125,-0.84765625,1.53125,0.17578125,-3.484375,-1.1015625,-0.1591796875,-0.875,0.59765625,0.01373291015625,0.099609375,0.546875,-0.36328125,-1.171875,-1.1328125,-0.33984375,-0.08056640625,1.015625,4,1.1484375,1.265625,1.2109375,-2.125,4.5625,-2.515625,-0.96484375,1.1015625,1.3515625,-1.1796875,3.921875,1.109375,0.2265625,-2,0.55859375,2.96875,0.765625,0.9453125,0.671875,1.28125,1.7421875,1.78125,-1,-1.8671875,1.5,-0.35546875,-2.5,0.012451171875,0.2578125],"index":1,"object":"embedding"}],"model":"doubao-embedding-text-240715","object":"list","usage":{"prompt_tokens":7,"total_tokens":7}}, + "headers": { + "Content-Type": "application/json" + } + } + }, + { + "httpRequest": { + "method": "POST", + "path": "/v1/openai/embedding" + }, + "httpResponse": { + "body": {"object":"list","data":[{"object":"embedding","index":0,"embedding":[-0.006929283495992422,-0.005336422007530928,-4.547132266452536e-05,-0.024047505110502243]}],"model":"text-embedding-3-small","usage":{"prompt_tokens":5,"total_tokens":5}}, + "headers": { + "Content-Type": "application/json" + } + } + }, + { + "httpRequest": { + "method": "POST", + "path": "/v1/custom/embedding", + "headers": { + "Authorization": [ + "Bearer xxxxxxxx" + ] + } + }, + "httpResponse": { + "body": {"created":1725001256,"id":"02172500125677376580aba8475a41c550bbf05104842f0405ef5","data":[{"embedding":[-1.625,0.07958984375,-1.5703125,-3.03125,-1.4609375,3.46875,-0.73046875,-2.578125,-0.66796875,1.71875,0.361328125,2,5.125,2.25,4.6875,1.4921875,-0.77734375,-0.466796875,0.0439453125,-2.46875,3.59375,4.96875,2.34375,-5.34375,0.11083984375,-5.875,3.0625,4.09375,3.4375,0.2265625,9,-1.9296875,2.25,0.765625,3.671875,-2.484375,-1.171875,-1.6171875,4.1875,2.390625,-6.90625,0.369140625,0.259765625,3.671875,-2.9375,-1.9140625,-0.71875,-1.6640625,0.29296875,0.396484375,-4.625,-1.9921875,5.15625,-1.3984375,3.015625,-3.203125,-1.453125,4,-8.75,-5.625,1.0546875,-3.28125,-1.2265625,0.287109375,2.09375,4.6875,0.1572265625,0.42578125,0.79296875,3.234375,-0.169921875,0.9296875,7.40625,-3.296875,5.53125,3.890625,0.62109375,1.1171875,-0.373046875,4.125,-2.78125,0.333984375,3.9375,4.59375,6,1.53125,-0.373046875,1.109375,-4.0625,1.96875,1.421875,0.57421875,-0.56640625,-2.390625,0.734375,1.1875,-2.859375,-6.09375,-5.96875,1.8359375,-3,0.80859375,-0.130859375,-5.3125,-2.859375,1.484375,-4.53125,-6.90625,-2.25,0.7734375,-1.2734375,1.1484375,3.421875,-3.484375,2.65625,1.3359375,1.1484375,-4.09375,-5.625,2.625,-0.283203125,-3.46875,2.3125,-0.220703125,4.21875,3.75,-0.37109375,0.9609375,7.25,-0.87890625,7.03125,2.34375,4.5,-1.609375,-6.46875,-6.125,-2.59375,2.234375,3.78125,1.3046875,-5.5,1.953125,-3.421875,-5.9375,3.25,-3.4375,-8.3125,-2.546875,3.640625,0.267578125,-0.220703125,0.294921875,-0.4140625,2.515625,-1.0546875,-5.21875,6.6875,3.640625,0.2314453125,-2.5,1,1.6640625,0.59765625,2.75,1.1328125,1.1328125,-4.96875,4.53125,-0.349609375,3.390625,-0.193359375,7.625,2.921875,-3.484375,4.1875,8.5,-1.9140625,6.3125,2.5625,3.0625,0.40234375,0.76953125,-4.78125,3.53125,-2.765625,0.1591796875,-0.1025390625,-3.875,2.203125,0.03076171875,1.765625,1.859375,2.15625,-1.2578125,-4.40625,-0.62890625,4.4375,-1.78125,2.671875,2.765625,-1.7890625,-8.3125,-0.02197265625,1.640625,-3.96875,-3.15625,2.796875,1.1875,2,1.15625,2.359375,1.3984375,4.21875,-2.953125,8.5,3.46875,3.578125,0.90625,-1.8828125,2.15625,3.921875,4.125,-0.9609375,-2.171875,2.328125,2.921875,1.9765625,1.0703125,4.03125,6.28125,-3.59375,-0.94921875,5.6875,-1.9140625,-5.1875,-4.25,-7.71875,1.7109375,-1.59375,3.765625,-5.3125,-3.9375,-3.796875,2.90625,2.859375,-2.203125,-1.78125,-3.796875,0.1708984375,-5.15625,0.298828125,1.828125,-7.1875,1.6953125,6.125,2.671875,0.1728515625,3.375,0.609375,-4.78125,0.40625,-3.875,-6.4375,0.6953125,1.171875,-2.140625,5.8125,-1.640625,5.90625,-0.1650390625,4.9375,-2.421875,1.609375,-3.171875,-4.71875,7.6875,-1.09375,-1.9296875,0.033447265625,-3.46875,-2.671875,-8.875,2.4375,-1.1015625,4.40625,-3.53125,1.546875,2.359375,-3.15625,10.625,7.46875,-3.0625,-0.044677734375,0.90234375,-5.28125,-3,-1.2890625,0.59375,-6.34375,-1.8203125,5.40625,5.78125,-1.578125,2.46875,-2.171875,-1.71875,-0.38671875,-2.21875,-0.150390625,4.65625,-3.46875,1.5625,4.4375,-2.609375,1.6875,-2.828125,-6.03125,5.15625,-2.296875,-1.65625,-2.3125,-4.75,-3.3125,-3.703125,-1.9296875,-6.59375,3.640625,-0.62890625,4.8125,0.11279296875,2.515625,0.9921875,-3.03125,-5.40625,7.5625,-1.765625,4.4375,4.25,-0.140625,3.671875,-2.984375,-2.734375,2.203125,-6.96875,-1.1640625,2.390625,1.3515625,-1.75,2.96875,-3.75,-0.109375,2.5,0.796875,5.21875,7.8125,-4,1.171875,0.435546875,1.2734375,-3.015625,1.90625,-1.21875,5.9375,-0.31640625,-4.03125,-3.59375,1.09375,4.65625,-0.81640625,-2.046875,0.80859375,-5.375,2,-2.265625,5.34375,-0.46875,-1.3359375,-2.953125,-4.875,-0.53515625,-3,1.8203125,-2.59375,-1.4765625,6.28125,2.09375,0.1318359375,2.40625,-0.09130859375,-2.421875,-1.78125,1.59375,0.48828125,-0.310546875,-0.2353515625,0.1748046875,0.4453125,2.078125,-1.046875,1.46875,0.6953125,-0.52734375,-0.19140625,-2.28125,-0.515625,0.47265625,-1.2421875,-8.3125,1.1875,2.015625,-4.46875,3.734375,1.453125,-2.8125,-2.78125,5.875,-0.38671875,1.171875,-6.5,1.8046875,-2.15625,4,3.375,-0.78125,0.87890625,-1.796875,-1.265625,2.59375,3.96875,1.7421875,2.296875,2.78125,-5.8125,-2.046875,-0.1201171875,-4.1875,3.96875,-3.484375,-4.125,1.21875,3.484375,0.3828125,3.8125,1.90625,-8.3125,-2.15625,2.578125,2.578125,-1.34375,-3.359375,4.71875,-1.640625,-3.484375,2.046875,3.0625,-1.03125,-2.96875,6.96875,3.703125,-0.29296875,-0.423828125,2.640625,-1.265625,3.9375,-0.314453125,-4.15625,-2.171875,0.2734375,6.375,-6.21875,-6.3125,4.6875,-0.053466796875,0.045166015625,2.765625,2.953125,1.078125,-0.453125,1.96875,-6.71875,-3.375,-4.1875,2.515625,-0.5390625,-1.9296875,-4.03125,1.1953125,8.1875,1.0078125,0.80859375,-1.15625,-1.53125,2.875,-3.921875,1.953125,4.09375,6.59375,-4.5625,-1.2109375,3.5,-8.1875,0.294921875,-3.453125,-0.9921875,-2.015625,1.5,0.6328125,-4.90625,-2.765625,1.0546875,4.25,-2.390625,-5.96875,7.15625,-5.4375,-3.953125,1.15625,-0.017822265625,2.90625,2.78125,-2.21875,3.390625,1.9453125,2.171875,1.8671875,-1.125,-3.65625,-1.359375,0.96484375,2.5625,-2.9375,1.2734375,4.15625,-6,-0.2021484375,-1.8515625,-0.56640625,-1.671875,1.546875,5.8125,-0.640625,1.140625,-5.71875,-0.40625,0.5390625,-1.6640625,0.3203125,-2.375,4.9375,-2.453125,-1.59375,0.1669921875,1.6796875,-0.81640625,1.765625,-3.125,-1.234375,0.84375,-0.96484375,0.232421875,-0.01300048828125,-6.03125,4.25,5.625,0.65625,-1.6015625,1.390625,5.65625,3.0625,0.287109375,-0.08203125,4.15625,-1.5703125,-0.609375,-6.34375,2.203125,-3.84375,-2.53125,-3.390625,3.15625,4.59375,-4.46875,5.0625,-3.09375,3.328125,-0.65625,1.8515625,-9.375,1.609375,-1.515625,-2.5625,-2.953125,0.734375,2.375,1.3515625,0.390625,1.8671875,0.07080078125,1.328125,3.6875,0.2421875,0.73828125,3.1875,1.65625,2.75,2.859375,-2.8125,-7.75,1.53125,-1.1015625,-1.6875,6.3125,3.03125,-2.46875,0.77734375,-0.34765625,-1.78125,-1.4453125,3.40625,3.140625,-3.96875,3.984375,-3.21875,5.375,-2.890625,2.90625,-2.375,-6.1875,-2.4375,0.34375,-4.46875,-2.421875,3.40625,-1.2578125,4.59375,4.125,-6,0.003936767578125,1.1484375,2.359375,4.09375,0.5703125,-1.328125,-6.03125,4.5,3.234375,-2.140625,5.03125,-2.640625,0.041748046875,-0.90234375,4.375,-2.125,-0.1630859375,2.421875,-2.078125,1.1328125,-3.53125,1.0234375,-0.2734375,-9.125,-6.03125,0.73828125,-0.87109375,6.59375,-0.65625,-2.109375,-3.359375,2.40625,-0.0157470703125,5.96875,2.390625,3.078125,5.65625,5.09375,-1.5859375,1.78125,-0.921875,-8.0625,7.0625,-5.71875,-2.375,2.359375,2.65625,-1.453125,-1.2265625,1.984375,-2.125,-5.46875,-5.25,-1.78125,-4.28125,3.375,-2.09375,1.984375,-0.75,-5.0625,1.46875,-1.8671875,-2.875,-1.859375,2.609375,-5.5,2.484375,5.65625,1.875,-0.94921875,3.890625,4.125,0.8984375,-2.796875,0.95703125,-7.9375,1.7890625,3.453125,-1.9296875,-0.69140625,-5.84375,2.171875,-3.4375,2.921875,0.890625,-2.203125,-2.375,-1.6328125,-2.65625,0.8515625,-7.28125,2.484375,1.6484375,-0.8359375,-0.859375,0.232421875,1.921875,0.73046875,-0.30078125,1.515625,4.9375,0.7109375,-0.43359375,-3.140625,-2.796875,-0.2431640625,2.265625,-2.53125,6.875,-0.54296875,-1.5625,3.96875,0.44921875,-3.640625,-4.25,4.375,-1.875,0.45703125,-1.2265625,5.65625,0.298828125,3.921875,-1.703125,-2.8125,-3.328125,1.7578125,3.3125,-1.6875,-3.234375,2.09375,2.375,5.40625,-3.234375,-7.09375,1.984375,4.125,-0.8046875,-2.71875,8.6875,-1.296875,-2.625,-3,-3.78125,1.359375,1.515625,2.875,0.11279296875,-1.5859375,1.078125,3.46875,-1.390625,0.6328125,0.24609375,-3.765625,3.515625,0.380859375,2.609375,-0.80078125,-2.484375,-2.15625,-1.3203125,0.02490234375,4.03125,8.25,-1.5234375,-1.1953125,1.2109375,0.3125,-1.7421875,5.625,-0.76953125,5.90625,1.15625,0.1640625,1.390625,0.82421875,-0.322265625,3.21875,-4.65625,-4.5,-1.765625,3.171875,-4.3125,-1.4375,-2.546875,-0.9140625,4.28125,0.609375,-3.171875,3.671875,0.48046875,-0.9140625,-4,-2.4375,-5.34375,-1.96875,0.828125,1.953125,-2.140625,-2.59375,-0.353515625,4.78125,-4.09375,-3.921875,0.03173828125,1.8359375,1.3984375,-0.65234375,-1.15625,0.1611328125,0.50390625,2.90625,-1.875,-3.40625,0.498046875,8.75,3.90625,-4.53125,0.67578125,-0.765625,1.8359375,-5.3125,-2.15625,-0.6796875,-1.8984375,-3.046875,-1.7734375,-1.390625,-2.71875,-2.015625,5.84375,-3.28125,0.55859375,0.8046875,3.984375,0.99609375,3.015625,0.458984375,5.3125,3.1875,-1.2421875,-5.84375,-1.3828125,-0.04052734375,-5.75,-1.8828125,3.234375,6,3.171875,1.5703125,-2.828125,0.033203125,-0.953125,0.640625,5.3125,-5.75,-3.78125,-1.984375,-7.9375,-6.84375,-3.859375,-2.65625,-3.15625,-6.84375,-0.9765625,-1.375,-7.1875,-1.1328125,-2.109375,-1.546875,-1,0.640625,4.625,-4.65625,2.3125,3.703125,2.6875,3.0625,-2.28125,3.34375,0.474609375,-1.46875,0.34765625,-2.03125,5.25,-1.4609375,5.875,3.984375,-0.87890625,-3.8125,4.46875,4.40625,5.90625,-4.875,-3.53125,-2.53125,-1.8125,-0.39453125,-1.2421875,2.203125,-3.828125,-3.59375,-1.0859375,-3.453125,0.1845703125,5.625,0.421875,5.3125,-1.3671875,0.30859375,1.5234375,2.953125,0.1064453125,2.59375,1.5546875,-4.46875,3.609375,-0.81640625,1.390625,0.8359375,-2.78125,2.125,-1.6875,0.365234375,2.234375,3.875,10.4375,1.15625,2.328125,-0.09326171875,-0.76171875,-2.609375,-2.96875,2.40625,1.6796875,1.4921875,-3.65625,0.74609375,-0.8828125,2.03125,-0.306640625,-16.875,-3.328125,-5.53125,-2.109375,4.625,-1.0546875,-1.984375,1.0625,3.6875,2.671875,7.09375,-1.484375,4.03125,-1.078125,-0.7265625,2.515625,-4.3125,1.578125,3.6875,1.890625,4.625,1.7734375,-1.8125,-2.828125,6.9375,5.0625,-4.5,0.193359375,5.09375,-1.3515625,-1.140625,4.40625,-2.96875,2.078125,-4.75,3.078125,7.09375,2.75,-2.953125,-4.125,-2.375,-2.0625,1.0234375,3.046875,-2.578125,1.578125,2.921875,-5.65625,2.28125,2.28125,-0.259765625,-3.484375,-0.37109375,2.71875,1.625,-0.158203125,-4.5,2.5625,0.98828125,3.84375,4.8125,-2.796875,-2.140625,2.34375,2.90625,2.1875,1.5546875,2.578125,2.8125,-1.8515625,-2.984375,0.310546875,-1.328125,-0.0234375,-1.9765625,0.83984375,3.65625,2.046875,-4.5625,2.171875,2.234375,-2.109375,-0.0439453125,-4.0625,-3.5,2.09375,-2.21875,-2.5,0.703125,-2.953125,-1.28125,3.234375,-4.6875,4.1875,-2.484375,8.75,-0.53125,-1.8203125,1.171875,-3.0625,4.78125,-2.484375,-3.453125,3.765625,-2.6875,1.5625,-3.828125,1.9296875,-1.765625,1.2421875,5.0625,-4.65625,-2.0625,0.53125,3.265625,-2.875,-2.296875,0.29296875,3.859375,0.123046875,-4.46875,4.09375,-2.796875,3.96875,-3.890625,1.875,-4.46875,-0.5078125,2.140625,0.3203125,4.84375,5.03125,-5.34375,-4.96875,-1.3203125,-5.03125,-4.875,-4.5625,5.03125,-2.625,-0.75,1.046875,2.109375,-0.130859375,1.890625,-1.8125,2.53125,6.53125,-2.09375,0.87890625,-0.41015625,-0.412109375,-4.09375,-2.421875,-4.46875,6.40625,0.43359375,1.2578125,3.734375,-1.7109375,2.953125,1.8125,-1.1171875,-1.7109375,2.15625,1.859375,-2.015625,-2.25,1.7734375,-3.578125,4.15625,-3.328125,-3.28125,-4.71875,2.953125,1.40625,-0.287109375,1.5703125,3.53125,1.578125,3.171875,-4.34375,-3.125,5.78125,3.453125,-2.046875,4.3125,-1.2265625,-1.84375,0.640625,2.625,0.12890625,-3.25,-4.6875,5.28125,2.65625,2.015625,-4.4375,-5.75,-3.625,4.0625,4.59375,-0.78125,-2.484375,-2.03125,-3.75,1.6875,-4.15625,2.734375,-1.65625,-3.453125,-0.89453125,3.71875,2.453125,-4.15625,2.09375,0.82421875,-2.03125,0.052978515625,4.4375,1.734375,-3.71875,1.375,-0.349609375,-1.75,-7,3.59375,-2.625,-0.427734375,-4.40625,-3.84375,-3.265625,-3.796875,0.74609375,2.65625,1.6171875,3.609375,-0.7890625,3.890625,2.796875,-0.8671875,-0.43359375,2.15625,-1.7578125,-3.640625,2.375,-4.65625,-3.5,1.3984375,-7.1875,-1.5,5.0625,-2.625,4.0625,-1.171875,3.796875,-1.453125,-2.9375,-4,-1.3046875,0.91796875,6.59375,0.64453125,-0.91796875,0.64453125,2.703125,2.1875,-2.296875,-1.015625,-1.9921875,5,-0.298828125,2.953125,-5.125,-5.03125,5.375,-1.1328125,-4.46875,-0.5546875,-3.09375,1.5703125,5.34375,0.765625,-4.46875,-2.421875,-6.75,2.8125,-1.6171875,3.109375,-5.59375,0.87109375,-4.875,2.53125,4.46875,-7.21875,-3.1875,2.4375,3,5.1875,1.84375,-2.625,-6.21875,2.21875,0.306640625,-1.90625,-4.09375,-2.34375,-1.3046875,-3.875,4.4375,-2.328125,2.546875,-3.875,-2.40625,0.80078125,0.34765625,1,0.828125,1.4453125,-0.859375,3.03125,1.109375,5.15625,1.1953125,-3.8125,2.734375,4.21875,0.345703125,-1.2109375,2.0625,-0.79296875,-2.8125,2.109375,2.96875,-2.90625,5.15625,3.359375,4.3125,-5.53125,-2.875,1.515625,3.515625,-2.75,1.7109375,-4.9375,0.7265625,3.71875,-0.4765625,1.34375,0.049560546875,2.796875,-1.421875,-1.7890625,1.5,2.3125,4.21875,1.6875,3.015625,3.3125,-1.1640625,3.546875,-0.375,-1.2265625,-1.59375,3.609375,-3.015625,-2.546875,-4.625,1.046875,-1.796875,4.75,2.515625,1.1484375,0.8984375,-1.4140625,-2.328125,0.037841796875,-5.78125,-1.5859375,0.11669921875,3.015625,-0.83984375,0.84375,-0.82421875,0.96484375,4.0625,0.0400390625,4.25,-2.28125,1.3515625,1,1.5625,-2.8125,3.15625,-2.609375,-0.142578125,1.578125,-2.875,3.75,-4.3125,-1.359375,-2.578125,-0.69140625,2.84375,3.75,-4.75,-5.5625,0.84765625,0.380859375,5.125,3.0625,-3.140625,-0.93359375,0.73046875,0.0303955078125,4.3125,0.85546875,2.703125,-4.28125,5.625,5.90625,0.4296875,0.76953125,-0.9140625,-1.71875,-4.5,3.828125,-0.4609375,2.21875,-1.9453125,2.5,4.15625,1.8984375,3.984375,-5.75,2.953125,0.2734375,3.890625,-0.76171875,-3.90625,0.337890625,1.96875,0.69140625,-0.70703125,3.578125,0.046142578125,0.765625,-2.734375,4.28125,4.3125,2.578125,-4.40625,1.921875,-2.90625,1.7734375,-3.90625,1.1484375,-5.625,1.65625,2.703125,5.34375,-1.9296875,-6.1875,4.5,1.5625,-0.9140625,-3.953125,4.65625,4.5625,2.484375,-5.15625,-2.375,1.625,-1.328125,-0.26171875,-5.25,3.328125,2.0625,-3.609375,-3.71875,1.6171875,1.046875,-3.1875,-3.71875,-3.34375,1.9609375,2.5625,3.609375,1.59375,-2.484375,4.125,-0.80078125,1.9140625,4.78125,-1.09375,0.140625,3.171875,-3.578125,2.640625,-0.6640625,-2.65625,-1.4375,0.47265625,-2.46875,2.6875,-2.515625,-1.0234375,-2.09375,-0.138671875,-0.5078125,1.5,4.15625,-3.09375,0.158203125,4.4375,-1.96875,-3,-1.9609375,2.09375,-1.7734375,-1.09375,-1.8984375,3.3125,1.9765625,0.8671875,0.2890625,0.66796875,-1.9765625,-3.640625,-4.90625,2.0625,-4.0625,3.59375,-0.84765625,-6.21875,1.515625,3.890625,3.640625,-0.2734375,-2.046875,0.875,3.78125,0.07470703125,-1.078125,-1.4921875,3.671875,-2.796875,-3.6875,2.75,2.78125,-5.40625,1.7890625,-4.28125,-2.265625,-0.98046875,4.46875,0.173828125,-2.25,-2.875,-3.84375,-1.7421875,-1.6171875,-3.21875,1.9140625,1.7421875,2.671875,1.09375,1.4375,-3.5,2.59375,19.125,0.0101318359375,-8.4375,1.3515625,-3.625,4.4375,4.65625,1.8125,0.423828125,-1.5,0.62890625,4.21875,0.609375,0.5390625,-2.390625,0.984375,-0.79296875,2.078125,-3.703125,-3.109375,-2.265625,-1.0234375,-0.328125,1.9765625,2.5,2.375,0.8046875,-2.265625,1.2734375,-3.390625,-4.375,-4.71875,3.765625,-2.921875,3.125,-3.171875,4.65625,-0.7890625,-3.3125,-2.984375,-3.296875,-2.796875,2.375,-0.12255859375,-3.21875,5.21875,0.1982421875,0.2138671875,-1.1796875,-0.130859375,-4.34375,-1.4453125,-2.5,6.3125,1.0625,-6.15625,-0.5703125,-3.203125,-3.546875,-1.375,2.9375,-0.53515625,1.7578125,2.71875,-1.9453125,-2.640625,-3.046875,0.49609375,1.0078125,-3,-4.84375,0.2119140625,1.2265625,1.3515625,1.609375,-4.84375,2.46875,2.140625,2.171875,1.75,0.67578125,-0.60546875,-2.46875,-2.234375,-0.9453125,1.2421875,-3.15625,0.006744384765625,3.359375,-1.765625,8.375,-8.3125,5.8125,5.15625,-2.0625,-0.470703125,1.5,-0.30859375,-2.421875,-0.2294921875,0.95703125,1.8828125,4.84375,-0.68359375,4.625,1.359375,0.373046875,0.83203125,2.640625,4.34375,0.7578125,3.109375,-0.412109375,-2,2.15625,-0.08349609375,-3.140625,-3,-3.703125,-2.5625,3.6875,1.7890625,-3.296875,0.89453125,-7.5,-5.40625,-2.25,-7.625,4.34375,-1.34375,-0.14453125,3.515625,-2.46875,-1.2109375,-2.46875,-3.921875,1.265625,3.65625,1.4375,-1.46875,-5.03125,2.59375,3.890625,-2.765625,-2.4375,0.353515625,-4.21875,4.4375,-0.376953125,3.9375,-2.09375,3.96875,3.234375,-2.203125,-6.875,5.15625,-3.6875,-4.34375,-6.625,-2.90625,-4.9375,-3.34375,0.412109375,-0.9453125,-0.5703125,-1.3046875,3.21875,-0.65234375,-1.6796875,3.171875,3.453125,-4.4375,-1.2578125,0.828125,1.1796875,-4.375,0.1787109375,4,0.53515625,1.328125,-0.546875,0.271484375,-0.5546875,-3.859375,-0.2216796875,0.86328125,-4.53125,-1.3828125,-0.60546875,-5.46875,-1.3515625,-1.2890625,-3.734375,2.9375,2.40625,-3.984375,0.875,-2.953125,-0.9765625,-1.6328125,-1.25,3.96875,1.6953125,0.0072021484375,5.875,-0.921875,-3.46875,-3.140625,-0.26953125,0.2265625,-2.09375,7.0625,-1.09375,0.30078125,-6.03125,5.34375,2.359375,1.6640625,-0.99609375,4.625,4.25,-2.484375,-4,0.89453125,3.0625,4.1875,-4.28125,3.953125,0.6328125,-0.74609375,-1.53125,2.015625,-1.1796875,1.03125,-1.6484375,-5.4375,0.3671875,1.8125,-0.326171875,1.546875,4.03125,-3.34375,0.484375,2.5,-1.4140625,3.34375,4.25,-1.7890625,1.09375,2.171875,5.34375,-1.5625,0.98828125,-5.09375,-3.625,-2.640625,-2.46875,3.109375,-2.515625,0.09033203125,0.21484375,-3.921875,3.125,-4.1875,1.2109375,1.3671875,1.1875,-5.4375,4.59375,3.890625,-2.8125,3.328125,-5.125,-1.9765625,-1.4296875,2.34375,-2.71875,-5.875,3.125,3.453125,-1.515625,3.546875,2.265625,-0.52734375,1.9375,-2.859375,2.703125,-3.359375,4.75,1.2734375,3.09375,3.65625,-0.255859375,-0.1044921875,-5.75,-0.3359375,-0.77734375,-2.234375,6.1875,-3.84375,0.19921875,4.25,6.4375,-10.5,-1.5078125,0.7265625,0.2890625,3.921875,5.0625,0.09814453125,0.68359375,3.109375,1.015625,2.671875,0.0257568359375,-0.4765625,-4,5.15625,0.2314453125,-4.6875,3.1875,3.984375,-2.609375,3.4375,-2.375,-3.734375,-0.07568359375,2.75,-5.3125,1.9296875,4.625,-1.6484375,2.875,3.734375,-1.34375,3.875,-1.9921875,-11.3125,-1.53125,3.296875,5.71875,0.80859375,1.7578125,0.48046875,-2.015625,1.4765625,-0.5546875,0.71484375,-0.7578125,-11.1875,0.9765625,-3,-0.09765625,-1.9453125,-3.8125,-2.5,4.375,1.65625,1.1015625,3.328125,2.84375,0.84375,4.5625,0.11279296875,-5.84375,1.1484375,1.7578125,-4.8125,-0.59765625,3.234375,1.125,-1.859375,-2.515625,3.78125,-1.7421875,-0.69921875,5.8125,3.765625,1.578125,-1.84375,-5.03125,0.984375,-3.375,-1.9140625,1.1953125,-0.384765625,2.8125,-2.203125,2.828125,1.1171875,-3.75,-4.15625,-2.25,-3.5625,1.5,2.671875,2.171875,-2.609375,-1.7265625,2.8125,2.5,-0.455078125,-1.546875,2.1875,-0.1884765625,-2.984375,-1.4765625,2.0625,-4.46875,-2.90625,4.0625,1.8359375,0.443359375,-0.7734375,-3.140625,2.171875,1.734375,-1.8515625,-1.84375,-1.234375,2.15625,5.34375,-2.484375,-5.6875,-1.2734375,0.1806640625,-4.375,-3.5625,0.89453125,-1.15625,0.75,3.09375,-2.25,1.1875,4.6875,-1.3359375,-3.875,3.53125,4.4375,-2.671875,-0.75,-0.458984375,-2.53125,3.8125,5,-1.2421875,-2.109375,-0.50390625,-2.734375,-4.90625,1.0234375,2.421875,-3.34375,-10.125,6.46875,3.671875,5.40625,1.546875,-2.59375,3.8125,-1.6953125,3.703125,-0.423828125,0.82421875,1.515625,-7.59375,-2.40625,-2.0625,-5.0625,0.59375,-0.345703125,-4.75,1.4921875,6.25,-2.15625,-1.8671875,-2.703125,-3.9375,4.28125,-3.484375,-5.9375,1.984375,-7.4375,1.4609375,-1.9609375,3.265625,-5.875,1.8359375,-0.017333984375,2.046875,-0.5859375,-0.671875,-2.328125,1.1953125,-2.65625,3.625,0.7890625,3.9375,-0.365234375,2.90625,-1.2421875,0.314453125,-3.265625,1.6640625,1.7109375,0.60546875,0.384765625,2.296875,-2.28125,-0.8046875,-1.0546875,1.046875,2.796875,0.61328125,-0.625,0.10693359375,4.21875,-0.6484375,2.03125,-2.3125,-0.173828125,-1.015625,-0.224609375,0.74609375,-0.86328125,0.0145263671875,0.1318359375,1.7109375,1.421875,0.486328125,-0.19921875,0.140625,1.2734375,1.015625,1.5625,-1.65625,-0.45703125,-0.435546875,-0.0206298828125,1.828125,1.734375,-2.734375,1.65625,-2.09375,-0.6875,-0.2421875,2.125,1.1015625,0.1064453125,1.59375,-1.875,1.828125,0.15234375,-1.2421875,1.25,-0.765625,-2.265625,2.34375,-2.109375,-0.921875,0.6640625,-1.2734375,-1.4765625,-0.73828125,2.21875,-0.84375,1.328125,-1.171875,-0.181640625,0.306640625,-1.171875,0.279296875,0.94140625,1.171875,-3.921875,3.15625,1.2421875,0.52734375,-0.1630859375,1.0390625,-1.46875,-0.08447265625,1.0390625,-0.37109375,0.921875,1.859375,-1.8046875,0.54296875,-0.8203125,-1.09375,1.1640625,1.515625,0.54296875,-1.65625,-1,1.5234375,1.4453125,-1.1953125,0.359375,-0.062255859375,-2.09375,3.03125,1.21875,-3.15625,-0.357421875,-0.169921875,0.546875,-0.73828125,-0.126953125,1.046875,-2.75,-0.2314453125,0.2421875,0.306640625,-1.1328125,1.8984375,0.00469970703125,3.9375,0.8515625,1.1328125,1.1875,1.3984375,2.046875,-1.3515625,0.25390625,-0.9921875,3.234375,-0.373046875,0.8828125,1.3828125,-1.921875,-0.484375,-0.81640625,0.61328125,1.4296875,-0.70703125,-0.404296875,2.53125,1.625,0.494140625,2.375,-2.03125,0.33984375,0.291015625,-0.68359375,-1.625,1.625,-0.478515625,0.349609375,-2.0625,-1.25,-0.1484375,-0.44140625,0.67578125,0.3671875,0.4921875,0.236328125,1.1953125,0.5078125,-2.375,1.3671875,-0.341796875,0.6328125,-1.7265625,-1.328125,0.84375,-0.08935546875,1.0625,0.90625,1.984375,2.828125,1.109375,-1.3671875,1.03125,1.0625,1.75,0.263671875,-1.234375,-0.09228515625,-0.13671875,0.271484375,0.58203125,-0.9375,-1.28125,0.4609375,-0.95703125,-0.1552734375,-1.5703125,3.375,-0.9609375,-1.1796875,-0.419921875,-1.5,0.58984375,-1.3125,1,-1.578125,2.484375,1.34375,3.34375,1.4296875,-0.671875,-0.984375,0.30859375,0.72265625,-0.337890625,-0.06982421875,-1.125,-0.44921875,-0.62890625,5.40625,0.263671875,1.0390625,-2.03125,3.296875,0.68359375,-0.10986328125,-1.078125,-0.2412109375,-2.078125,-0.13671875,-1.4375,-1.390625,0.29296875,-1.1484375,-4.0625,-2.703125,-0.302734375,0.77734375,-1.640625,-0.0390625,3.890625,0.375,1.2890625,1.5,2.640625,0.19140625,-1.78125,-0.5859375,1.6328125,-1.234375,2,0.8125,-1.9453125,-2.78125,-0.3671875,-2.328125,-1.9453125,-0.59375,-0.8046875,1.9921875,-0.265625,-0.03515625,-1.3125,-1.5234375,-3.03125,-0.458984375,-0.1279296875,2.375,1.53125,0.67578125,-0.55078125,-0.4296875,0.515625,-1.75,0.6640625,-1.65625,4.25,-0.326171875,-1.4296875,2.53125,0.396484375,3.140625,0.859375,-1.3671875,-1.8828125,-0.828125,0.45703125,0.7109375,3.0625,-0.2578125,0.6328125,0.57421875,-0.85546875,0.5625,1.0234375,-0.296875,-4.84375,-1.578125,-0.486328125,2.59375,-1.2109375,0.09765625,2.59375,-0.87109375,-0.7890625,-1.7421875,-2.34375,-0.2490234375,-0.82421875,0.8046875,2.078125,-0.7265625,-0.10400390625,-0.703125,-1.046875,0.46875,-1.7734375,1.09375,-0.30859375,0.0181884765625,0.2734375,-2.703125,-0.470703125,0.67578125,-1.921875,-1.0078125,1.6328125,0.2021484375,1.359375,1.6796875,-1.6015625,1.5703125,0.6484375,-2.859375,-0.63671875,-0.8359375,1.34375,0.0556640625,0.4375,1.765625,-1.1484375,-1.90625,-1.453125,0.57421875,0.84375,-0.349609375,0.251953125,-0.0927734375,0.416015625,-0.40625,-2.71875,-0.48046875,0.4140625,-0.2109375,0.96484375,1.0859375,1.453125,1.15625,1.375,-0.478515625,1.375,-1.8828125,1.6484375,0.9921875,-2.171875,0.5859375,2.03125,-2.125,0.314453125,1.1796875,-0.4921875,-0.72265625,-0.80078125,0.5546875,-0.52734375,0.58203125,-0.52734375,1.9453125,1.71875,-0.328125,1.453125,-2.203125,-2.09375,-2.625,0.2177734375,-0.82421875,0.3359375,-2.203125,1.375,-1.7578125,-0.072265625,-0.4765625,-0.38671875,-1.9453125,1.5625,1.7578125,0.4453125,0.640625,0.0255126953125,-0.5703125,3.796875,-1.0703125,-0.1201171875,0.93359375,1.15625,-2.078125,3.484375,0.5234375,2.109375,0.0037078857421875,1.3359375,-0.796875,1.25,0.1455078125,0.86328125,0.478515625,1.828125,0.31640625,-0.296875,-0.154296875,-1.53125,-1.1640625,0.6484375,1.0703125,-5.375,0.86328125,0.890625,0.48828125,0.84765625,-2.828125,1.1015625,0.4765625,3.296875,-0.00408935546875,-0.40234375,3.421875,0.61328125,-1.46875,1.1875,0.953125,0.0771484375,-2.78125,-1.171875,-0.86328125,2.9375,-1.0703125,0.1015625,-0.279296875,-0.90625,3.046875,0.6796875,-1.6640625,1.453125,0.443359375,-0.439453125,-1.453125,-3.40625,-0.1689453125,1.71875,-0.9453125,2.234375,0.158203125,0.87109375,0.66796875,-1.640625,1,0.265625,0.267578125,-0.90625,1.75,-0.2041015625,-1.59375,1.65625,-1.1484375,-1.78125,2.421875,1.6953125,-2.328125,0.027587890625,-0.494140625,-0.3203125,-0.01953125,0.58203125,-2.28125,0.546875,0.62109375,0.90625,-0.921875,-1.53125,2.484375,1.890625,2.953125,2.359375,-0.90234375,0.171875,-2.234375,0.33984375,-0.45703125,-0.87109375,0.08251953125,1.8671875,-1.0078125,1.5703125,-0.30078125,0.921875,-1.8046875,1.609375,2.703125,0.92578125,0.40625,-0.26171875,-0.322265625,-1.8671875,-0.5,-2.296875,0.62109375,0.6953125,1.1640625,0.1376953125,-1.4296875,1.5390625],"index":0,"object":"embedding"},{"embedding":[-2.28125,-0.7734375,-0.8359375,-2.3125,3.046875,4.125,-1.0390625,-2.890625,0.0103759765625,1.9296875,0.1015625,1.75,2.4375,2.015625,5.09375,1.203125,-2.140625,-2.828125,-1.328125,-4.6875,1.0078125,6.8125,0.578125,-4.71875,-0.80859375,-6.25,1.578125,4.25,4.46875,-1.0078125,8,-2.3125,2.546875,-0.00555419921875,1.5625,-1.8671875,-2.375,-2.53125,5.25,-0.69140625,-2.96875,-0.68359375,1.6171875,2.96875,-3.015625,-1.734375,0.4140625,-2.9375,2.53125,-1.6640625,-4.5625,-1.9296875,3.234375,-2.734375,2.359375,-4.125,-3.046875,4.5,-5.875,-2.984375,-1.8515625,-2.8125,-0.7734375,0.46484375,1.3984375,5.28125,0.68359375,-1.3359375,0.51171875,8.625,-0.055908203125,3.578125,6.5,-2.390625,6.34375,5.5625,0.7265625,1.578125,-2.921875,4.90625,-2.953125,-0.62890625,2.453125,3.46875,4.5625,2.671875,-1.9140625,0.859375,-3.03125,1.703125,1.96875,0.59375,-1.4140625,-3.140625,-1.2109375,1.2890625,-3.21875,-6.5625,-6.78125,2.765625,-0.78515625,-0.3515625,1.8125,-4.53125,-5.03125,2.171875,-1.8515625,-5.46875,-1.78125,0.380859375,2.640625,1.65625,3.640625,-2.140625,2.46875,1.21875,4.28125,-2.796875,-4.40625,2.796875,-2.0625,-1.9765625,4.28125,-0.6796875,4.4375,4.28125,-4.03125,-0.01416015625,5.53125,-1.4609375,7.25,3.578125,3.6875,-2.375,-8.0625,-4.71875,-1.9453125,3.71875,4.3125,4.40625,-5.03125,3.21875,-3.734375,-6.625,4.1875,-3.4375,-6.4375,-3.15625,3.859375,-1.9140625,-1.78125,1.8046875,0.5,2.3125,-1.2421875,-4.375,4.0625,3.875,0.1259765625,-1.0546875,2.015625,3.328125,1.1484375,1.7265625,1.8046875,-0.462890625,-5.625,3.6875,-1.0390625,2.5625,0.90625,10.4375,4.28125,-4.5625,1.9765625,8.625,-1.328125,8.625,1.4609375,2.203125,0.81640625,-0.640625,-2.90625,4.53125,-2.15625,1.5,0.12255859375,-5.6875,3.140625,1.2890625,1.578125,1.5625,2.71875,-1,-4.84375,-1.8671875,3.484375,-2.578125,3.4375,0.1025390625,-1.40625,-7.375,1.4921875,1.5546875,-4.71875,-3.765625,2.703125,-1.71875,3.078125,-0.380859375,2.265625,0.24609375,3.21875,-2.0625,7.65625,2.640625,2.734375,2.046875,1.8359375,2.46875,4.53125,3.484375,1.8359375,-2.078125,-0.83984375,2.03125,5.8125,0.439453125,3.75,8.6875,0.251953125,0.408203125,6.84375,-2.515625,-1.78125,-3.578125,-3.78125,1.6015625,-0.279296875,2.671875,-5.65625,-4.0625,-2.328125,2.984375,3.515625,-3.359375,-2.34375,-2.703125,-0.51171875,-6.4375,1.484375,3.671875,-9.0625,1.8828125,5.625,3.96875,1.984375,1.265625,-0.33203125,-4.125,0.333984375,-2.4375,-5.875,-0.58203125,1.890625,-2.390625,5.09375,-1.5546875,3.515625,-0.7421875,5.1875,-2.28125,-0.0927734375,-3.046875,-4.3125,8.8125,-0.232421875,-1.90625,1.0703125,-3.078125,-3.5625,-10.25,2.5,1.1171875,4.96875,-2.921875,1.40625,0.40234375,-3.640625,12.75,3.90625,-1.8203125,1.9921875,-0.63671875,-6.03125,-1.984375,-2.046875,2.046875,-5.59375,1.84375,3.6875,4.5,-1.9296875,3.4375,-1.7421875,-0.9296875,-1.109375,-4.5625,-1.9375,2.671875,-3.765625,2.34375,9.625,-4.75,2.03125,-2.109375,-6.1875,4.75,-0.03662109375,-0.11376953125,-2.140625,-5.125,-1.9921875,-2.78125,-1.4296875,-6.65625,4.96875,-0.984375,5.375,0.97265625,3,3.296875,-4.1875,-5.03125,8.4375,-1.5,3.296875,5.71875,0.55078125,0.68359375,-3.515625,-4.6875,2.46875,-5.46875,0.953125,5.71875,3.328125,-1.640625,1.0234375,-6.21875,2.40625,2.328125,-0.68359375,6.53125,6.90625,-2.265625,2.78125,1.9140625,-0.71484375,-2.28125,-0.2294921875,-1.078125,6.34375,1.1875,-3.890625,-3.796875,-0.5859375,5.03125,-2.375,0.7734375,-1.21875,-4.15625,2.59375,-1.15625,3.6875,0.91796875,0.90625,-1.8046875,-5.125,0.087890625,-2.625,0.29296875,-1.7734375,-3.28125,4.25,1.515625,-0.484375,1.59375,0.67578125,-3.53125,-0.46484375,0.59765625,-1.15625,0.65625,2.5625,-0.5703125,-0.984375,1.5546875,-0.3828125,-2.21875,1.0546875,-1.2734375,2.40625,-6.9375,-0.6484375,-0.2490234375,-2.125,-8.375,-0.4765625,1.0703125,-3.78125,2.71875,1.96875,-1.2578125,-3.0625,4.4375,1.421875,1.8671875,-6.90625,2.15625,-1.8828125,3.328125,2.140625,-1.7421875,0.59375,-1.4296875,-2.765625,4.375,3.546875,-0.69921875,3.453125,0.68359375,-3.265625,-3.625,0.1630859375,-4.90625,4.75,-0.236328125,-1.859375,5.21875,2.203125,-1.5,1.625,0.98828125,-6.28125,-4.78125,2.96875,3.171875,-3.078125,-3.96875,0.470703125,-1.4296875,-4.4375,3.078125,3.84375,-1.1171875,-2.8125,3.40625,4.375,-2.203125,0.0830078125,1.1171875,0.52734375,2.703125,-1.9375,-3.140625,-0.1103515625,0.130859375,4.71875,-5.8125,-6.84375,3.015625,-2.875,0.2001953125,1.15625,4.5625,0.46875,-1.8984375,-1.9296875,-3.0625,-3.46875,-2.828125,3.53125,-1.078125,-2.53125,-2.90625,0.29296875,8.3125,1.90625,0.369140625,-2.375,-0.11572265625,2.453125,-1.71875,0.50390625,4.4375,7.90625,-4.03125,-0.63671875,3.53125,-8.125,0.94921875,-1.375,-1.15625,-0.94921875,2.3125,2.1875,-6.25,-0.7890625,0.0115966796875,5.03125,-3.453125,-3.828125,5.15625,-4.8125,-3.09375,1.859375,-0.6875,4.0625,1.296875,-1.34375,2.875,2.984375,2.65625,1.8203125,-2.53125,-3.640625,-3.3125,1.2890625,2.265625,-2.234375,2.296875,4,-5.4375,0.90234375,-2.25,-0.6953125,-0.212890625,-0.515625,5.90625,2.125,2.25,-6.09375,1.2578125,0.50390625,-0.416015625,-0.7421875,-1.1484375,6.71875,-0.5,-0.2294921875,0.94921875,2.09375,-1.1953125,1.640625,-3.796875,-2.453125,-3.109375,-1.796875,-1.0234375,-4.03125,-5.5,4.4375,6,-1.234375,-1.6796875,2.171875,5.5,3.984375,-0.84375,1.515625,3.421875,-2.5,0.23828125,-5.40625,2.609375,-7.84375,-2.53125,-1.6875,2.921875,3.75,-4.15625,3.765625,-2.578125,2.4375,-1.4375,4.4375,-10.5625,2.046875,-2.15625,-2.796875,-2.28125,-0.57421875,3.171875,-0.44921875,2.109375,1.3671875,-0.75,3.953125,5.46875,-1.5,1.765625,2.1875,2.46875,-0.5859375,2.515625,-2.125,-8.25,1.3125,-1.1484375,1.09375,7.5625,1.9375,-1.7734375,2.46875,0.88671875,-1.5703125,-1.7265625,4.0625,3.015625,-1.546875,4.25,-3.90625,5.40625,-3.28125,1.7265625,-3.265625,-6.15625,0.279296875,1.9296875,-5.5625,-4.09375,2.859375,0.216796875,5.78125,3.421875,-5.375,1.21875,-0.41796875,1.109375,2,0.30078125,-0.03759765625,-4.75,3.921875,4.1875,-2.40625,7.03125,-1.5703125,-1.6484375,-1.1171875,2.40625,-1.7734375,0.373046875,1.84375,0.287109375,-0.78125,-3.484375,0.96484375,0.5703125,-6.625,-7.21875,1.7265625,-1.7734375,7.0625,0.73046875,-0.859375,-3.15625,2,1.5546875,6.375,3.3125,3.765625,4.5,3.765625,-2.390625,2.671875,-3.6875,-6.09375,7,-6.53125,-1.8515625,1.015625,0.859375,-0.2578125,-1.0234375,-0.3515625,-0.71484375,-3.484375,-6.09375,-2.359375,-1.875,2.015625,-1.6484375,2.203125,0.57421875,-4.09375,-0.5703125,-1.6484375,-1.6875,-1.6640625,4.15625,-5.625,1.484375,5.71875,2.046875,-1.5234375,4.15625,3.09375,-0.47265625,-4.78125,0.7109375,-6.875,1.6015625,1.46875,-0.6015625,0.50390625,-8,2.03125,-2.4375,3.5,-0.671875,-0.05078125,-1.265625,-3.296875,-1.3984375,-0.91796875,-5.40625,-0.171875,1.6953125,1.125,-1.8359375,0.671875,3.078125,-0.52734375,0.384765625,-1.125,2.046875,0.40625,2.34375,-4.78125,-2.90625,1.28125,0.9140625,-2.03125,6.53125,0.91796875,0.79296875,3.546875,1.7265625,-5.5,-5.78125,3.921875,-2.8125,-1.796875,-3.25,2.421875,-1.359375,6.53125,-2.21875,-5.53125,-3.703125,1.6484375,3.15625,-2.609375,-3.09375,4.78125,1.8359375,2.765625,-2.15625,-7.5,1.609375,0.98828125,-0.146484375,-1.140625,8.625,-1.9296875,-0.4765625,-4.4375,-3.234375,2.046875,0.875,2.046875,-0.76171875,-1.2734375,0.69921875,0.4765625,-2.34375,-0.55078125,0.6015625,-2.546875,1.75,0.07177734375,4.875,-2.53125,0.3984375,-1.2734375,-0.50390625,-0.10009765625,4.3125,8.75,-1.765625,-0.96875,0.35546875,2.984375,-3.59375,6.6875,1.3515625,7.75,-1.1640625,0.25,1.03125,0.375,-2.171875,4.59375,-5.25,-2.84375,-1.890625,1.21875,-2.5625,0.671875,-3.984375,-0.498046875,4.40625,-0.455078125,-0.007568359375,2.609375,0.79296875,-0.201171875,-3.09375,-1.3125,-4.71875,-2.515625,-0.14453125,2.03125,-3.03125,-0.4921875,-0.33984375,5.84375,-0.357421875,-1.4453125,-2.59375,1.53125,1.859375,1.171875,-0.8046875,0.255859375,0.58984375,3.3125,-1.015625,-4.34375,-0.94921875,8.4375,4.21875,-6.875,1.5703125,-0.43359375,1.4453125,-4.8125,-1.4609375,-2.15625,-1.4921875,-4.1875,1.1328125,0.419921875,-3,-0.06494140625,4.5,-1.2890625,-0.15625,3.46875,4.0625,0.478515625,2.96875,-2.125,4.375,2.21875,-2.09375,-5.96875,-1.703125,0.48046875,-2.75,-1.4140625,2.03125,6.15625,0.55859375,2.625,-1.0625,2.28125,-1.6953125,3.78125,5.125,-4.59375,-2.703125,-2.3125,-9.5625,-4.03125,-1.7421875,-2.921875,-5.34375,-4.25,-0.86328125,-1.2421875,-8,0.0966796875,-2.234375,-3.265625,1.4453125,2.953125,1.7578125,-5.75,3.125,4.125,2.578125,2.546875,0.84765625,5.46875,-0.050537109375,-2.96875,1.4453125,-3.4375,4.15625,-1.03125,3.546875,6.25,-0.453125,-4.96875,4.78125,2.96875,5.53125,-7.375,-2.625,-0.337890625,-1.671875,-0.458984375,-1.7578125,2.546875,-4.5,-5.5,1.078125,-3.203125,1.2265625,4.6875,-0.8046875,6.78125,1.6328125,0.419921875,2.140625,2.71875,0.62109375,0.169921875,1.7421875,-5.9375,3.234375,-2.171875,3.265625,-0.296875,-1.5234375,2.734375,-0.7578125,-0.310546875,2.8125,2.734375,10.3125,0.515625,4,-2.3125,0.63671875,-1.7265625,-0.2392578125,2.25,2.015625,0.79296875,-1.4765625,0.7890625,-0.44921875,0.478515625,-0.4609375,-13.25,-1.9609375,-7.25,-1.9296875,7.0625,-2.1875,-1.9921875,1.4296875,2.6875,3.484375,5.125,-0.58984375,3.375,-0.60546875,0.80859375,5.96875,-4.25,1.03125,3.359375,2.546875,5.21875,0.154296875,-0.44921875,-3.203125,8,2.25,-1.4140625,0.8359375,2.796875,-1.3046875,-2.34375,3.09375,-3.171875,2.96875,-4.9375,0.5859375,4.15625,0.65625,-3.890625,-3.4375,-2,-0.62890625,1.3828125,1.375,-2.59375,0.18359375,0.94921875,-4.1875,3.328125,-0.59375,0.140625,-5.53125,1.03125,4.65625,0.703125,-0.109375,-1.8515625,1.4453125,-0.8984375,4.3125,2.78125,-2.734375,0.2734375,2.21875,1.7421875,-0.125,1.03125,1.1328125,2.921875,-3.09375,-0.353515625,-0.44140625,-1.625,1.4765625,-3.1875,1.6640625,3.203125,1.3984375,-3.984375,2.21875,0.79296875,-0.11669921875,2.96875,-5.125,-1.9921875,-1.1015625,-0.71484375,-4.0625,-0.9140625,-4.375,-0.1455078125,5.46875,-5,3.4375,-2.515625,8.1875,0.1298828125,-1.421875,1.2890625,-2.828125,2.59375,-3.390625,-1.234375,3.484375,-0.92578125,2.125,-3.546875,1.8984375,-2.078125,-0.46484375,6.09375,-3.953125,-1.9765625,0.7421875,3.21875,-5.0625,-3.296875,0.1611328125,0.8515625,0.009765625,-1.8984375,1.4765625,-2.03125,4.4375,-4.75,3.390625,-4.65625,-3.90625,0.28125,0.07568359375,7.90625,4.25,-3.796875,-3.421875,-0.6015625,-7.0625,-3.421875,-3.859375,6.65625,-0.52734375,0.96875,2.078125,2.390625,-0.01031494140625,1.46875,-2.96875,3.203125,5.28125,0.294921875,3.046875,2.1875,-1.125,-4.40625,0.3125,-3.171875,7.0625,3.0625,0.404296875,3,-1.8984375,1.484375,-1.03125,-1.0625,-2.828125,2.171875,1.71875,-2.5,-3.28125,1.046875,-3.859375,0.72265625,-5.40625,-2.578125,-5.3125,2.765625,2.3125,-0.81640625,-0.7578125,4.4375,0.318359375,3.328125,-5.53125,-3.890625,3.8125,0.9765625,0.333984375,2.84375,-0.6796875,-5.03125,-0.9375,0.201171875,1.9140625,-4.1875,-3.609375,3.328125,2.46875,0.283203125,-3.9375,-4.40625,-3.453125,2.390625,4.1875,-0.96484375,0.353515625,0.06005859375,-1.53125,2.171875,-2.65625,4.5,-3.109375,-4.15625,-0.47265625,0.734375,3.578125,-3.203125,-1.0703125,1.4296875,-3.4375,0.7578125,1.2734375,-0.11279296875,-1.9453125,3.171875,-2,-3.65625,-5.4375,5.78125,-2.0625,0.45703125,-3.875,-2.65625,-3.1875,-1.421875,-0.6640625,1.7421875,0.0703125,5.78125,-0.63671875,2.8125,0.478515625,-0.8828125,0.0712890625,3.453125,-0.271484375,-2.90625,1.8359375,-4.59375,-4.65625,0.7578125,-8.0625,-2.0625,2.90625,-2.40625,2.671875,-2.671875,2.375,-1.1015625,-2.21875,-1.8203125,-0.8203125,0.83984375,5.375,2.171875,0.2216796875,0.38671875,1.8984375,0.859375,-1.109375,-1.8515625,-0.25,5.34375,0.62109375,2.765625,-3.359375,-2.34375,4.46875,-0.59375,-3.75,0.8984375,-0.357421875,0.6640625,4.5625,0.9609375,-3.796875,-2.9375,-6.15625,4.03125,0.73828125,1.828125,-4.625,1.5,-3.0625,0.1748046875,2.03125,-6.5625,-2.546875,3.328125,2.828125,5.46875,1.328125,-2.421875,-4.53125,2.203125,-0.396484375,-1.6171875,-2.234375,-1.7265625,-0.96875,-3.765625,4.125,-2.515625,4.25,-1.3359375,-2.8125,-0.8671875,0.61328125,-0.203125,0.47265625,-0.353515625,-0.88671875,4.0625,-0.3515625,7,2.171875,-4.0625,4.59375,2.515625,0.412109375,-1.5625,3.75,-1.109375,-2.3125,3.921875,2.890625,-4.0625,4.96875,2.125,3.375,-3.46875,-2.1875,-0.9921875,4.5625,0.287109375,1.28125,-4.34375,0.1630859375,4.0625,-0.1884765625,0.8671875,-1.765625,0.3046875,0.65234375,0.52734375,2,1.921875,3.4375,-0.52734375,1,-0.92578125,-1.2265625,2.328125,-0.1328125,-0.703125,-1.8828125,3.21875,-1.6953125,-1.875,-6,1.2421875,-3.46875,2.21875,3.1875,2.875,2.234375,-2.828125,-1.625,-2.640625,-5.25,-3.140625,1.75,1.09375,-1.75,1.875,-0.1181640625,2.546875,5.84375,0.130859375,4.6875,-3.109375,2.5,1.140625,0.875,0.046630859375,4.3125,-1.8203125,-2.21875,3.640625,-4.46875,3.71875,-4.53125,-3.078125,-0.63671875,-0.10986328125,2.640625,6.625,-4.5625,-3.953125,5.21875,1.328125,4.59375,3.78125,-2.078125,-1.484375,0.79296875,1.3515625,5.46875,0.93359375,2.953125,-2.734375,6.9375,5.65625,0.90625,2.359375,0.166015625,-2.6875,-6.4375,5.125,1.3984375,1.984375,-2.375,1.6875,3.109375,0.1533203125,3.640625,-5.5,0.8671875,1.2109375,0.90625,0.5234375,-3.15625,0.103515625,2.640625,0.33203125,-1.6875,5.84375,0.97265625,4.125,-0.72265625,3.34375,2.328125,3.703125,-2.03125,1.5234375,-3.46875,3.578125,-1.3984375,2.15625,-5.5,1.0546875,3.640625,4.3125,-1.625,-3.5625,2.21875,0.275390625,-0.5,-4.46875,4.21875,3.59375,2.5625,-6.9375,-3.328125,-0.05029296875,0.2060546875,1.234375,-3.484375,1.171875,1.6796875,-4.625,-3.265625,1.296875,1.625,-5.65625,-6.0625,-3.203125,1.65625,1.3203125,3.1875,3.21875,-0.8203125,3.40625,-0.55078125,3.046875,4.28125,-1.1328125,1.5546875,0.9375,-2.75,4.125,-0.263671875,-2.671875,1.5546875,-0.50390625,-2.140625,0.50390625,-2.296875,-1.0703125,-4.21875,-0.85546875,2.328125,-1.09375,5.125,-3.96875,0.30078125,3.609375,-1.4375,-2.28125,-2.65625,0.5703125,-2.921875,-2.578125,-1.9140625,3.609375,2.984375,2.046875,0.58203125,-0.6015625,-3.265625,-6.40625,-5.65625,3.578125,-2.515625,2.859375,0.439453125,-4.25,2.078125,2.8125,1.78125,-0.1640625,-0.55859375,2.765625,4.59375,0.455078125,-1.7265625,-0.466796875,3.609375,-4.5625,-3.78125,0.515625,1,-3.171875,2.28125,-3.125,-1.8359375,0.79296875,4.5,-0.5078125,-2.859375,-1.75,-2.40625,-2.875,-3.03125,-2.859375,2.5625,1.859375,3.296875,0.1689453125,-0.421875,-5,3.71875,16.875,0.9375,-4.71875,2.421875,-3.140625,2.65625,3.171875,4.8125,-1.7109375,-1.96875,-2.1875,1.765625,0.01031494140625,1.4140625,-2.140625,1.7421875,1.9921875,-0.48828125,-4.125,-1.9765625,-1.328125,0.84765625,-0.7578125,2.96875,0.408203125,2.265625,-0.734375,-0.259765625,0.2333984375,-3.234375,-4.46875,-4.4375,2.265625,-1.7578125,4.75,-4.25,5.375,0.1845703125,-2.9375,-2.09375,-3.296875,-3.171875,1.0234375,-0.75,-1.9453125,4.34375,-0.72265625,1.09375,0.37890625,-0.337890625,-3.546875,-3.046875,-2.6875,7.25,0.62890625,-5.71875,-1.546875,-4.84375,-4.5625,0.58984375,2.796875,-2.328125,1.6328125,1.453125,-1.828125,-2.171875,-1.953125,0.85546875,3,-5.125,-5.625,0.13671875,1.5546875,3.359375,2.796875,-4.0625,1.5703125,5.3125,2.6875,0.69140625,-0.75,1.4453125,-1.3828125,-2.5,-0.91015625,1.4609375,-4.03125,1.109375,1.4453125,-4.875,11.25,-8.625,4.8125,4.0625,-4.75,-0.1865234375,2.796875,1.796875,-1.6796875,-0.169921875,2.953125,2.453125,3.359375,-0.306640625,6.09375,1.5234375,0.388671875,0.73828125,2.9375,3.578125,2.4375,2.9375,-0.828125,-1.9609375,1.3046875,1.7734375,-2.484375,-3.46875,-1.4609375,-4.4375,6,1.6171875,-2.765625,-1.2578125,-10.5,-3.421875,-2.328125,-5.84375,4.5,-2.65625,2.46875,3.421875,-0.609375,-1.078125,-2.53125,-5,2.296875,4.0625,0.208984375,-0.3984375,-6.0625,2.84375,3.546875,-3.984375,-2.09375,1.4453125,-3.265625,3.296875,-0.1923828125,4.9375,-3.578125,3.9375,2.03125,-2.546875,-5.8125,3.171875,-3.765625,-2.234375,-5.3125,-2.453125,-2.078125,-3.328125,-0.6171875,-0.35546875,-2.078125,-1.03125,1.6171875,-0.60546875,-3.15625,2.921875,2.96875,-4.375,-2.625,0.58203125,0.73046875,-4.28125,1.1875,5.1875,-0.54296875,1.5,0.55078125,0.078125,-0.3203125,-4.34375,0.81640625,1.71875,-4.03125,-0.71875,-1.359375,-2.828125,-2.4375,-2.78125,-3.375,3.875,3.59375,-5.0625,1.9609375,-0.34765625,0.014892578125,-1.4453125,-1.546875,6.4375,2.234375,-1.6484375,5.59375,1.03125,-4.15625,-2,-2.046875,-1.1484375,-1.2734375,6.3125,1.2578125,2.375,-5.90625,7.53125,2.453125,1.7265625,-0.43359375,2.34375,1.6796875,-3.71875,-5.40625,2.46875,2.75,3.84375,-4.59375,0.6328125,0.53515625,0.53125,-4.28125,1.90625,-0.259765625,0.482421875,-3.140625,-7.59375,-0.109375,0.90625,-1.8828125,1.5234375,4.25,-2.96875,1.3828125,0.95703125,-0.58984375,3.640625,3.28125,-2.828125,1.90625,-0.1904296875,2.625,-2.34375,1.4921875,-3.71875,-4.96875,-3.109375,-1.765625,1.8828125,-2.625,0.67578125,-0.357421875,-4.1875,2.109375,-2.25,1.125,1.09375,0.2578125,-6.25,3.984375,5.1875,-4.15625,4.4375,-5.53125,-2.4375,-1.640625,2.21875,-1.9140625,-6.46875,2.0625,4.5,-3.390625,2.203125,3.546875,-1.625,-0.4453125,-2.25,5.3125,-1.015625,4.78125,-0.6953125,3.953125,3.9375,-1.28125,-0.061279296875,-5.125,0.470703125,-2.28125,-3.84375,5.53125,-1.921875,2.46875,5.21875,4.9375,-9,-1.96875,0.54296875,-0.1845703125,3.578125,3.109375,-1.3671875,1.0234375,0.028076171875,-0.30859375,4.4375,-0.9296875,-1.46875,-3.65625,4.96875,-0.1728515625,-4.0625,2.984375,2.609375,-4.15625,4.34375,-2.75,-2.6875,-0.6875,-0.1396484375,-5.625,1.8046875,2.6875,-0.92578125,3.4375,3.109375,1.203125,3.59375,-2.640625,-10.0625,0.0703125,2.75,5.3125,1.7265625,2.3125,0.0859375,-1.0625,3.640625,-4.5625,0.46875,-1.484375,-9.5,0.255859375,-4.15625,-1.609375,-3.453125,-1.4921875,-1.9453125,3.90625,1.3984375,-0.8515625,3.5,2.921875,0.453125,4.15625,-0.361328125,-3.578125,1.2734375,1.75,-5.28125,-1.90625,4.8125,3.578125,-2.203125,-2.0625,3.84375,-4.28125,-0.70703125,4.3125,4.28125,2.15625,-0.828125,-3.234375,2.84375,-2.546875,-2.828125,1.703125,-3.421875,2.453125,-1.4375,2.578125,1.296875,-2.640625,-2.03125,-4.15625,-2.71875,3.484375,0.28515625,0.9765625,-2.265625,-1.1171875,3.234375,3.5625,-2.359375,-2.109375,2.796875,-1.3515625,-4.28125,-1.0859375,1.0859375,-5.90625,-2.609375,2.734375,3.4375,-2.5625,-3.5625,-2.125,1.6171875,1.3046875,-0.8984375,-0.1318359375,-3.53125,2.65625,5.0625,-2.9375,-3.75,-1.6171875,-0.486328125,-5.03125,-3.609375,-0.1767578125,1.140625,-0.73046875,3.890625,-1.40625,0.47265625,4.4375,-3.65625,-3.21875,3.96875,3.359375,-3.203125,-1.46875,2.25,-3.375,1.03125,5.4375,-2.390625,-2.234375,0.41796875,-2.171875,-4.28125,2.34375,1.2265625,-3.734375,-7.875,5.96875,1.0703125,4.34375,4.125,-3.90625,4.0625,-4.6875,1.8828125,-1.265625,1.015625,1.3828125,-5.65625,-1.1875,-2.5,-3.5,0.5390625,-1.734375,-3.5625,0.66015625,8.0625,-1.328125,-2.59375,-2.953125,-3.515625,3.3125,-4.15625,-7.625,0.1181640625,-7.34375,1.734375,-2.1875,1.75,-5.59375,1.9140625,-1.078125,1.734375,-2.984375,0.27734375,-0.384765625,1.21875,0.54296875,4.6875,1.2109375,1.984375,-0.1484375,2.71875,0.0791015625,1.875,-1.453125,-0.4921875,1.21875,-1.234375,0.33203125,0.69921875,-2.734375,0.1708984375,-1.7578125,-0.263671875,-1.015625,1.7578125,2.9375,-0.640625,-0.291015625,-1.6875,1.703125,-4.5,1.3125,-1.796875,0.859375,-0.78515625,-1.0078125,1.9609375,-2.328125,1.6640625,1.015625,1.640625,0.01068115234375,-1.5,2.234375,2.6875,-0.031982421875,-2.328125,-1.8046875,-0.55859375,-1.7421875,1.7421875,0.55078125,-2.0625,2.9375,-1.640625,-0.41015625,0.890625,1.7265625,0.44140625,-1.6484375,2.40625,-1.8671875,1.2890625,1.0859375,-1.5234375,2.609375,0.63671875,1.03125,1.2734375,0.9765625,-2,0.64453125,0.2578125,-1.4375,-0.291015625,3.484375,-1.7265625,0.31640625,-1.078125,-0.5625,1.0859375,-0.8671875,1.2109375,0.15625,-0.396484375,-2.75,2.640625,-2.125,-1.2578125,-0.42578125,0.29296875,-0.5703125,0.8984375,0.08935546875,1.2109375,-0.29296875,2.28125,-0.73828125,2.171875,-0.020263671875,-0.2060546875,1.3359375,3.421875,-1.984375,0.7421875,-2.0625,-1.1328125,1.3203125,-0.3046875,1.15625,-0.93359375,-2,1.2421875,1.1328125,-2.984375,-0.734375,2.265625,-0.189453125,-1.1328125,-0.609375,1.2265625,-0.75390625,-0.38671875,0.419921875,-0.89453125,2,3.265625,-1.0625,2.5,-1.453125,0.396484375,0.73046875,1.046875,2.3125,0.07958984375,-2.34375,-0.9296875,2.71875,-1.4375,0.37109375,0.890625,-1.53125,-0.1396484375,1.3359375,0.5703125,1.640625,-0.06982421875,-1.859375,-0.330078125,-0.6796875,1.609375,1.65625,-1.6875,0.68359375,-1.8359375,-0.53125,-1.015625,2.765625,-1.7578125,-2.140625,-0.78515625,-1.1015625,-0.83203125,-0.498046875,0.11962890625,-0.1298828125,0.60546875,1.125,1.5,0.4296875,-0.609375,1.4375,-0.08056640625,0.68359375,-1.1875,-1.5234375,1.484375,1.2421875,2.34375,-1.359375,1.34375,0.9296875,0.8828125,-1.1796875,1.9453125,-0.5234375,0.314453125,0.010986328125,-0.1181640625,1.40625,2.21875,0.318359375,0.5859375,-0.1328125,1.40625,0.69921875,1.375,-1.3046875,-2.203125,-1.0078125,-1.4296875,-2.125,0.361328125,-0.0615234375,-1.3046875,-0.1904296875,0.034912109375,-0.86328125,1.375,1.1796875,1.5390625,-0.828125,-0.58203125,0.1787109375,-0.328125,0.25390625,0.8828125,-0.8046875,-0.78125,-1.1171875,-2.0625,1.578125,0.88671875,-1.09375,-0.2890625,2.0625,-1.5,1.0078125,-2.78125,0.55078125,-1.828125,-0.341796875,0.0859375,-3.265625,0.34765625,-0.12451171875,-2.15625,-3.078125,-1.75,-0.85546875,-2.375,-0.3203125,4,-0.81640625,-1.21875,2.03125,0.08203125,-1.0078125,-0.94921875,1.7578125,2.84375,-0.8203125,3.859375,0.349609375,-0.16015625,-1.3984375,-1.265625,0.52734375,-1.2890625,0.294921875,-0.84765625,-0.8046875,-1.6796875,-3.109375,0.05859375,-4.1875,-2.125,0.1337890625,0.90625,1.890625,-0.08447265625,-0.7421875,-0.56640625,-0.96875,2.796875,-0.267578125,0.18359375,1.4375,0.27734375,0.46875,-1.4140625,0.92578125,-0.84375,2.953125,-1.171875,-0.50390625,-2.65625,-1.5546875,-4.1875,1.453125,2.484375,0.421875,2.96875,1.3671875,-0.5546875,-2.5625,0.07421875,0.00909423828125,-4.75,-0.373046875,-0.7265625,0.07275390625,-1.4140625,-0.7109375,-0.1318359375,-0.609375,-1.328125,-0.51953125,-1.828125,-0.271484375,-2.28125,2.984375,1.7890625,1.875,2.3125,0.3125,-0.31640625,1.1875,2.359375,1.1484375,0.6953125,0.255859375,0.408203125,-1.09375,2.09375,0.337890625,0.4609375,-1.2265625,0.2275390625,1.1875,2.5625,1.734375,-0.76171875,0.85546875,0.328125,-1.9140625,-1.40625,0.31640625,0.296875,1.140625,0.333984375,1.03125,-1.2890625,0.416015625,-0.6875,0.9453125,1.7578125,-1.953125,1.109375,-0.134765625,0.1787109375,-1.5,1.203125,1.15625,1.8203125,-0.48046875,2.140625,1.1640625,0.48828125,1.8515625,2.609375,-0.361328125,1.421875,-0.86328125,1.953125,0.51953125,-2.484375,3.15625,-0.34375,-0.47265625,-0.56640625,1.2890625,1.359375,-0.60546875,-0.25,-0.38671875,2.015625,0.52734375,0.14453125,1.8828125,0.67578125,-0.546875,-0.77734375,-0.6015625,-1.09375,-2.328125,-1.0078125,-3.0625,-0.37109375,-0.9375,1.765625,-0.828125,-1.484375,-0.142578125,1.390625,-0.02099609375,1.3203125,1.6171875,-1.0859375,2.09375,0.154296875,0.1962890625,0.89453125,-0.97265625,-1.2421875,1.15625,0.82421875,-0.59765625,4.625,0.1962890625,2.28125,-0.65625,-1.0390625,-0.78515625,3.59375,-0.44921875,-0.4375,-1.6953125,1.140625,-0.296875,-1.25,-0.76953125,-1.3984375,-0.9765625,1.78125,-0.87109375,-3.234375,-2.171875,0.330078125,-1.875,0.48828125,-1.859375,-1.0390625,2.40625,1.734375,-0.63671875,0.216796875,1.125,-1.0234375,0.58984375,-0.4296875,0.3515625,1.6015625,-1.2109375,1.765625,0.5859375,2.796875,-3.921875,-0.298828125,2.171875,1.578125,-0.458984375,-1.015625,-0.51171875,2.109375,0.369140625,-0.018798828125,-0.50390625,-4.46875,0.0135498046875,-0.043212890625,-3.21875,-0.09423828125,0.4921875,1.2421875,0.6640625,-3.15625,0.73046875,-1.5078125,-1.6328125,3.46875,-0.55078125,-0.41796875,0.58203125,1.1640625,-0.83203125,-0.84765625,1.53125,0.17578125,-3.484375,-1.1015625,-0.1591796875,-0.875,0.59765625,0.01373291015625,0.099609375,0.546875,-0.36328125,-1.171875,-1.1328125,-0.33984375,-0.08056640625,1.015625,4,1.1484375,1.265625,1.2109375,-2.125,4.5625,-2.515625,-0.96484375,1.1015625,1.3515625,-1.1796875,3.921875,1.109375,0.2265625,-2,0.55859375,2.96875,0.765625,0.9453125,0.671875,1.28125,1.7421875,1.78125,-1,-1.8671875,1.5,-0.35546875,-2.5,0.012451171875,0.2578125],"index":1,"object":"embedding"}],"model":"doubao-embedding-text-240715","object":"list","usage":{"prompt_tokens":7,"total_tokens":7}}, + "headers": { + "Content-Type": "application/json" + } + } + } +] + + + diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java index 0e891710733..c38d4adc927 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java @@ -170,25 +170,24 @@ public JobMetricsSummary getJobMetricsSummary(Long jobId) { public Map getClusterHealthMetrics() { Set members = hazelcastClient.getHazelcastInstance().getCluster().getMembers(); Map healthMetricsMap = new HashMap<>(); - members.stream() - .forEach( - member -> { - String metrics = - hazelcastClient.requestAndDecodeResponse( - member.getUuid(), - SeaTunnelGetClusterHealthMetricsCodec.encodeRequest(), - SeaTunnelGetClusterHealthMetricsCodec::decodeResponse); - String[] split = metrics.split(","); - Map kvMap = new LinkedHashMap<>(); - Arrays.stream(split) - .forEach( - kv -> { - String[] kvArr = kv.split("="); - kvMap.put(kvArr[0], kvArr[1]); - }); - healthMetricsMap.put( - member.getAddress().toString(), JsonUtils.toJsonString(kvMap)); - }); + members.forEach( + member -> { + String metrics = + hazelcastClient.requestAndDecodeResponse( + member.getUuid(), + SeaTunnelGetClusterHealthMetricsCodec.encodeRequest(), + SeaTunnelGetClusterHealthMetricsCodec::decodeResponse); + String[] split = metrics.split(","); + Map kvMap = new LinkedHashMap<>(); + Arrays.stream(split) + .forEach( + kv -> { + String[] kvArr = kv.split("="); + kvMap.put(kvArr[0], kvArr[1]); + }); + healthMetricsMap.put( + member.getAddress().toString(), JsonUtils.toJsonString(kvMap)); + }); return healthMetricsMap; } diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties b/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties index c5b54e52eb9..8e1c93a35cc 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties +++ b/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties @@ -29,7 +29,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -39,7 +39,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java index fdb21025814..60551202e83 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java @@ -24,6 +24,8 @@ public class Constant { public static final String DEFAULT_SEATUNNEL_CLUSTER_NAME = "seatunnel"; + public static final String REST_SUBMIT_JOBS_PARAMS = "params"; + /** * The default port number for the cluster auto-discovery mechanism's multicast communication. */ diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java index a58e68d92d8..666021de1f8 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java @@ -134,10 +134,12 @@ public Enumeration getResources(String name) throws IOException { return new Enumeration() { final Iterator iter = result.iterator(); + @Override public boolean hasMoreElements() { return iter.hasNext(); } + @Override public URL nextElement() { return iter.next(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java index fe227df50de..0e057907138 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java @@ -21,6 +21,8 @@ import org.apache.seatunnel.api.common.metrics.RawJobMetrics; import org.apache.seatunnel.api.event.EventHandler; import org.apache.seatunnel.api.event.EventProcessor; +import org.apache.seatunnel.api.tracing.MDCExecutorService; +import org.apache.seatunnel.api.tracing.MDCTracer; import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.common.utils.StringFormatUtils; @@ -316,7 +318,7 @@ private void restoreAllRunningJobFromMasterNodeSwitch() { "restore job (%s) from master active switch finished", entry.getKey())); }, - executorService)) + MDCTracer.tracing(entry.getKey(), executorService))) .collect(Collectors.toList()); try { @@ -468,11 +470,12 @@ public PassiveCompletableFuture submitJob( return new PassiveCompletableFuture<>(jobSubmitFuture); } + MDCExecutorService mdcExecutorService = MDCTracer.tracing(jobId, executorService); JobMaster jobMaster = new JobMaster( jobImmutableInformation, this.nodeEngine, - executorService, + mdcExecutorService, getResourceManager(), getJobHistoryService(), runningJobStateIMap, @@ -482,7 +485,7 @@ public PassiveCompletableFuture submitJob( metricsImap, engineConfig, seaTunnelServer); - executorService.submit( + mdcExecutorService.submit( () -> { try { if (!isStartWithSavePoint @@ -571,9 +574,11 @@ public PassiveCompletableFuture waitForJobComplete(long jobId) { } CompletableFuture future = new CompletableFuture<>(); - if (jobState == null) future.complete(new JobResult(JobStatus.UNKNOWABLE, null)); - else + if (jobState == null) { + future.complete(new JobResult(JobStatus.UNKNOWABLE, null)); + } else { future.complete(new JobResult(jobState.getJobStatus(), jobState.getErrorMessage())); + } return new PassiveCompletableFuture<>(future); } else { return new PassiveCompletableFuture<>(runningJobMaster.getJobMasterCompleteFuture()); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java new file mode 100644 index 00000000000..0c7b654b216 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java @@ -0,0 +1,100 @@ +/* + * 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.server; + +import org.apache.seatunnel.shade.com.google.common.util.concurrent.ThreadFactoryBuilder; + +import org.apache.seatunnel.api.event.Event; +import org.apache.seatunnel.common.utils.RetryUtils; +import org.apache.seatunnel.engine.server.event.JobEventReportOperation; +import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; + +import com.hazelcast.spi.impl.NodeEngineImpl; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +@Slf4j +public class EventService { + private final BlockingQueue eventBuffer; + + private ExecutorService eventForwardService; + + private final NodeEngineImpl nodeEngine; + + public EventService(NodeEngineImpl nodeEngine) { + eventBuffer = new ArrayBlockingQueue<>(2048); + initEventForwardService(); + this.nodeEngine = nodeEngine; + } + + private void initEventForwardService() { + eventForwardService = + Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setNameFormat("event-forwarder-%d").build()); + eventForwardService.submit( + () -> { + List events = new ArrayList<>(); + RetryUtils.RetryMaterial retryMaterial = + new RetryUtils.RetryMaterial(2, true, e -> true); + while (!Thread.currentThread().isInterrupted()) { + try { + events.clear(); + + Event first = eventBuffer.take(); + events.add(first); + + eventBuffer.drainTo(events, 500); + JobEventReportOperation operation = new JobEventReportOperation(events); + + RetryUtils.retryWithException( + () -> + NodeEngineUtil.sendOperationToMasterNode( + nodeEngine, operation) + .join(), + retryMaterial); + + log.debug("Event forward success, events " + events.size()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + log.info("Event forward thread interrupted"); + } catch (Throwable t) { + log.warn("Event forward failed, discard events " + events.size(), t); + } + } + }); + } + + public void reportEvent(Event e) { + while (!eventBuffer.offer(e)) { + eventBuffer.poll(); + log.warn("Event buffer is full, discard the oldest event"); + } + } + + public void shutdownNow() { + if (eventForwardService != null) { + eventForwardService.shutdownNow(); + } + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java index aad8c3db7f7..67aac64aca2 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java @@ -76,6 +76,7 @@ public LiteNodeDropOutTcpIpJoiner(Node node) { joinConfig = getActiveMemberNetworkConfig(config).getJoin(); } + @Override public boolean isClaimingMastership() { return claimingMastership; } @@ -198,6 +199,7 @@ private boolean tryClaimMastership(Collection
addresses) { return consensus; } + @Override protected Collection
getPossibleAddressesForInitialJoin() { return getPossibleAddresses(); } @@ -372,6 +374,7 @@ private Address getRequiredMemberAddress() { } @SuppressWarnings({"checkstyle:npathcomplexity", "checkstyle:cyclomaticcomplexity"}) + @Override protected Collection
getPossibleAddresses() { final Collection possibleMembers = getMembers(); final Set
possibleAddresses = new HashSet<>(); @@ -478,6 +481,7 @@ private boolean isLocalAddress(final Address address) throws UnknownHostExceptio return local; } + @Override protected Collection getMembers() { return getConfigurationMembers(config); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java index b76af4c19a0..99cd27d5642 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java @@ -76,6 +76,8 @@ public class SeaTunnelServer private volatile boolean isRunning = true; + @Getter private EventService eventService; + public SeaTunnelServer(@NonNull SeaTunnelConfig seaTunnelConfig) { this.liveOperationRegistry = new LiveOperationRegistry(); this.seaTunnelConfig = seaTunnelConfig; @@ -116,6 +118,8 @@ public void init(NodeEngine engine, Properties hzProperties) { new DefaultClassLoaderService( seaTunnelConfig.getEngineConfig().isClassloaderCacheMode()); + eventService = new EventService(nodeEngine); + if (EngineConfig.ClusterRole.MASTER_AND_WORKER.ordinal() == seaTunnelConfig.getEngineConfig().getClusterRole().ordinal()) { startWorker(); @@ -149,7 +153,7 @@ private void startMaster() { private void startWorker() { taskExecutionService = new TaskExecutionService( - classLoaderService, nodeEngine, nodeEngine.getProperties()); + classLoaderService, nodeEngine, nodeEngine.getProperties(), eventService); nodeEngine.getMetricsRegistry().registerDynamicMetricsProvider(taskExecutionService); taskExecutionService.start(); getSlotService(); @@ -176,6 +180,10 @@ public void shutdown(boolean terminate) { if (coordinatorService != null) { coordinatorService.shutdown(); } + + if (eventService != null) { + eventService.shutdownNow(); + } } @Override 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 a4717c6a81b..b32dd7c6a97 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 @@ -19,8 +19,9 @@ import org.apache.seatunnel.api.common.metrics.MetricTags; import org.apache.seatunnel.api.event.Event; +import org.apache.seatunnel.api.tracing.MDCExecutorService; +import org.apache.seatunnel.api.tracing.MDCTracer; import org.apache.seatunnel.common.utils.ExceptionUtils; -import org.apache.seatunnel.common.utils.RetryUtils; import org.apache.seatunnel.common.utils.StringFormatUtils; import org.apache.seatunnel.engine.common.Constant; import org.apache.seatunnel.engine.common.config.ConfigProvider; @@ -30,7 +31,6 @@ import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture; import org.apache.seatunnel.engine.core.classloader.ClassLoaderService; import org.apache.seatunnel.engine.core.job.ConnectorJarIdentifier; -import org.apache.seatunnel.engine.server.event.JobEventReportOperation; import org.apache.seatunnel.engine.server.exception.TaskGroupContextNotFoundException; import org.apache.seatunnel.engine.server.execution.ExecutionState; import org.apache.seatunnel.engine.server.execution.ProgressState; @@ -49,12 +49,10 @@ import org.apache.seatunnel.engine.server.task.SeaTunnelTask; import org.apache.seatunnel.engine.server.task.TaskGroupImmutableInformation; import org.apache.seatunnel.engine.server.task.operation.NotifyTaskStatusOperation; -import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; import org.apache.commons.collections4.CollectionUtils; 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; @@ -74,7 +72,6 @@ import java.io.IOException; import java.net.URL; -import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -82,7 +79,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; @@ -148,13 +144,13 @@ public class TaskExecutionService implements DynamicMetricsProvider { private final ServerConnectorPackageClient serverConnectorPackageClient; - private final BlockingQueue eventBuffer; - private final ExecutorService eventForwardService; + private final EventService eventService; public TaskExecutionService( ClassLoaderService classLoaderService, NodeEngineImpl nodeEngine, - HazelcastProperties properties) { + HazelcastProperties properties, + EventService eventService) { seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); this.hzInstanceName = nodeEngine.getHazelcastInstance().getName(); this.nodeEngine = nodeEngine; @@ -177,42 +173,7 @@ public TaskExecutionService( serverConnectorPackageClient = new ServerConnectorPackageClient(nodeEngine, seaTunnelConfig); - eventBuffer = new ArrayBlockingQueue<>(2048); - eventForwardService = - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setNameFormat("event-forwarder-%d").build()); - eventForwardService.submit( - () -> { - List events = new ArrayList<>(); - RetryUtils.RetryMaterial retryMaterial = - new RetryUtils.RetryMaterial(2, true, e -> true); - while (!Thread.currentThread().isInterrupted()) { - try { - events.clear(); - - Event first = eventBuffer.take(); - events.add(first); - - eventBuffer.drainTo(events, 500); - JobEventReportOperation operation = new JobEventReportOperation(events); - - RetryUtils.retryWithException( - () -> - NodeEngineUtil.sendOperationToMasterNode( - nodeEngine, operation) - .join(), - retryMaterial); - - logger.fine("Event forward success, events " + events.size()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - logger.info("Event forward thread interrupted"); - } catch (Throwable t) { - logger.warning( - "Event forward failed, discard events " + events.size(), t); - } - } - }); + this.eventService = eventService; } public void start() { @@ -223,7 +184,6 @@ public void shutdown() { isRunning = false; executorService.shutdownNow(); scheduledExecutorService.shutdown(); - eventForwardService.shutdownNow(); } public TaskGroupContext getExecutionContext(TaskGroupLocation taskGroupLocation) { @@ -276,6 +236,7 @@ private void submitThreadShareTask( private void submitBlockingTask( TaskGroupExecutionTracker taskGroupExecutionTracker, List tasks) { + MDCExecutorService mdcExecutorService = MDCTracer.tracing(executorService); CountDownLatch startedLatch = new CountDownLatch(tasks.size()); taskGroupExecutionTracker.blockingFutures = @@ -292,7 +253,7 @@ private void submitBlockingTask( "BlockingWorker-" + taskGroupExecutionTracker.taskGroup .getTaskGroupLocation())) - .map(executorService::submit) + .map(mdcExecutorService::submit) .collect(toList()); // Do not return from this method until all workers have started. Otherwise, @@ -415,10 +376,15 @@ public PassiveCompletableFuture deployLocalTask( seaTunnelConfig .getEngineConfig() .getTaskExecutionThreadShareMode(); - if (mode.equals(ThreadShareMode.ALL)) return true; - if (mode.equals(ThreadShareMode.OFF)) return false; - if (mode.equals(ThreadShareMode.PART)) + if (mode.equals(ThreadShareMode.ALL)) { + return true; + } + if (mode.equals(ThreadShareMode.OFF)) { + return false; + } + if (mode.equals(ThreadShareMode.PART)) { return t.isThreadsShare(); + } return true; })); executionContexts.put( @@ -459,7 +425,7 @@ public PassiveCompletableFuture deployLocalTask( r.getTaskGroupLocation(), r.getExecutionState())); notifyTaskStatusToMaster(taskGroup.getTaskGroupLocation(), r); }), - executorService); + MDCTracer.tracing(executorService)); return new PassiveCompletableFuture<>(resultFuture); } @@ -531,7 +497,8 @@ public void asyncExecuteFunction(TaskGroupLocation taskGroupLocation, Runnable t if (!taskAsyncFunctionFuture.containsKey(taskGroupLocation)) { taskAsyncFunctionFuture.put(taskGroupLocation, new ConcurrentHashMap<>()); } - CompletableFuture future = CompletableFuture.runAsync(task, executorService); + CompletableFuture future = + CompletableFuture.runAsync(task, MDCTracer.tracing(executorService)); taskAsyncFunctionFuture.get(taskGroupLocation).put(id, future); future.whenComplete( (r, e) -> { @@ -682,10 +649,7 @@ public void printTaskExecutionRuntimeInfo() { } public void reportEvent(Event e) { - while (!eventBuffer.offer(e)) { - eventBuffer.poll(); - logger.warning("Event buffer is full, discard the oldest event"); - } + eventService.reportEvent(e); } private final class BlockingWorker implements Runnable { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java index 8735048eac3..515f3ad871d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.engine.server.checkpoint; +import org.apache.seatunnel.api.tracing.MDCTracer; import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.common.utils.RetryUtils; import org.apache.seatunnel.common.utils.SeaTunnelException; @@ -187,6 +188,7 @@ public CheckpointCoordinator( return thread; }); ((ScheduledThreadPoolExecutor) this.scheduler).setRemoveOnCancelPolicy(true); + this.scheduler = MDCTracer.tracing(scheduler); this.serializer = new ProtoStuffSerializer(); this.pipelineTasks = getPipelineTasks(plan.getPipelineSubtasks()); this.pipelineTaskStatus = new ConcurrentHashMap<>(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java index 8ed7985fd68..63afe8c20fa 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java @@ -68,7 +68,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java index 62ec42f0a6a..4457b7eda2c 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java @@ -76,7 +76,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java index 967e1572311..a02cb67a49d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java @@ -40,7 +40,7 @@ public CheckpointErrorReportOperation(TaskLocation taskLocation, Throwable e) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService() .getJobMaster(taskLocation.getJobId()) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java index 40569c987c5..1e97bd4b461 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java @@ -76,7 +76,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java index b04477370d9..4cd83941c47 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java @@ -80,7 +80,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java index 62224aea47a..0f0a3b5bddb 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java @@ -46,7 +46,7 @@ public int getClassId() { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java index 7355811613a..9262cb3c7d5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java @@ -22,11 +22,11 @@ import org.apache.seatunnel.engine.server.checkpoint.CheckpointBarrier; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -37,7 +37,8 @@ @Getter @AllArgsConstructor @Slf4j -public class TaskAcknowledgeOperation extends Operation implements IdentifiedDataSerializable { +public class TaskAcknowledgeOperation extends TracingOperation + implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -59,6 +60,7 @@ public int getClassId() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeObject(taskLocation); out.writeObject(barrier); out.writeObject(states); @@ -66,13 +68,14 @@ protected void writeInternal(ObjectDataOutput out) throws IOException { @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); taskLocation = in.readObject(); barrier = in.readObject(); states = in.readObject(); } @Override - public void run() { + public void runInternal() { log.debug("TaskAcknowledgeOperation {}", taskLocation); ((SeaTunnelServer) getService()) .getCoordinatorService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java index 960787fc5fe..177b6f01394 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java @@ -23,12 +23,12 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import org.apache.seatunnel.engine.server.task.statemachine.SeaTunnelTaskState; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -38,7 +38,8 @@ @Getter @NoArgsConstructor @AllArgsConstructor -public class TaskReportStatusOperation extends Operation implements IdentifiedDataSerializable { +public class TaskReportStatusOperation extends TracingOperation + implements IdentifiedDataSerializable { private TaskLocation location; private SeaTunnelTaskState status; @@ -55,18 +56,20 @@ public int getClassId() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeObject(location); out.writeObject(status); } @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); location = in.readObject(TaskLocation.class); status = in.readObject(); } @Override - public void run() throws Exception { + public void runInternal() throws Exception { CoordinatorService coordinatorService = ((SeaTunnelServer) getService()).getCoordinatorService(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java index 66be8cc9f30..5bd4ee5db4d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java @@ -20,11 +20,11 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -36,7 +36,7 @@ @Getter @AllArgsConstructor @NoArgsConstructor -public class TriggerSchemaChangeAfterCheckpointOperation extends Operation +public class TriggerSchemaChangeAfterCheckpointOperation extends TracingOperation implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -53,16 +53,18 @@ public int getClassId() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); taskLocation = in.readObject(); } @Override - public void run() { + public void runInternal() { log.debug("call TriggerSchemaChangeAfterCheckpointOperation start {}", taskLocation); ((SeaTunnelServer) getService()) .getCoordinatorService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java index 54daedd8c62..ba73ce64266 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java @@ -20,11 +20,11 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -36,7 +36,7 @@ @Getter @AllArgsConstructor @NoArgsConstructor -public class TriggerSchemaChangeBeforeCheckpointOperation extends Operation +public class TriggerSchemaChangeBeforeCheckpointOperation extends TracingOperation implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -62,7 +62,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void run() { + public void runInternal() { log.debug("call TriggerSchemaChangeBeforeCheckpointOperation {}", taskLocation); ((SeaTunnelServer) getService()) .getCoordinatorService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java index 8f5cb3f7761..3755477f45a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java @@ -150,10 +150,10 @@ private Set generateExecutionEdges(Set logicalEdges) Collections.sort( sortedLogicalEdges, (o1, o2) -> { - if (o1.getInputVertexId() != o2.getInputVertexId()) { + if (!o1.getInputVertexId().equals(o2.getInputVertexId())) { return o1.getInputVertexId() > o2.getInputVertexId() ? 1 : -1; } - if (o1.getTargetVertexId() != o2.getTargetVertexId()) { + if (!o1.getTargetVertexId().equals(o2.getTargetVertexId())) { return o1.getTargetVertexId() > o2.getTargetVertexId() ? 1 : -1; } return 0; diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java index d2b184e850f..5a771c66e2f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java @@ -62,5 +62,6 @@ default MetricsContext getMetricsContext() { return null; } + @Override default void provideDynamicMetrics(MetricDescriptor tagger, MetricsCollectionContext context) {} } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java index 2d2a0d04167..eb76a599800 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java @@ -23,17 +23,17 @@ import org.apache.seatunnel.engine.server.resourcemanager.worker.WorkerProfile; import org.apache.seatunnel.engine.server.serializable.ResourceDataSerializerHook; import org.apache.seatunnel.engine.server.service.slot.WrongTargetSlotException; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import lombok.extern.slf4j.Slf4j; import java.io.IOException; @Slf4j -public class ReleaseSlotOperation extends Operation implements IdentifiedDataSerializable { +public class ReleaseSlotOperation extends TracingOperation implements IdentifiedDataSerializable { private long jobID; private SlotProfile slotProfile; @@ -47,7 +47,7 @@ public ReleaseSlotOperation(long jobID, SlotProfile slotProfile) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); try { server.getSlotService().releaseSlot(jobID, slotProfile); @@ -68,12 +68,14 @@ public Object getResponse() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeObject(slotProfile); out.writeLong(jobID); } @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); slotProfile = in.readObject(); jobID = in.readLong(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java index d7fa05822d3..e851172a648 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java @@ -21,15 +21,15 @@ import org.apache.seatunnel.engine.server.resourcemanager.resource.ResourceProfile; import org.apache.seatunnel.engine.server.serializable.ResourceDataSerializerHook; import org.apache.seatunnel.engine.server.service.slot.SlotAndWorkerProfile; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class RequestSlotOperation extends Operation implements IdentifiedDataSerializable { +public class RequestSlotOperation extends TracingOperation implements IdentifiedDataSerializable { private ResourceProfile resourceProfile; private long jobID; @@ -43,13 +43,14 @@ public RequestSlotOperation(long jobID, ResourceProfile resourceProfile) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); result = server.getSlotService().requestSlot(jobID, resourceProfile); } @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeObject(resourceProfile); out.writeLong(jobID); } @@ -61,6 +62,7 @@ public Object getResponse() { @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); resourceProfile = in.readObject(); jobID = in.readLong(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java index 0d0f8c8054b..f357a690da9 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java @@ -85,6 +85,7 @@ public void writeData(ObjectDataOutput out) throws IOException { out.writeObject(unassignedSlot); } out.writeBoolean(dynamicSlot); + out.writeObject(attributes); } @Override @@ -103,5 +104,6 @@ public void readData(ObjectDataInput in) throws IOException { unassignedSlots[i] = in.readObject(); } dynamicSlot = in.readBoolean(); + attributes = in.readObject(); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java index 72487737033..1d26a0b5fe1 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java @@ -50,7 +50,6 @@ public class RestConstant { @Deprecated public static final String RUNNING_JOB_URL = "/hazelcast/rest/maps/running-job"; public static final String JOB_INFO_URL = "/hazelcast/rest/maps/job-info"; public static final String FINISHED_JOBS_INFO = "/hazelcast/rest/maps/finished-jobs"; - public static final String SUBMIT_JOB_URL = "/hazelcast/rest/maps/submit-job"; public static final String ENCRYPT_CONFIG = "/hazelcast/rest/maps/encrypt-config"; // only for test use @@ -59,5 +58,8 @@ public class RestConstant { public static final String SYSTEM_MONITORING_INFORMATION = "/hazelcast/rest/maps/system-monitoring-information"; + public static final String SUBMIT_JOB_URL = "/hazelcast/rest/maps/submit-job"; + public static final String SUBMIT_JOBS_URL = "/hazelcast/rest/maps/submit-jobs"; public static final String STOP_JOB_URL = "/hazelcast/rest/maps/stop-job"; + public static final String STOP_JOBS_URL = "/hazelcast/rest/maps/stop-jobs"; } 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 fec77708b68..6827a2cfb03 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 @@ -474,7 +474,9 @@ private void processMetric( String tableName, JsonNode metricNode, Map[] tableMetricsMaps) { - if (metricNode == null) return; + if (metricNode == null) { + return; + } // Define index constant final int SOURCE_COUNT_IDX = 0, diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java index 150aae54c1e..9b8f0f8bca4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java @@ -44,17 +44,23 @@ import com.hazelcast.internal.ascii.rest.HttpCommandProcessor; import com.hazelcast.internal.ascii.rest.HttpPostCommand; import com.hazelcast.internal.json.Json; +import com.hazelcast.internal.json.JsonArray; import com.hazelcast.internal.json.JsonObject; import com.hazelcast.internal.serialization.Data; +import scala.Tuple2; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_400; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_500; import static org.apache.seatunnel.engine.server.rest.RestConstant.ENCRYPT_CONFIG; +import static org.apache.seatunnel.engine.server.rest.RestConstant.STOP_JOBS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.STOP_JOB_URL; +import static org.apache.seatunnel.engine.server.rest.RestConstant.SUBMIT_JOBS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.SUBMIT_JOB_URL; public class RestHttpPostCommandProcessor extends HttpCommandProcessor { @@ -78,10 +84,14 @@ protected RestHttpPostCommandProcessor( public void handle(HttpPostCommand httpPostCommand) { String uri = httpPostCommand.getURI(); try { - if (uri.startsWith(SUBMIT_JOB_URL)) { + if (uri.startsWith(SUBMIT_JOBS_URL)) { + handleSubmitJobs(httpPostCommand); + } else if (uri.startsWith(SUBMIT_JOB_URL)) { handleSubmitJob(httpPostCommand, uri); + } else if (uri.startsWith(STOP_JOBS_URL)) { + handleStopJobs(httpPostCommand); } else if (uri.startsWith(STOP_JOB_URL)) { - handleStopJob(httpPostCommand, uri); + handleStopJob(httpPostCommand); } else if (uri.startsWith(ENCRYPT_CONFIG)) { handleEncrypt(httpPostCommand); } else { @@ -93,7 +103,6 @@ public void handle(HttpPostCommand httpPostCommand) { logger.warning("An error occurred while handling request " + httpPostCommand, e); prepareResponse(SC_500, httpPostCommand, exceptionResponse(e)); } - this.textCommandService.sendResponse(httpPostCommand); } @@ -103,11 +112,41 @@ private SeaTunnelServer getSeaTunnelServer() { return (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); } + private void handleSubmitJobs(HttpPostCommand httpPostCommand) throws IllegalArgumentException { + List, Config>> configTuples = + RestUtil.buildConfigList(requestHandle(httpPostCommand), false); + + JsonArray jsonArray = + configTuples.stream() + .map( + tuple -> { + String urlParams = mapToUrlParams(tuple._1); + Map requestParams = new HashMap<>(); + RestUtil.buildRequestParams(requestParams, urlParams); + return submitJobInternal(tuple._2, requestParams); + }) + .collect(JsonArray::new, JsonArray::add, JsonArray::add); + + prepareResponse(httpPostCommand, jsonArray); + } + + private String mapToUrlParams(Map params) { + return params.entrySet().stream() + .map(entry -> entry.getKey() + "=" + entry.getValue()) + .collect(Collectors.joining("&", "?", "")); + } + private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri) throws IllegalArgumentException { Map requestParams = new HashMap<>(); RestUtil.buildRequestParams(requestParams, uri); Config config = RestUtil.buildConfig(requestHandle(httpPostCommand), false); + + JsonObject jsonObject = submitJobInternal(config, requestParams); + this.prepareResponse(httpPostCommand, jsonObject); + } + + private JsonObject submitJobInternal(Config config, Map requestParams) { ReadonlyConfig envOptions = ReadonlyConfig.fromConfig(config.getConfig("env")); String jobName = envOptions.get(EnvCommonOptions.JOB_NAME); @@ -146,15 +185,35 @@ private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri) submitJob(seaTunnelServer, jobImmutableInformation, jobConfig); } + return new JsonObject() + .add(RestConstant.JOB_ID, String.valueOf(jobId)) + .add(RestConstant.JOB_NAME, jobConfig.getName()); + } + + private void handleStopJobs(HttpPostCommand command) { + List jobList = JsonUtils.toList(requestHandle(command).toString(), Map.class); + JsonArray jsonResponse = new JsonArray(); + + jobList.forEach( + job -> { + handleStopJob(job); + jsonResponse.add( + new JsonObject() + .add(RestConstant.JOB_ID, (Long) job.get(RestConstant.JOB_ID))); + }); + + this.prepareResponse(command, jsonResponse); + } + + private void handleStopJob(HttpPostCommand httpPostCommand) { + Map map = JsonUtils.toMap(requestHandle(httpPostCommand)); + handleStopJob(map); this.prepareResponse( httpPostCommand, - new JsonObject() - .add(RestConstant.JOB_ID, String.valueOf(jobId)) - .add(RestConstant.JOB_NAME, jobConfig.getName())); + new JsonObject().add(RestConstant.JOB_ID, map.get(RestConstant.JOB_ID).toString())); } - private void handleStopJob(HttpPostCommand httpPostCommand, String uri) { - Map map = JsonUtils.toMap(requestHandle(httpPostCommand)); + private void handleStopJob(Map map) { boolean isStopWithSavePoint = false; if (map.get(RestConstant.JOB_ID) == null) { throw new IllegalArgumentException("jobId cannot be empty."); @@ -186,10 +245,7 @@ private void handleStopJob(HttpPostCommand httpPostCommand, String uri) { coordinatorService.cancelJob(jobId); } } - - this.prepareResponse( - httpPostCommand, - new JsonObject().add(RestConstant.JOB_ID, map.get(RestConstant.JOB_ID).toString())); + logger.info("Stop job with jobId: " + jobId); } private void handleEncrypt(HttpPostCommand httpPostCommand) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java index 7c0ae38bfd8..3a5e27171c1 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java @@ -149,6 +149,7 @@ public synchronized SlotAndWorkerProfile requestSlot( return new SlotAndWorkerProfile(getWorkerProfile(), profile); } + @Override public SlotContext getSlotContext(SlotProfile slotProfile) { if (!contexts.containsKey(slotProfile.getSlotID())) { throw new WrongTargetSlotException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java index d1fc333ade2..8004068ce68 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.api.source.SourceEvent; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; +import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; import org.apache.seatunnel.engine.core.dag.actions.SourceAction; import org.apache.seatunnel.engine.core.job.ConnectorJarIdentifier; import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; @@ -121,6 +123,7 @@ public void close() throws IOException { super.close(); if (enumerator != null) { enumerator.close(); + enumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); } progress.done(); } @@ -309,6 +312,7 @@ private void stateProcess() throws Exception { if (startCalled && readerRegisterComplete) { currState = STARTING; enumerator.open(); + enumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); } else { Thread.sleep(100); } 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 cacaa75aaef..3234560fe4b 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 @@ -24,6 +24,7 @@ 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.event.WriterCloseEvent; import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; @@ -69,6 +70,7 @@ public class SinkFlowLifeCycle sinkAction; private SinkWriter writer; + private SinkWriter.Context writerContext; private transient Optional> commitInfoSerializer; private transient Optional> writerStateSerializer; @@ -150,6 +152,7 @@ private Address getCommitterTaskAddress() throws ExecutionException, Interrupted public void close() throws IOException { super.close(); writer.close(); + writerContext.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); @@ -283,19 +286,11 @@ public void restoreState(List actionStateList) throws Except .deserialize(bytes))) .collect(Collectors.toList()); } + this.writerContext = new SinkWriterContext(indexID, metricsContext, eventListener); if (states.isEmpty()) { - this.writer = - sinkAction - .getSink() - .createWriter( - new SinkWriterContext(indexID, metricsContext, eventListener)); + this.writer = sinkAction.getSink().createWriter(writerContext); } else { - this.writer = - sinkAction - .getSink() - .restoreWriter( - new SinkWriterContext(indexID, metricsContext, eventListener), - states); + this.writer = sinkAction.getSink().restoreWriter(writerContext, states); } if (this.writer instanceof SupportResourceShare) { resourceManager = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java index ca137b3e069..6c596da0c33 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java @@ -23,6 +23,8 @@ import org.apache.seatunnel.api.source.SourceEvent; import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.event.ReaderCloseEvent; +import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.engine.core.checkpoint.CheckpointType; import org.apache.seatunnel.engine.core.checkpoint.InternalCheckpointListener; @@ -83,6 +85,7 @@ public class SourceFlowLifeCycle extends ActionFl private final MetricsContext metricsContext; private final EventListener eventListener; + private SourceReader.Context context; private final AtomicReference schemaChangePhase = new AtomicReference<>(); @@ -111,21 +114,20 @@ public void setCollector(SeaTunnelSourceCollector collector) { @Override public void init() throws Exception { this.splitSerializer = sourceAction.getSource().getSplitSerializer(); - this.reader = - sourceAction - .getSource() - .createReader( - new SourceReaderContext( - indexID, - sourceAction.getSource().getBoundedness(), - this, - metricsContext, - eventListener)); + this.context = + new SourceReaderContext( + indexID, + sourceAction.getSource().getBoundedness(), + this, + metricsContext, + eventListener); + this.reader = sourceAction.getSource().createReader(context); this.enumeratorTaskAddress = getEnumeratorTaskAddress(); } @Override public void open() throws Exception { + context.getEventListener().onEvent(new ReaderOpenEvent()); reader.open(); register(); } @@ -140,6 +142,7 @@ private Address getEnumeratorTaskAddress() throws ExecutionException, Interrupte @Override public void close() throws IOException { + context.getEventListener().onEvent(new ReaderCloseEvent()); reader.close(); super.close(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java index a6c5f7b087d..f2b01bf0404 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java @@ -24,7 +24,6 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; @@ -34,7 +33,7 @@ * is cancelled, the {@link org.apache.seatunnel.engine.server.TaskExecutionService} will notified * JobMaster */ -public class CancelTaskOperation extends Operation implements IdentifiedDataSerializable { +public class CancelTaskOperation extends TracingOperation implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; public CancelTaskOperation() {} @@ -54,7 +53,7 @@ public int getClassId() { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); server.getTaskExecutionService().cancelTaskGroup(taskGroupLocation); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java index d4e158abdbe..e48e1fcac98 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java @@ -25,11 +25,10 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CheckTaskGroupIsExecutingOperation extends Operation +public class CheckTaskGroupIsExecutingOperation extends TracingOperation implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; @@ -42,7 +41,7 @@ public CheckTaskGroupIsExecutingOperation(TaskGroupLocation taskGroupLocation) { } @Override - public void run() { + public void runInternal() { SeaTunnelServer server = getService(); try { response = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java index 42d83131382..19099f563fa 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java @@ -24,11 +24,10 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CleanTaskGroupContextOperation extends Operation +public class CleanTaskGroupContextOperation extends TracingOperation implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; @@ -40,7 +39,7 @@ public CleanTaskGroupContextOperation(TaskGroupLocation taskGroupLocation) { } @Override - public void run() { + public void runInternal() { // remove TaskGroupContext for TaskExecutionService SeaTunnelServer service = getService(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java index 296d5f0c9bc..e684e188c6e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java @@ -27,12 +27,11 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import lombok.NonNull; import java.io.IOException; -public class DeployTaskOperation extends Operation implements IdentifiedDataSerializable { +public class DeployTaskOperation extends TracingOperation implements IdentifiedDataSerializable { private Data taskImmutableInformation; private SlotProfile slotProfile; @@ -47,7 +46,7 @@ public DeployTaskOperation( } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); state = server.getSlotService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java index ecb94e85c0f..20415e86635 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java @@ -27,12 +27,12 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; import java.util.Objects; -public class GetTaskGroupAddressOperation extends Operation implements IdentifiedDataSerializable { +public class GetTaskGroupAddressOperation extends TracingOperation + implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -45,7 +45,7 @@ public GetTaskGroupAddressOperation(TaskLocation taskLocation) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); response = RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java index 26f947bca8a..52fc15418b5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java @@ -25,11 +25,11 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class NotifyTaskStatusOperation extends Operation implements IdentifiedDataSerializable { +public class NotifyTaskStatusOperation extends TracingOperation + implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; private TaskExecutionState taskExecutionState; @@ -68,7 +68,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService().updateTaskExecutionState(taskExecutionState); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java index 729d14dc072..986fe3131e7 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java @@ -23,7 +23,6 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -33,7 +32,7 @@ @Getter @NoArgsConstructor @AllArgsConstructor -public abstract class TaskOperation extends Operation implements IdentifiedDataSerializable { +public abstract class TaskOperation extends TracingOperation implements IdentifiedDataSerializable { protected TaskLocation taskLocation; @@ -44,11 +43,13 @@ public String getServiceName() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); taskLocation = in.readObject(TaskLocation.class); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java new file mode 100644 index 00000000000..08d859a4b14 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java @@ -0,0 +1,63 @@ +/* + * 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.server.task.operation; + +import org.apache.seatunnel.api.tracing.MDCContext; + +import com.hazelcast.nio.ObjectDataInput; +import com.hazelcast.nio.ObjectDataOutput; +import com.hazelcast.spi.impl.operationservice.Operation; + +import java.io.IOException; + +public abstract class TracingOperation extends Operation { + private MDCContext context; + + public TracingOperation() { + this(MDCContext.current()); + } + + public TracingOperation(MDCContext context) { + this.context = context; + } + + @Override + public final void run() throws Exception { + try { + context.put(); + + runInternal(); + } finally { + context.clear(); + } + } + + public abstract void runInternal() throws Exception; + + @Override + protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); + out.writeString(context.toString()); + } + + @Override + protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); + context = MDCContext.valueOf(in.readString()); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java index 89d3b3805e5..e105f8bde04 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java @@ -69,7 +69,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java index f12361bdfb1..e8eaaf1fd1a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java @@ -24,15 +24,15 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSeaTunnelTask; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CloseRequestOperation extends Operation implements IdentifiedDataSerializable { +public class CloseRequestOperation extends TracingOperation implements IdentifiedDataSerializable { private TaskLocation readerLocation; @@ -43,7 +43,7 @@ public CloseRequestOperation(TaskLocation readerLocation) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java index 5ed6f81a7aa..db3ae487a31 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java @@ -69,7 +69,7 @@ public int getClassId() { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { TaskExecutionService taskExecutionService = ((SeaTunnelServer) getService()).getTaskExecutionService(); SinkAggregatedCommitterTask committerTask = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java index 33c781309e9..32ba9145e74 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SinkAggregatedCommitterTask; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.cluster.Address; import com.hazelcast.logging.ILogger; @@ -31,11 +32,10 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class SinkRegisterOperation extends Operation implements IdentifiedDataSerializable { +public class SinkRegisterOperation extends TracingOperation implements IdentifiedDataSerializable { private static final ILogger LOGGER = Logger.getLogger(SinkRegisterOperation.class); private TaskLocation writerTaskID; @@ -49,7 +49,7 @@ public SinkRegisterOperation(TaskLocation writerTaskID, TaskLocation committerTa } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); Address readerAddress = getCallerAddress(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java index b21111e18fd..b080d2f1bad 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java @@ -25,17 +25,17 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSeaTunnelTask; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; import java.util.ArrayList; import java.util.List; -public class AssignSplitOperation extends Operation +public class AssignSplitOperation extends TracingOperation implements IdentifiedDataSerializable { private List splits; @@ -49,7 +49,7 @@ public AssignSplitOperation(TaskLocation taskID, List splits) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { @@ -84,6 +84,7 @@ public void run() throws Exception { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeInt(splits.size()); for (byte[] split : splits) { out.writeByteArray(split); @@ -93,6 +94,7 @@ protected void writeInternal(ObjectDataOutput out) throws IOException { @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); int splitCount = in.readInt(); splits = new ArrayList<>(splitCount); for (int i = 0; i < splitCount; i++) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java index abedf1a4993..b6c722203a5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java @@ -20,15 +20,16 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CloseIdleReaderOperation extends Operation implements IdentifiedDataSerializable { +public class CloseIdleReaderOperation extends TracingOperation + implements IdentifiedDataSerializable { private long jobId; private TaskLocation taskLocation; @@ -40,7 +41,7 @@ public CloseIdleReaderOperation(long jobId, TaskLocation taskLocation) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService() .getJobMaster(jobId) @@ -50,12 +51,14 @@ public void run() throws Exception { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeLong(jobId); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); jobId = in.readLong(); taskLocation = in.readObject(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java index 1309b92b451..d2391017d21 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java @@ -20,15 +20,16 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class LastCheckpointNotifyOperation extends Operation implements IdentifiedDataSerializable { +public class LastCheckpointNotifyOperation extends TracingOperation + implements IdentifiedDataSerializable { private long jobId; private TaskLocation taskLocation; @@ -41,7 +42,7 @@ public LastCheckpointNotifyOperation(long jobId, TaskLocation taskLocation) { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService() .getJobMaster(jobId) @@ -51,12 +52,14 @@ public void run() throws Exception { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); out.writeLong(jobId); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); jobId = in.readLong(); taskLocation = in.readObject(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java index 5750046d0c0..1a4b7469db5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java @@ -24,15 +24,15 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSplitEnumeratorTask; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class RequestSplitOperation extends Operation implements IdentifiedDataSerializable { +public class RequestSplitOperation extends TracingOperation implements IdentifiedDataSerializable { private TaskLocation enumeratorTaskID; @@ -46,7 +46,7 @@ public RequestSplitOperation(TaskLocation taskID, TaskLocation enumeratorTaskID) } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java index e3f6c908159..a6ee60f6f12 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java @@ -81,7 +81,7 @@ public int getClassId() { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); TaskExecutionService taskExecutionService = server.getTaskExecutionService(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java index 60b67c0c83b..35667bf9b57 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java @@ -24,15 +24,16 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSplitEnumeratorTask; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class SourceNoMoreElementOperation extends Operation implements IdentifiedDataSerializable { +public class SourceNoMoreElementOperation extends TracingOperation + implements IdentifiedDataSerializable { private TaskLocation currentTaskID; private TaskLocation enumeratorTaskID; @@ -45,7 +46,7 @@ public SourceNoMoreElementOperation(TaskLocation currentTaskID, TaskLocation enu } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java index 8c3f4e0133d..56a376dc5dc 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java @@ -45,7 +45,7 @@ public int getClassId() { } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java index faf44b4c40d..12685ae8b44 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java @@ -25,12 +25,12 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSplitEnumeratorTask; +import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.cluster.Address; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; -import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; @@ -38,7 +38,8 @@ * For {@link org.apache.seatunnel.api.source.SourceReader} to register with the {@link * org.apache.seatunnel.api.source.SourceSplitEnumerator} */ -public class SourceRegisterOperation extends Operation implements IdentifiedDataSerializable { +public class SourceRegisterOperation extends TracingOperation + implements IdentifiedDataSerializable { private TaskLocation readerTaskID; private TaskLocation enumeratorTaskID; @@ -51,7 +52,7 @@ public SourceRegisterOperation(TaskLocation readerTaskID, TaskLocation enumerato } @Override - public void run() throws Exception { + public void runInternal() throws Exception { SeaTunnelServer server = getService(); Address readerAddress = getCallerAddress(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java index 9aaa8cd5951..b114073aadb 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java @@ -26,11 +26,17 @@ import org.apache.seatunnel.engine.server.rest.RestConstant; import com.hazelcast.internal.util.StringUtil; +import scala.Tuple2; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URLDecoder; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.seatunnel.engine.common.Constant.REST_SUBMIT_JOBS_PARAMS; public class RestUtil { private RestUtil() {} @@ -69,4 +75,19 @@ public static Config buildConfig(JsonNode jsonNode, boolean isEncrypt) { Map objectMap = JsonUtils.toMap(jsonNode); return ConfigBuilder.of(objectMap, isEncrypt, true); } + + public static List, Config>> buildConfigList( + JsonNode jsonNode, boolean isEncrypt) { + return StreamSupport.stream(jsonNode.spliterator(), false) + .filter(JsonNode::isObject) + .map( + node -> { + Map nodeMap = JsonUtils.toMap(node); + Map params = + (Map) nodeMap.remove(REST_SUBMIT_JOBS_PARAMS); + Config config = ConfigBuilder.of(nodeMap, isEncrypt, true); + return new Tuple2<>(params, config); + }) + .collect(Collectors.toList()); + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java index b3df40f6aaa..53cc68b9423 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java @@ -35,19 +35,32 @@ public class FixSlotResourceTest extends AbstractSeaTunnelServerTest { + private final int totalSlots = 3; + @Override public SeaTunnelConfig loadSeaTunnelConfig() { SeaTunnelConfig seaTunnelConfig = super.loadSeaTunnelConfig(); SlotServiceConfig slotServiceConfig = seaTunnelConfig.getEngineConfig().getSlotServiceConfig(); slotServiceConfig.setDynamicSlot(false); - slotServiceConfig.setSlotNum(3); + slotServiceConfig.setSlotNum(totalSlots); seaTunnelConfig.getEngineConfig().setSlotServiceConfig(slotServiceConfig); return seaTunnelConfig; } @Test public void testEnoughResource() throws ExecutionException, InterruptedException { + // wait all slot ready + await().atMost(20000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertEquals( + totalSlots, + server.getCoordinatorService() + .getResourceManager() + .getUnassignedSlots(null) + .size()); + }); long jobId = System.currentTimeMillis(); List resourceProfiles = new ArrayList<>(); resourceProfiles.add(new ResourceProfile()); @@ -83,7 +96,7 @@ public void testNotEnoughResource() throws ExecutionException, InterruptedExcept .untilAsserted( () -> { Assertions.assertEquals( - 3, + totalSlots, server.getCoordinatorService() .getResourceManager() .getUnassignedSlots(null) diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties b/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties index 153d4d97c68..d40ba6afd72 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties +++ b/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties @@ -25,7 +25,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -35,7 +35,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java index 213f458b9ee..eec10aeabe1 100644 --- a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java +++ b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java @@ -58,7 +58,6 @@ * fs.oss.accessKeySecret = "your script key" * fs.oss.endpoint = "such as: oss-cn-hangzhou.aliyuncs.com" * oss.bucket= "oss://your bucket" - * fs.oss.credentials.provider = "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider" * */ @AutoService(CheckpointStorageFactory.class) diff --git a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java index 3d7299c266f..9e6ef2409b9 100644 --- a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java +++ b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java @@ -40,9 +40,6 @@ public static void setup() throws CheckpointStorageException { config.put("fs.oss.accessKeySecret", "your access key secret"); config.put("fs.oss.endpoint", "oss-cn-hangzhou.aliyuncs.com"); config.put("oss.bucket", "oss://seatunnel-test/"); - config.put( - "fs.oss.credentials.provider", - "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider"); STORAGE = new HdfsStorage(config); initStorageData(); } diff --git a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties index 2dc1b8ca501..0bf06d2b773 100644 --- a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties +++ b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties @@ -25,7 +25,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -35,7 +35,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java b/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java index dd92b65e762..c642670016c 100644 --- a/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java +++ b/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java @@ -68,9 +68,6 @@ public class IMapFileOSSStorageTest { CONF.set("fs.oss.accessKeyId", OSS_ACCESS_KEY_ID); CONF.set("fs.oss.accessKeySecret", OSS_ACCESS_KEY_SECRET); CONF.set("fs.oss.impl", "org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem"); - CONF.set( - "fs.oss.credentials.provider", - "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider"); STORAGE = new IMapFileStorage(); Map properties = new HashMap<>(); @@ -81,9 +78,6 @@ public class IMapFileOSSStorageTest { properties.put("fs.oss.accessKeyId", OSS_ACCESS_KEY_ID); properties.put("fs.oss.accessKeySecret", OSS_ACCESS_KEY_SECRET); properties.put("fs.oss.impl", "org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem"); - properties.put( - "fs.oss.credentials.provider", - "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider"); properties.put(FileConstants.FileInitProperties.BUSINESS_KEY, BUSINESS); properties.put(FileConstants.FileInitProperties.NAMESPACE_KEY, NAMESPACE); properties.put(FileConstants.FileInitProperties.CLUSTER_NAME, CLUSTER_NAME); diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties index db3dcb88e90..77985ea23a9 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml index ef801bdb9c0..99c75d324a8 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml @@ -112,12 +112,6 @@ ${flink.1.15.3.version} - - org.apache.flink - flink-runtime-web - ${flink.1.15.3.version} - - com.squareup.okhttp3 mockwebserver diff --git a/seatunnel-formats/pom.xml b/seatunnel-formats/pom.xml index a330e9b0e05..c66c6c8eafa 100644 --- a/seatunnel-formats/pom.xml +++ b/seatunnel-formats/pom.xml @@ -32,6 +32,7 @@ seatunnel-format-compatible-debezium-json seatunnel-format-compatible-connect-json seatunnel-format-avro + seatunnel-format-protobuf diff --git a/seatunnel-formats/seatunnel-format-protobuf/pom.xml b/seatunnel-formats/seatunnel-format-protobuf/pom.xml new file mode 100644 index 00000000000..8f8f7030f64 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/pom.xml @@ -0,0 +1,59 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-formats + ${revision} + + + seatunnel-format-protobuf + SeaTunnel : Formats : Protobuf + + + 8 + 8 + UTF-8 + 3.25.3 + 3.11.4 + + + + + org.apache.seatunnel + seatunnel-api + ${project.version} + provided + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + com.google.protobuf + protobuf-java-util + ${protobuf.version} + + + com.github.os72 + protoc-jar + ${protoc.jar.version} + + + diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java new file mode 100644 index 00000000000..17a14bc84f7 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java @@ -0,0 +1,107 @@ +/* + * 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.format.protobuf; + +import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.format.protobuf.exception.ProtobufFormatErrorCode; +import org.apache.seatunnel.format.protobuf.exception.SeaTunnelProtobufFormatException; + +import com.github.os72.protocjar.Protoc; +import com.google.protobuf.DescriptorProtos; +import com.google.protobuf.Descriptors; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.List; + +public class CompileDescriptor { + + public static Descriptors.Descriptor compileDescriptorTempFile( + String protoContent, String messageName) + throws IOException, InterruptedException, Descriptors.DescriptorValidationException { + // Because Protobuf can only be dynamically parsed through the descriptor file, the file + // needs to be compiled and generated. The following method is used here to solve the + // problem: generate a temporary directory and compile .proto into a descriptor temporary + // file. The temporary file and directory are deleted after the JVM runs. + File tmpDir = createTempDirectory(); + File protoFile = createProtoFile(tmpDir, protoContent); + String targetDescPath = compileProtoToDescriptor(tmpDir, protoFile); + + try (FileInputStream fis = new FileInputStream(targetDescPath)) { + DescriptorProtos.FileDescriptorSet descriptorSet = + DescriptorProtos.FileDescriptorSet.parseFrom(fis); + Descriptors.FileDescriptor[] descriptorsArray = buildFileDescriptors(descriptorSet); + return descriptorsArray[0].findMessageTypeByName(messageName); + } finally { + tmpDir.delete(); + protoFile.delete(); + new File(targetDescPath).delete(); + } + } + + private static File createTempDirectory() throws IOException { + File tmpDir = File.createTempFile("tmp_protobuf_", "_proto"); + tmpDir.delete(); + tmpDir.mkdirs(); + tmpDir.deleteOnExit(); + return tmpDir; + } + + private static File createProtoFile(File tmpDir, String protoContent) throws IOException { + File protoFile = new File(tmpDir, ".proto"); + protoFile.deleteOnExit(); + FileUtils.writeStringToFile(protoFile.getPath(), protoContent); + return protoFile; + } + + private static String compileProtoToDescriptor(File tmpDir, File protoFile) + throws IOException, InterruptedException { + String targetDesc = tmpDir + "/.desc"; + new File(targetDesc).deleteOnExit(); + + int exitCode = + Protoc.runProtoc( + new String[] { + "--proto_path=" + protoFile.getParent(), + "--descriptor_set_out=" + targetDesc, + protoFile.getPath() + }); + + if (exitCode != 0) { + throw new SeaTunnelProtobufFormatException( + ProtobufFormatErrorCode.DESCRIPTOR_CONVERT_FAILED, + "Protoc compile error, exit code: " + exitCode); + } + return targetDesc; + } + + private static Descriptors.FileDescriptor[] buildFileDescriptors( + DescriptorProtos.FileDescriptorSet descriptorSet) + throws Descriptors.DescriptorValidationException { + List fileDescriptors = descriptorSet.getFileList(); + Descriptors.FileDescriptor[] descriptorsArray = + new Descriptors.FileDescriptor[fileDescriptors.size()]; + for (int i = 0; i < fileDescriptors.size(); i++) { + descriptorsArray[i] = + Descriptors.FileDescriptor.buildFrom( + fileDescriptors.get(i), new Descriptors.FileDescriptor[] {}); + } + return descriptorsArray; + } +} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java new file mode 100644 index 00000000000..34e79bc7ff1 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java @@ -0,0 +1,68 @@ +/* + * 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.format.protobuf; + +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +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 com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; + +import java.io.IOException; +import java.util.Optional; + +public class ProtobufDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = -7907358485475741366L; + + private final SeaTunnelRowType rowType; + private final ProtobufToRowConverter converter; + private final CatalogTable catalogTable; + private final String protoContent; + private final String messageName; + + public ProtobufDeserializationSchema(CatalogTable catalogTable) { + this.catalogTable = catalogTable; + this.rowType = catalogTable.getSeaTunnelRowType(); + this.messageName = catalogTable.getOptions().get("protobuf_message_name"); + this.protoContent = catalogTable.getOptions().get("protobuf_schema"); + this.converter = new ProtobufToRowConverter(protoContent, messageName); + } + + @Override + public SeaTunnelRow deserialize(byte[] message) throws IOException { + Descriptors.Descriptor descriptor = this.converter.getDescriptor(); + DynamicMessage dynamicMessage = DynamicMessage.parseFrom(descriptor, message); + SeaTunnelRow seaTunnelRow = this.converter.converter(descriptor, dynamicMessage, rowType); + Optional tablePath = + Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); + if (tablePath.isPresent()) { + seaTunnelRow.setTableId(tablePath.toString()); + } + return seaTunnelRow; + } + + @Override + public SeaTunnelDataType getProducedType() { + return this.rowType; + } +} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.java new file mode 100644 index 00000000000..b733c542dc2 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.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.format.protobuf; + +import org.apache.seatunnel.api.serialization.SerializationSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import com.google.protobuf.Descriptors; + +import java.io.IOException; + +public class ProtobufSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = 4438784443025715370L; + + private final RowToProtobufConverter converter; + + public ProtobufSerializationSchema( + SeaTunnelRowType rowType, String protobufMessageName, String protobufSchema) { + try { + Descriptors.Descriptor descriptor = + CompileDescriptor.compileDescriptorTempFile( + protobufSchema, protobufMessageName); + this.converter = new RowToProtobufConverter(rowType, descriptor); + } catch (IOException | InterruptedException | Descriptors.DescriptorValidationException e) { + throw new RuntimeException(e); + } + } + + @Override + public byte[] serialize(SeaTunnelRow element) { + return converter.convertRowToGenericRecord(element); + } +} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java new file mode 100644 index 00000000000..3f7d01b3ff6 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.format.protobuf; + +import org.apache.seatunnel.api.table.type.ArrayType; +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 com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; + +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.Array; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class ProtobufToRowConverter implements Serializable { + private static final long serialVersionUID = 8177020083886379563L; + + private Descriptors.Descriptor descriptor = null; + private String protoContent; + private String messageName; + + public ProtobufToRowConverter(String protoContent, String messageName) { + this.protoContent = protoContent; + this.messageName = messageName; + } + + public Descriptors.Descriptor getDescriptor() { + if (descriptor == null) { + try { + descriptor = createDescriptor(); + } catch (IOException + | Descriptors.DescriptorValidationException + | InterruptedException e) { + throw new RuntimeException(e); + } + } + return descriptor; + } + + private Descriptors.Descriptor createDescriptor() + throws IOException, InterruptedException, Descriptors.DescriptorValidationException { + + return CompileDescriptor.compileDescriptorTempFile(protoContent, messageName); + } + + public SeaTunnelRow converter( + Descriptors.Descriptor descriptor, + DynamicMessage dynamicMessage, + SeaTunnelRowType rowType) { + String[] fieldNames = rowType.getFieldNames(); + Object[] values = new Object[fieldNames.length]; + for (int i = 0; i < fieldNames.length; i++) { + Descriptors.FieldDescriptor fieldByName = descriptor.findFieldByName(fieldNames[i]); + if (fieldByName == null && descriptor.findNestedTypeByName(fieldNames[i]) == null) { + values[i] = null; + } else { + values[i] = + convertField( + descriptor, + dynamicMessage, + rowType.getFieldType(i), + fieldByName == null ? null : dynamicMessage.getField(fieldByName), + fieldNames[i]); + } + } + return new SeaTunnelRow(values); + } + + private Object convertField( + Descriptors.Descriptor descriptor, + DynamicMessage dynamicMessage, + SeaTunnelDataType dataType, + Object val, + String fieldName) { + switch (dataType.getSqlType()) { + case STRING: + return val.toString(); + case BOOLEAN: + case INT: + case BIGINT: + case FLOAT: + case DOUBLE: + case NULL: + case DATE: + case DECIMAL: + case TIMESTAMP: + return val; + case BYTES: + return ((ByteString) val).toByteArray(); + case SMALLINT: + return ((Integer) val).shortValue(); + case TINYINT: + Class typeClass = dataType.getTypeClass(); + if (typeClass == Byte.class) { + Integer integer = (Integer) val; + return integer.byteValue(); + } + return val; + case MAP: + MapType mapType = (MapType) dataType; + Map res = + ((List) val) + .stream() + .collect( + Collectors.toMap( + dm -> + convertField( + descriptor, + dm, + mapType.getKeyType(), + getFieldValue(dm, "key"), + null), + dm -> + convertField( + descriptor, + dm, + mapType.getValueType(), + getFieldValue(dm, "value"), + null))); + + return res; + case ROW: + Descriptors.Descriptor nestedTypeByName = + descriptor.findNestedTypeByName(fieldName); + DynamicMessage s = + (DynamicMessage) + dynamicMessage.getField( + descriptor.findFieldByName(fieldName.toLowerCase())); + return converter(nestedTypeByName, s, (SeaTunnelRowType) dataType); + case ARRAY: + SeaTunnelDataType basicType = ((ArrayType) dataType).getElementType(); + List list = (List) val; + return convertArray(list, basicType); + default: + String errorMsg = + String.format( + "SeaTunnel avro format is not supported for this data type [%s]", + dataType.getSqlType()); + throw new RuntimeException(errorMsg); + } + } + + private Object getFieldValue(DynamicMessage dm, String fieldName) { + return dm.getAllFields().entrySet().stream() + .filter(entry -> entry.getKey().getName().equals(fieldName)) + .map(Map.Entry::getValue) + .findFirst() + .orElse(null); + } + + protected Object convertArray(List val, SeaTunnelDataType dataType) { + if (val == null) { + return null; + } + int length = val.size(); + Object instance = Array.newInstance(dataType.getTypeClass(), length); + for (int i = 0; i < val.size(); i++) { + Array.set(instance, i, convertField(null, null, dataType, val.get(i), null)); + } + return instance; + } +} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java new file mode 100644 index 00000000000..a6a75aaee3f --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java @@ -0,0 +1,152 @@ +/* + * 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.format.protobuf; + +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.format.protobuf.exception.ProtobufFormatErrorCode; +import org.apache.seatunnel.format.protobuf.exception.SeaTunnelProtobufFormatException; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Map; + +public class RowToProtobufConverter implements Serializable { + + private static final long serialVersionUID = -576124379280229724L; + private final Descriptors.Descriptor descriptor; + private final SeaTunnelRowType rowType; + + public RowToProtobufConverter(SeaTunnelRowType rowType, Descriptors.Descriptor descriptor) { + this.rowType = rowType; + this.descriptor = descriptor; + } + + public byte[] convertRowToGenericRecord(SeaTunnelRow element) { + DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor); + String[] fieldNames = rowType.getFieldNames(); + + for (int i = 0; i < fieldNames.length; i++) { + String fieldName = rowType.getFieldName(i); + Object value = element.getField(i); + Object resolvedValue = + resolveObject(fieldName, value, rowType.getFieldType(i), builder); + if (resolvedValue != null) { + if (resolvedValue instanceof byte[]) { + resolvedValue = ByteString.copyFrom((byte[]) resolvedValue); + } + builder.setField( + descriptor.findFieldByName(fieldName.toLowerCase()), resolvedValue); + } + } + + return builder.build().toByteArray(); + } + + private Object resolveObject( + String fieldName, + Object data, + SeaTunnelDataType seaTunnelDataType, + DynamicMessage.Builder builder) { + if (data == null) { + return null; + } + + switch (seaTunnelDataType.getSqlType()) { + case STRING: + case SMALLINT: + case INT: + case BIGINT: + case FLOAT: + case DOUBLE: + case BOOLEAN: + case DECIMAL: + case DATE: + case TIMESTAMP: + case BYTES: + return data; + case TINYINT: + if (data instanceof Byte) { + return Byte.toUnsignedInt((Byte) data); + } + return data; + case MAP: + return handleMapType(fieldName, data, seaTunnelDataType, builder); + case ARRAY: + return Arrays.asList((Object[]) data); + case ROW: + return handleRowType(fieldName, data, seaTunnelDataType); + default: + throw new SeaTunnelProtobufFormatException( + ProtobufFormatErrorCode.UNSUPPORTED_DATA_TYPE, + String.format( + "SeaTunnel protobuf format is not supported for this data type [%s]", + seaTunnelDataType.getSqlType())); + } + } + + private Object handleMapType( + String fieldName, + Object data, + SeaTunnelDataType seaTunnelDataType, + DynamicMessage.Builder builder) { + Descriptors.Descriptor mapEntryDescriptor = + descriptor.findFieldByName(fieldName).getMessageType(); + + if (data instanceof Map) { + Map mapData = (Map) data; + mapData.forEach( + (key, value) -> { + DynamicMessage mapEntry = + DynamicMessage.newBuilder(mapEntryDescriptor) + .setField(mapEntryDescriptor.findFieldByName("key"), key) + .setField( + mapEntryDescriptor.findFieldByName("value"), value) + .build(); + builder.addRepeatedField(descriptor.findFieldByName(fieldName), mapEntry); + }); + } + + return null; + } + + private Object handleRowType( + String fieldName, Object data, SeaTunnelDataType seaTunnelDataType) { + SeaTunnelRow seaTunnelRow = (SeaTunnelRow) data; + SeaTunnelDataType[] fieldTypes = ((SeaTunnelRowType) seaTunnelDataType).getFieldTypes(); + String[] fieldNames = ((SeaTunnelRowType) seaTunnelDataType).getFieldNames(); + Descriptors.Descriptor nestedTypeDescriptor = descriptor.findNestedTypeByName(fieldName); + DynamicMessage.Builder nestedBuilder = DynamicMessage.newBuilder(nestedTypeDescriptor); + + for (int i = 0; i < fieldNames.length; i++) { + Object resolvedValue = + resolveObject( + fieldNames[i], seaTunnelRow.getField(i), fieldTypes[i], nestedBuilder); + nestedBuilder.setField( + nestedTypeDescriptor.findFieldByName(fieldNames[i]), resolvedValue); + } + + return nestedBuilder.build(); + } +} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java new file mode 100644 index 00000000000..e43125a41a1 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java @@ -0,0 +1,43 @@ +/* + * 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.format.protobuf.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; + +public enum ProtobufFormatErrorCode implements SeaTunnelErrorCode { + DESCRIPTOR_CONVERT_FAILED("PROTOBUF-01", "Protobuf descriptor conversion failed."), + UNSUPPORTED_DATA_TYPE("PROTOBUF-02", "Unsupported data type."); + + private final String code; + private final String description; + + ProtobufFormatErrorCode(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-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.java new file mode 100644 index 00000000000..30f6dfd0312 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.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.format.protobuf.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class SeaTunnelProtobufFormatException extends SeaTunnelRuntimeException { + + public SeaTunnelProtobufFormatException( + SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } +} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java b/seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java new file mode 100644 index 00000000000..e3d6dd35a2d --- /dev/null +++ b/seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.format.protobuf; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +class ProtobufConverterTest { + + private SeaTunnelRow buildSeaTunnelRow() { + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(10); + + Map attributesMap = new HashMap<>(); + attributesMap.put("k1", 0.1F); + attributesMap.put("k2", 2.3F); + + String[] phoneNumbers = {"1", "2"}; + byte[] byteVal = {1, 2, 3}; + + SeaTunnelRow address = new SeaTunnelRow(3); + address.setField(0, "city_value"); + address.setField(1, "state_value"); + address.setField(2, "street_value"); + + seaTunnelRow.setField(0, 123); + seaTunnelRow.setField(1, 123123123123L); + seaTunnelRow.setField(2, 0.123f); + seaTunnelRow.setField(3, 0.123d); + seaTunnelRow.setField(4, false); + seaTunnelRow.setField(5, "test data"); + seaTunnelRow.setField(6, byteVal); + seaTunnelRow.setField(7, address); + seaTunnelRow.setField(8, attributesMap); + seaTunnelRow.setField(9, phoneNumbers); + + return seaTunnelRow; + } + + private SeaTunnelRowType buildSeaTunnelRowType() { + SeaTunnelRowType addressType = + new SeaTunnelRowType( + new String[] {"city", "state", "street"}, + new SeaTunnelDataType[] { + BasicType.STRING_TYPE, BasicType.STRING_TYPE, BasicType.STRING_TYPE + }); + + return new SeaTunnelRowType( + new String[] { + "c_int32", + "c_int64", + "c_float", + "c_double", + "c_bool", + "c_string", + "c_bytes", + "Address", + "attributes", + "phone_numbers" + }, + new SeaTunnelDataType[] { + BasicType.INT_TYPE, + BasicType.LONG_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + BasicType.BOOLEAN_TYPE, + BasicType.STRING_TYPE, + PrimitiveByteArrayType.INSTANCE, + addressType, + new MapType<>(BasicType.STRING_TYPE, BasicType.FLOAT_TYPE), + ArrayType.STRING_ARRAY_TYPE + }); + } + + @Test + public void testConverter() + throws Descriptors.DescriptorValidationException, IOException, InterruptedException { + SeaTunnelRowType rowType = buildSeaTunnelRowType(); + SeaTunnelRow originalRow = buildSeaTunnelRow(); + + String protoContent = + "syntax = \"proto3\";\n" + + "\n" + + "package org.apache.seatunnel.format.protobuf;\n" + + "\n" + + "option java_outer_classname = \"ProtobufE2E\";\n" + + "\n" + + "message Person {\n" + + " int32 c_int32 = 1;\n" + + " int64 c_int64 = 2;\n" + + " float c_float = 3;\n" + + " double c_double = 4;\n" + + " bool c_bool = 5;\n" + + " string c_string = 6;\n" + + " bytes c_bytes = 7;\n" + + "\n" + + " message Address {\n" + + " string street = 1;\n" + + " string city = 2;\n" + + " string state = 3;\n" + + " string zip = 4;\n" + + " }\n" + + "\n" + + " Address address = 8;\n" + + "\n" + + " map attributes = 9;\n" + + "\n" + + " repeated string phone_numbers = 10;\n" + + "}"; + + String messageName = "Person"; + Descriptors.Descriptor descriptor = + CompileDescriptor.compileDescriptorTempFile(protoContent, messageName); + + RowToProtobufConverter rowToProtobufConverter = + new RowToProtobufConverter(rowType, descriptor); + byte[] protobufMessage = rowToProtobufConverter.convertRowToGenericRecord(originalRow); + + ProtobufToRowConverter protobufToRowConverter = + new ProtobufToRowConverter(protoContent, messageName); + DynamicMessage dynamicMessage = DynamicMessage.parseFrom(descriptor, protobufMessage); + SeaTunnelRow convertedRow = + protobufToRowConverter.converter(descriptor, dynamicMessage, rowType); + + Assertions.assertEquals(originalRow, convertedRow); + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.java new file mode 100644 index 00000000000..287dd903524 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.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.transform.nlpmodel; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class CustomConfigPlaceholder { + + // Placeholder ${model} + public static final String REPLACE_PLACEHOLDER_MODEL = "model"; + // Placeholder ${input} + public static final String REPLACE_PLACEHOLDER_INPUT = "input"; + // Placeholder ${prompt} + public static final String REPLACE_PLACEHOLDER_PROMPT = "prompt"; + + public static String replacePlaceholders( + String input, String placeholderName, String value, String defaultValue) { + String placeholderRegex = "\\$\\{" + Pattern.quote(placeholderName) + "(:[^}]*)?\\}"; + Pattern pattern = Pattern.compile(placeholderRegex); + Matcher matcher = pattern.matcher(input); + + StringBuffer result = new StringBuffer(); + while (matcher.find()) { + String replacement = + value != null && !value.isEmpty() + ? value + : (matcher.group(1) != null + ? matcher.group(1).substring(1).trim() + : defaultValue); + if (replacement == null) { + continue; + } + matcher.appendReplacement(result, Matcher.quoteReplacement(replacement)); + } + matcher.appendTail(result); + return result.toString(); + } + + public static Boolean findPlaceholder(String input, String placeholderName) { + String placeholderRegex = "\\$\\{" + Pattern.quote(placeholderName) + "(:[^}]*)?\\}"; + Pattern pattern = Pattern.compile(placeholderRegex); + Matcher matcher = pattern.matcher(input); + return matcher.find(); + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java new file mode 100644 index 00000000000..c14877816fc --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.nlpmodel; + +import org.apache.commons.lang3.StringUtils; + +public enum ModelProvider { + OPENAI("https://api.openai.com/v1/chat/completions", "https://api.openai.com/v1/embeddings"), + DOUBAO( + "https://ark.cn-beijing.volces.com/api/v3/chat/completions", + "https://ark.cn-beijing.volces.com/api/v3/embeddings"), + QIANFAN("", "https://aip.baidubce.com/rpc/2.0/ai_custom/v1/wenxinworkshop/embeddings"), + CUSTOM("", ""), + LOCAL("", ""); + + private final String LLMProviderPath; + private final String EmbeddingProviderPath; + + ModelProvider(String llmProviderPath, String embeddingProviderPath) { + LLMProviderPath = llmProviderPath; + EmbeddingProviderPath = embeddingProviderPath; + } + + public String usedLLMPath(String path) { + if (StringUtils.isBlank(path)) { + return LLMProviderPath; + } + return path; + } + + public String usedEmbeddingPath(String path) { + if (StringUtils.isBlank(path)) { + return EmbeddingProviderPath; + } + return path; + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java new file mode 100644 index 00000000000..b1234597502 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java @@ -0,0 +1,115 @@ +/* + * 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.nlpmodel; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.table.type.SqlType; + +import java.io.Serializable; +import java.util.Map; + +public class ModelTransformConfig implements Serializable { + + public static final Option MODEL_PROVIDER = + Options.key("model_provider") + .enumType(ModelProvider.class) + .noDefaultValue() + .withDescription("The model provider of LLM/Embedding"); + + 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 MODEL = + Options.key("model") + .stringType() + .noDefaultValue() + .withDescription( + "The model of LLM/Embedding, eg: if the model provider is OpenAI LLM, 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/Embedding"); + + public static final Option SECRET_KEY = + Options.key("secret_key") + .stringType() + .noDefaultValue() + .withDescription("The Secret key of LLM/Embedding"); + + public static final Option API_PATH = + Options.key("api_path") + .stringType() + .noDefaultValue() + .withFallbackKeys("openai.api_path") + .withDescription("The API of LLM/Embedding"); + + public static final Option OAUTH_PATH = + Options.key("oauth_path") + .stringType() + .noDefaultValue() + .withDescription("The Oauth path of LLM/Embedding"); + + public static final Option PROCESS_BATCH_SIZE = + Options.key("process_batch_size") + .intType() + .defaultValue(100) + .withFallbackKeys("inference_batch_size") + .withDescription("The row batch size of each process"); + + public static class CustomRequestConfig { + + // Custom response parsing + public static final Option> CUSTOM_CONFIG = + Options.key("custom_config") + .type(new TypeReference>() {}) + .noDefaultValue() + .withDescription("The custom config of the custom model."); + + public static final Option CUSTOM_RESPONSE_PARSE = + Options.key("custom_response_parse") + .stringType() + .noDefaultValue() + .withDescription( + "The response parse of the custom model. You can use Jsonpath to parse the return object you want to parse. eg: $.choices[*].message.content"); + + public static final Option> CUSTOM_REQUEST_HEADERS = + Options.key("custom_request_headers") + .mapType() + .noDefaultValue() + .withDescription("The custom request headers of the custom model."); + + public static final Option> CUSTOM_REQUEST_BODY = + Options.key("custom_request_body") + .type(new TypeReference>() {}) + .noDefaultValue() + .withDescription( + "The custom request body of the custom model." + + "1. ${model} placeholder for selecting model name." + + "2. ${input} placeholder for Determine input type. eg: [\"${input}\"]" + + "3. ${prompt} placeholder for LLM model " + + "4. ..."); + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java new file mode 100644 index 00000000000..9e77043f0a7 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java @@ -0,0 +1,212 @@ +/* + * 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.nlpmodel.embadding; + +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.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.VectorType; +import org.apache.seatunnel.transform.common.MultipleFieldOutputTransform; +import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; +import org.apache.seatunnel.transform.exception.TransformCommonError; +import org.apache.seatunnel.transform.nlpmodel.ModelProvider; +import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.Model; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.custom.CustomModel; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.doubao.DoubaoModel; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.openai.OpenAIModel; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.qianfan.QianfanModel; +import org.apache.seatunnel.transform.nlpmodel.llm.LLMTransformConfig; + +import lombok.NonNull; +import lombok.SneakyThrows; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class EmbeddingTransform extends MultipleFieldOutputTransform { + + private final ReadonlyConfig config; + private List fieldNames; + private List fieldOriginalIndexes; + private Model model; + private Integer dimension; + + public EmbeddingTransform( + @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); + this.config = config; + initOutputFields( + inputCatalogTable.getTableSchema().toPhysicalRowDataType(), + config.get(EmbeddingTransformConfig.VECTORIZATION_FIELDS)); + } + + private void tryOpen() { + if (model == null) { + open(); + } + } + + @Override + public void open() { + // Initialize model + ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); + try { + switch (provider) { + case CUSTOM: + // load custom_config from the configuration + ReadonlyConfig customConfig = + config.getOptional( + ModelTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) + .map(ReadonlyConfig::fromMap) + .orElseThrow( + () -> + new IllegalArgumentException( + "Custom config can't be null")); + model = + new CustomModel( + config.get(ModelTransformConfig.MODEL), + provider.usedEmbeddingPath( + config.get(ModelTransformConfig.API_PATH)), + customConfig.get( + LLMTransformConfig.CustomRequestConfig + .CUSTOM_REQUEST_HEADERS), + customConfig.get( + ModelTransformConfig.CustomRequestConfig + .CUSTOM_REQUEST_BODY), + customConfig.get( + LLMTransformConfig.CustomRequestConfig + .CUSTOM_RESPONSE_PARSE), + config.get( + EmbeddingTransformConfig + .SINGLE_VECTORIZED_INPUT_NUMBER)); + break; + case OPENAI: + model = + new OpenAIModel( + config.get(ModelTransformConfig.API_KEY), + config.get(ModelTransformConfig.MODEL), + provider.usedEmbeddingPath( + config.get(ModelTransformConfig.API_PATH)), + config.get( + EmbeddingTransformConfig + .SINGLE_VECTORIZED_INPUT_NUMBER)); + break; + case DOUBAO: + model = + new DoubaoModel( + config.get(ModelTransformConfig.API_KEY), + config.get(ModelTransformConfig.MODEL), + provider.usedEmbeddingPath( + config.get(ModelTransformConfig.API_PATH)), + config.get( + EmbeddingTransformConfig + .SINGLE_VECTORIZED_INPUT_NUMBER)); + break; + case QIANFAN: + model = + new QianfanModel( + config.get(ModelTransformConfig.API_KEY), + config.get(ModelTransformConfig.SECRET_KEY), + config.get(ModelTransformConfig.MODEL), + provider.usedEmbeddingPath( + config.get(ModelTransformConfig.API_PATH)), + config.get(ModelTransformConfig.OAUTH_PATH), + config.get( + EmbeddingTransformConfig + .SINGLE_VECTORIZED_INPUT_NUMBER)); + break; + case LOCAL: + default: + throw new IllegalArgumentException("Unsupported model provider: " + provider); + } + // Initialize dimension + dimension = model.dimension(); + } catch (IOException e) { + throw new RuntimeException("Failed to initialize model", e); + } + } + + private void initOutputFields(SeaTunnelRowType inputRowType, Map fields) { + List fieldNames = new ArrayList<>(); + List fieldOriginalIndexes = new ArrayList<>(); + for (Map.Entry field : fields.entrySet()) { + String srcField = field.getValue(); + int srcFieldIndex; + try { + srcFieldIndex = inputRowType.indexOf(srcField); + } catch (IllegalArgumentException e) { + throw TransformCommonError.cannotFindInputFieldError(getPluginName(), srcField); + } + fieldNames.add(field.getKey()); + fieldOriginalIndexes.add(srcFieldIndex); + } + this.fieldNames = fieldNames; + this.fieldOriginalIndexes = fieldOriginalIndexes; + } + + @Override + protected Object[] getOutputFieldValues(SeaTunnelRowAccessor inputRow) { + tryOpen(); + try { + Object[] fieldArray = new Object[fieldOriginalIndexes.size()]; + for (int i = 0; i < fieldOriginalIndexes.size(); i++) { + fieldArray[i] = inputRow.getField(fieldOriginalIndexes.get(i)); + } + List vectorization = model.vectorization(fieldArray); + return vectorization.toArray(); + } catch (Exception e) { + throw new RuntimeException("Failed to data vectorization", e); + } + } + + @Override + protected Column[] getOutputColumns() { + Column[] columns = new Column[fieldNames.size()]; + for (int i = 0; i < fieldNames.size(); i++) { + columns[i] = + PhysicalColumn.of( + fieldNames.get(i), + VectorType.VECTOR_FLOAT_TYPE, + null, + dimension, + true, + "", + ""); + } + return columns; + } + + @Override + public String getPluginName() { + return "Embedding"; + } + + @SneakyThrows + @Override + public void close() { + if (model != null) { + model.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java new file mode 100644 index 00000000000..0309363abd8 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.nlpmodel.embadding; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; + +import java.util.Map; + +public class EmbeddingTransformConfig extends ModelTransformConfig { + + public static final Option SINGLE_VECTORIZED_INPUT_NUMBER = + Options.key("single_vectorized_input_number") + .intType() + .defaultValue(1) + .withDescription( + "The number of single vectorized inputs, default is 1 , which means 1 inputs will be vectorized in one request , eg: qianfan only allows a maximum of 16 simultaneous messages, depending on your own settings, etc"); + + public static final Option> VECTORIZATION_FIELDS = + Options.key("vectorization_fields") + .mapType() + .noDefaultValue() + .withDescription( + "Specify the field vectorization relationship between input and output"); +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java new file mode 100644 index 00000000000..db3464e9291 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.nlpmodel.embadding; + +import org.apache.seatunnel.shade.com.google.common.collect.Lists; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +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 org.apache.seatunnel.transform.nlpmodel.ModelProvider; +import org.apache.seatunnel.transform.nlpmodel.llm.LLMTransformConfig; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class EmbeddingTransformFactory implements TableTransformFactory { + @Override + public String factoryIdentifier() { + return "Embedding"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required( + EmbeddingTransformConfig.MODEL_PROVIDER, + EmbeddingTransformConfig.MODEL, + EmbeddingTransformConfig.VECTORIZATION_FIELDS) + .optional( + EmbeddingTransformConfig.API_PATH, + EmbeddingTransformConfig.SINGLE_VECTORIZED_INPUT_NUMBER, + EmbeddingTransformConfig.PROCESS_BATCH_SIZE) + .conditional( + EmbeddingTransformConfig.MODEL_PROVIDER, + Lists.newArrayList(ModelProvider.OPENAI, ModelProvider.DOUBAO), + EmbeddingTransformConfig.API_KEY) + .conditional( + EmbeddingTransformConfig.MODEL_PROVIDER, + ModelProvider.QIANFAN, + EmbeddingTransformConfig.API_KEY, + EmbeddingTransformConfig.SECRET_KEY, + EmbeddingTransformConfig.OAUTH_PATH) + .conditional( + LLMTransformConfig.MODEL_PROVIDER, + ModelProvider.CUSTOM, + LLMTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) + .build(); + } + + @Override + public TableTransform createTransform(TableTransformFactoryContext context) { + return () -> + new EmbeddingTransform(context.getOptions(), context.getCatalogTables().get(0)); + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java new file mode 100644 index 00000000000..2e8665559b0 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java @@ -0,0 +1,70 @@ +/* + * 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.nlpmodel.embadding.remote; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.seatunnel.common.utils.BufferUtils; + +import org.apache.commons.lang3.ArrayUtils; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +public abstract class AbstractModel implements Model { + + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + protected static final String DIMENSION_EXAMPLE = "dimension example"; + private final Integer singleVectorizedInputNumber; + + protected AbstractModel(Integer singleVectorizedInputNumber) { + this.singleVectorizedInputNumber = singleVectorizedInputNumber; + } + + @Override + public List vectorization(Object[] fields) throws IOException { + List result = new ArrayList<>(); + + List> vectors = batchProcess(fields, singleVectorizedInputNumber); + for (List vector : vectors) { + result.add(BufferUtils.toByteBuffer(vector.toArray(new Float[0]))); + } + return result; + } + + protected abstract List> vector(Object[] fields) throws IOException; + + public List> batchProcess(Object[] array, int batchSize) throws IOException { + List> merged = new ArrayList<>(); + if (array == null || array.length == 0) { + return merged; + } + for (int i = 0; i < array.length; i += batchSize) { + Object[] batch = ArrayUtils.subarray(array, i, i + batchSize); + List> vector = vector(batch); + merged.addAll(vector); + } + if (array.length != merged.size()) { + throw new RuntimeException( + "The number of vectors is not equal to the number of inputs, Please verify the configuration of the input field and the result returned."); + } + return merged; + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java new file mode 100644 index 00000000000..d02dec7718e --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java @@ -0,0 +1,30 @@ +/* + * 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.nlpmodel.embadding.remote; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; + +public interface Model extends Closeable { + + List vectorization(Object[] fields) throws IOException; + + Integer dimension() throws IOException; +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java new file mode 100644 index 00000000000..532d89bbd69 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java @@ -0,0 +1,152 @@ +/* + * 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.nlpmodel.embadding.remote.custom; + +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.transform.nlpmodel.CustomConfigPlaceholder; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.AbstractModel; + +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 com.jayway.jsonpath.JsonPath; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class CustomModel extends AbstractModel { + + private final CloseableHttpClient client; + private final String model; + private final String apiPath; + private final Map header; + private final Map body; + private final String parse; + + public CustomModel( + String model, + String apiPath, + Map header, + Map body, + String parse, + Integer vectorizedNumber) { + super(vectorizedNumber); + this.apiPath = apiPath; + this.model = model; + this.header = header; + this.body = body; + this.parse = parse; + this.client = HttpClients.createDefault(); + } + + @Override + protected List> vector(Object[] fields) throws IOException { + return vectorGeneration(fields); + } + + @Override + public Integer dimension() throws IOException { + return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).size(); + } + + private List> vectorGeneration(Object[] fields) throws IOException { + HttpPost post = new HttpPost(apiPath); + // Construct a request with custom parameters + for (Map.Entry entry : header.entrySet()) { + post.setHeader(entry.getKey(), entry.getValue()); + } + + post.setEntity( + new StringEntity( + OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); + + CloseableHttpResponse response = client.execute(post); + + String responseStr = EntityUtils.toString(response.getEntity()); + + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to get vector from custom, response: " + responseStr); + } + + return OBJECT_MAPPER.convertValue( + parseResponse(responseStr), new TypeReference>>() {}); + } + + @VisibleForTesting + public Object parseResponse(String responseStr) { + return JsonPath.parse(responseStr).read(parse); + } + + @VisibleForTesting + public ObjectNode createJsonNodeFromData(Object[] fields) throws IOException { + JsonNode rootNode = OBJECT_MAPPER.readTree(OBJECT_MAPPER.writeValueAsString(body)); + Iterator> bodyFields = rootNode.fields(); + while (bodyFields.hasNext()) { + Map.Entry field = bodyFields.next(); + String fieldName = field.getKey(); + JsonNode fieldValue = field.getValue(); + if (fieldValue.isTextual()) { + String value = fieldValue.asText(); + if (CustomConfigPlaceholder.findPlaceholder( + value, CustomConfigPlaceholder.REPLACE_PLACEHOLDER_MODEL)) { + ((ObjectNode) rootNode) + .put( + fieldName, + CustomConfigPlaceholder.replacePlaceholders( + value, + CustomConfigPlaceholder.REPLACE_PLACEHOLDER_MODEL, + model, + null)); + } else if (CustomConfigPlaceholder.findPlaceholder( + value, CustomConfigPlaceholder.REPLACE_PLACEHOLDER_INPUT)) { + ((ObjectNode) rootNode) + .put( + fieldName, + CustomConfigPlaceholder.replacePlaceholders( + value, + CustomConfigPlaceholder.REPLACE_PLACEHOLDER_INPUT, + fields[0].toString(), + null)); + } + } else if (fieldValue.isArray()) { + ArrayNode arrayNode = OBJECT_MAPPER.valueToTree(Arrays.asList(fields)); + ((ObjectNode) rootNode).set(fieldName, arrayNode); + } + } + return ((ObjectNode) rootNode); + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java new file mode 100644 index 00000000000..945767d8324 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.nlpmodel.embadding.remote.doubao; + +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.transform.nlpmodel.embadding.remote.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 java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class DoubaoModel extends AbstractModel { + + private final CloseableHttpClient client; + private final String apiKey; + private final String model; + private final String apiPath; + + public DoubaoModel(String apiKey, String model, String apiPath, Integer vectorizedNumber) { + super(vectorizedNumber); + this.apiKey = apiKey; + this.model = model; + this.apiPath = apiPath; + this.client = HttpClients.createDefault(); + } + + @Override + protected List> vector(Object[] fields) throws IOException { + return vectorGeneration(fields); + } + + @Override + public Integer dimension() throws IOException { + return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).size(); + } + + private List> vectorGeneration(Object[] fields) throws IOException { + HttpPost post = new HttpPost(apiPath); + post.setHeader("Authorization", "Bearer " + apiKey); + post.setHeader("Content-Type", "application/json"); + post.setConfig( + RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); + + post.setEntity( + new StringEntity( + OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); + + CloseableHttpResponse response = client.execute(post); + String responseStr = EntityUtils.toString(response.getEntity()); + + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to get vector from doubao, response: " + responseStr); + } + + JsonNode data = OBJECT_MAPPER.readTree(responseStr).get("data"); + List> embeddings = new ArrayList<>(); + + if (data.isArray()) { + for (JsonNode node : data) { + JsonNode embeddingNode = node.get("embedding"); + List embedding = + OBJECT_MAPPER.readValue( + embeddingNode.traverse(), new TypeReference>() {}); + embeddings.add(embedding); + } + } + return embeddings; + } + + @VisibleForTesting + public ObjectNode createJsonNodeFromData(Object[] fields) { + ArrayNode arrayNode = OBJECT_MAPPER.valueToTree(Arrays.asList(fields)); + return OBJECT_MAPPER.createObjectNode().put("model", model).set("input", arrayNode); + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java new file mode 100644 index 00000000000..62a81cc097b --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java @@ -0,0 +1,116 @@ +/* + * 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.nlpmodel.embadding.remote.openai; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.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 java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class OpenAIModel extends AbstractModel { + + private final CloseableHttpClient client; + private final String apiKey; + private final String model; + private final String apiPath; + + public OpenAIModel(String apiKey, String model, String apiPath, Integer vectorizedNumber) { + super(vectorizedNumber); + this.apiKey = apiKey; + this.model = model; + this.apiPath = apiPath; + this.client = HttpClients.createDefault(); + } + + @Override + protected List> vector(Object[] fields) throws IOException { + if (fields.length > 1) { + throw new IllegalArgumentException("OpenAI model only supports single input"); + } + return vectorGeneration(fields); + } + + @Override + public Integer dimension() throws IOException { + return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).size(); + } + + private List> vectorGeneration(Object[] fields) throws IOException { + HttpPost post = new HttpPost(apiPath); + post.setHeader("Authorization", "Bearer " + apiKey); + post.setHeader("Content-Type", "application/json"); + post.setConfig( + RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); + + post.setEntity( + new StringEntity( + OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); + + CloseableHttpResponse response = client.execute(post); + String responseStr = EntityUtils.toString(response.getEntity()); + + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to get vector from openai, response: " + responseStr); + } + + JsonNode data = OBJECT_MAPPER.readTree(responseStr).get("data"); + List> embeddings = new ArrayList<>(); + + if (data.isArray()) { + for (JsonNode node : data) { + JsonNode embeddingNode = node.get("embedding"); + List embedding = + OBJECT_MAPPER.readValue( + embeddingNode.traverse(), new TypeReference>() {}); + embeddings.add(embedding); + } + } + return embeddings; + } + + @VisibleForTesting + public ObjectNode createJsonNodeFromData(Object[] data) throws JsonProcessingException { + ObjectNode objectNode = OBJECT_MAPPER.createObjectNode(); + objectNode.put("model", model); + objectNode.put("input", data[0].toString()); + return objectNode; + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java new file mode 100644 index 00000000000..081377e3def --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.nlpmodel.embadding.remote.qianfan; + +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.transform.nlpmodel.embadding.remote.AbstractModel; + +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +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 java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class QianfanModel extends AbstractModel { + + private final CloseableHttpClient client; + private final String apiKey; + private final String secretKey; + private final String model; + private final String apiPath; + private final String oauthPath; + private final String oauthSuffixPath = + "?grant_type=client_credentials&client_id=%s&client_secret=%s"; + private String accessToken; + + public QianfanModel( + String apiKey, + String secretKey, + String model, + String apiPath, + String oauthPath, + Integer vectorizedNumber) + throws IOException { + super(vectorizedNumber); + this.apiKey = apiKey; + this.secretKey = secretKey; + this.model = model; + this.apiPath = apiPath; + this.oauthPath = oauthPath; + this.client = HttpClients.createDefault(); + this.accessToken = getAccessToken(); + } + + public QianfanModel( + String apiKey, + String secretKey, + String model, + String apiPath, + Integer vectorizedNumber, + String oauthPath, + String accessToken) + throws IOException { + super(vectorizedNumber); + this.apiKey = apiKey; + this.secretKey = secretKey; + this.model = model; + this.apiPath = apiPath; + this.oauthPath = oauthPath; + this.client = HttpClients.createDefault(); + this.accessToken = accessToken; + } + + private String getAccessToken() throws IOException { + HttpGet get = new HttpGet(String.format(oauthPath + oauthSuffixPath, apiKey, secretKey)); + CloseableHttpResponse response = client.execute(get); + String responseStr = EntityUtils.toString(response.getEntity()); + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to Oauth for qianfan, response: " + responseStr); + } + JsonNode result = OBJECT_MAPPER.readTree(responseStr); + return result.get("access_token").asText(); + } + + @Override + public List> vector(Object[] fields) throws IOException { + return vectorGeneration(fields); + } + + @Override + public Integer dimension() throws IOException { + return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).get(0).size(); + } + + private List> vectorGeneration(Object[] fields) throws IOException { + String formattedApiPath = + String.format( + (apiPath.endsWith("/") ? apiPath : apiPath + "/") + "%s?access_token=%s", + model, + accessToken); + HttpPost post = new HttpPost(formattedApiPath); + post.setHeader("Content-Type", "application/json"); + post.setConfig( + RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); + + post.setEntity( + new StringEntity( + OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); + + CloseableHttpResponse response = client.execute(post); + String responseStr = EntityUtils.toString(response.getEntity()); + + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to get vector from qianfan, response: " + responseStr); + } + + JsonNode result = OBJECT_MAPPER.readTree(responseStr); + JsonNode errorCode = result.get("error_code"); + + if (errorCode != null) { + // Handle access token expiration + if (errorCode.asInt() == 110) { + this.accessToken = getAccessToken(); + } + throw new IOException( + "Failed to get vector from qianfan, response: " + result.get("error_msg")); + } + + List> embeddings = new ArrayList<>(); + JsonNode data = result.get("data"); + if (data.isArray()) { + for (JsonNode node : data) { + List embedding = + OBJECT_MAPPER.readValue( + node.get("embedding").traverse(), + new TypeReference>() {}); + embeddings.add(embedding); + } + } + return embeddings; + } + + @VisibleForTesting + public ObjectNode createJsonNodeFromData(Object[] data) { + ArrayNode arrayNode = OBJECT_MAPPER.valueToTree(Arrays.asList(data)); + return OBJECT_MAPPER.createObjectNode().set("input", arrayNode); + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} 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/nlpmodel/llm/LLMTransform.java similarity index 57% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java index d19960044f1..92db061ccca 100644 --- 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/nlpmodel/llm/LLMTransform.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.llm; +package org.apache.seatunnel.transform.nlpmodel.llm; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; @@ -26,8 +26,11 @@ 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 org.apache.seatunnel.transform.nlpmodel.ModelProvider; +import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; +import org.apache.seatunnel.transform.nlpmodel.llm.remote.Model; +import org.apache.seatunnel.transform.nlpmodel.llm.remote.custom.CustomModel; +import org.apache.seatunnel.transform.nlpmodel.llm.remote.openai.OpenAIModel; import lombok.NonNull; import lombok.SneakyThrows; @@ -61,18 +64,47 @@ public String getPluginName() { @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); + ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); + switch (provider) { + case CUSTOM: + // load custom_config from the configuration + ReadonlyConfig customConfig = + config.getOptional(ModelTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) + .map(ReadonlyConfig::fromMap) + .orElseThrow( + () -> + new IllegalArgumentException( + "Custom config can't be null")); + model = + new CustomModel( + inputCatalogTable.getSeaTunnelRowType(), + outputDataType.getSqlType(), + config.get(LLMTransformConfig.PROMPT), + config.get(LLMTransformConfig.MODEL), + provider.usedLLMPath(config.get(LLMTransformConfig.API_PATH)), + customConfig.get( + LLMTransformConfig.CustomRequestConfig + .CUSTOM_REQUEST_HEADERS), + customConfig.get( + LLMTransformConfig.CustomRequestConfig.CUSTOM_REQUEST_BODY), + customConfig.get( + LLMTransformConfig.CustomRequestConfig + .CUSTOM_RESPONSE_PARSE)); + break; + case OPENAI: + case DOUBAO: + model = + new OpenAIModel( + inputCatalogTable.getSeaTunnelRowType(), + outputDataType.getSqlType(), + config.get(LLMTransformConfig.PROMPT), + config.get(LLMTransformConfig.MODEL), + config.get(LLMTransformConfig.API_KEY), + provider.usedLLMPath(config.get(LLMTransformConfig.API_PATH))); + break; + case QIANFAN: + default: + throw new IllegalArgumentException("Unsupported model provider: " + provider); } } 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/nlpmodel/llm/LLMTransformConfig.java similarity index 55% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java index ca3da7e6706..8800f061db7 100644 --- 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/nlpmodel/llm/LLMTransformConfig.java @@ -15,27 +15,13 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.llm; +package org.apache.seatunnel.transform.nlpmodel.llm; import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; -import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; -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 class LLMTransformConfig extends ModelTransformConfig { public static final Option PROMPT = Options.key("prompt") @@ -43,19 +29,6 @@ public class LLMTransformConfig implements Serializable { .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() @@ -67,5 +40,5 @@ public class LLMTransformConfig implements Serializable { Options.key("openai.api_path") .stringType() .defaultValue("https://api.openai.com/v1/chat/completions") - .withDescription("The API path of OpenAI"); + .withDescription("The API path of OpenAI LLM"); } 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/nlpmodel/llm/LLMTransformFactory.java similarity index 67% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformFactory.java index 6fe5d53fe5b..eda57e1275f 100644 --- 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/nlpmodel/llm/LLMTransformFactory.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.llm; +package org.apache.seatunnel.transform.nlpmodel.llm; + +import org.apache.seatunnel.shade.com.google.common.collect.Lists; import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.table.catalog.CatalogTable; @@ -23,6 +25,8 @@ 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 org.apache.seatunnel.transform.nlpmodel.ModelProvider; +import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; import com.google.auto.service.AutoService; @@ -39,15 +43,25 @@ public OptionRule optionRule() { .required( LLMTransformConfig.MODEL_PROVIDER, LLMTransformConfig.MODEL, - LLMTransformConfig.PROMPT, - LLMTransformConfig.API_KEY) + LLMTransformConfig.PROMPT) .optional( + LLMTransformConfig.API_PATH, LLMTransformConfig.OUTPUT_DATA_TYPE, - LLMTransformConfig.INFERENCE_BATCH_SIZE) + LLMTransformConfig.PROCESS_BATCH_SIZE) + .conditional( + LLMTransformConfig.MODEL_PROVIDER, + Lists.newArrayList(ModelProvider.OPENAI, ModelProvider.DOUBAO), + LLMTransformConfig.API_KEY) + .conditional( + LLMTransformConfig.MODEL_PROVIDER, + ModelProvider.QIANFAN, + LLMTransformConfig.API_KEY, + LLMTransformConfig.SECRET_KEY, + ModelTransformConfig.OAUTH_PATH) .conditional( LLMTransformConfig.MODEL_PROVIDER, - ModelProvider.OPENAI, - LLMTransformConfig.OPENAI_API_PATH) + ModelProvider.CUSTOM, + LLMTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) .build(); } 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/nlpmodel/llm/remote/AbstractModel.java similarity index 98% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/AbstractModel.java index 51d674c0ad8..e658e514597 100644 --- 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/nlpmodel/llm/remote/AbstractModel.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.llm.model; +package org.apache.seatunnel.transform.nlpmodel.llm.remote; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; 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/nlpmodel/llm/remote/Model.java similarity index 94% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/Model.java index 77a8da63281..d055ac8b9e9 100644 --- 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/nlpmodel/llm/remote/Model.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.llm.model; +package org.apache.seatunnel.transform.nlpmodel.llm.remote; import org.apache.seatunnel.api.table.type.SeaTunnelRow; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java new file mode 100644 index 00000000000..af893e92ddc --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java @@ -0,0 +1,152 @@ +/* + * 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.nlpmodel.llm.remote.custom; + +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.shade.com.fasterxml.jackson.databind.node.TextNode; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.transform.nlpmodel.CustomConfigPlaceholder; +import org.apache.seatunnel.transform.nlpmodel.llm.remote.AbstractModel; + +import org.apache.groovy.util.Maps; +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 com.jayway.jsonpath.JsonPath; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class CustomModel extends AbstractModel { + + private final CloseableHttpClient client; + private final String model; + private final String apiPath; + private final Map header; + private final Map body; + private final String parse; + + public CustomModel( + SeaTunnelRowType rowType, + SqlType outputType, + String prompt, + String model, + String apiPath, + Map header, + Map body, + String parse) { + super(rowType, outputType, prompt); + this.apiPath = apiPath; + this.model = model; + this.header = header; + this.body = body; + this.parse = parse; + this.client = HttpClients.createDefault(); + } + + @Override + protected List chatWithModel(String promptWithLimit, String rowsJson) + throws IOException { + HttpPost post = new HttpPost(apiPath); + // Construct a request with custom parameters + for (Map.Entry entry : header.entrySet()) { + post.setHeader(entry.getKey(), entry.getValue()); + } + + post.setEntity( + new StringEntity( + OBJECT_MAPPER.writeValueAsString( + createJsonNodeFromData(promptWithLimit, rowsJson)), + "UTF-8")); + + CloseableHttpResponse response = client.execute(post); + + String responseStr = EntityUtils.toString(response.getEntity()); + + if (response.getStatusLine().getStatusCode() != 200) { + throw new IOException("Failed to get vector from custom, response: " + responseStr); + } + return OBJECT_MAPPER.convertValue( + parseResponse(responseStr), new TypeReference>() {}); + } + + @VisibleForTesting + public Object parseResponse(String responseStr) { + return JsonPath.parse(responseStr).read(parse); + } + + @VisibleForTesting + public ObjectNode createJsonNodeFromData(String prompt, String data) throws IOException { + JsonNode jsonNode = OBJECT_MAPPER.readTree(OBJECT_MAPPER.writeValueAsString(body)); + Map placeholderValues = + Maps.of( + CustomConfigPlaceholder.REPLACE_PLACEHOLDER_INPUT, data, + CustomConfigPlaceholder.REPLACE_PLACEHOLDER_PROMPT, prompt, + CustomConfigPlaceholder.REPLACE_PLACEHOLDER_MODEL, model); + + return (ObjectNode) replacePlaceholders(jsonNode, placeholderValues); + } + + private static JsonNode replacePlaceholders( + JsonNode node, Map placeholderValues) { + if (node.isObject()) { + ObjectNode objectNode = (ObjectNode) node; + Iterator> fields = objectNode.fields(); + while (fields.hasNext()) { + Map.Entry field = fields.next(); + objectNode.set( + field.getKey(), replacePlaceholders(field.getValue(), placeholderValues)); + } + } else if (node.isArray()) { + ArrayNode arrayNode = (ArrayNode) node; + for (int i = 0; i < arrayNode.size(); i++) { + arrayNode.set(i, replacePlaceholders(arrayNode.get(i), placeholderValues)); + } + } else if (node.isTextual()) { + String textValue = node.asText(); + for (Map.Entry entry : placeholderValues.entrySet()) { + if (CustomConfigPlaceholder.findPlaceholder(textValue, entry.getKey())) { + textValue = + CustomConfigPlaceholder.replacePlaceholders( + textValue, entry.getKey(), entry.getValue(), null); + } + } + return new TextNode(textValue); + } + return node; + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} 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/nlpmodel/llm/remote/openai/OpenAIModel.java similarity index 96% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/openai/OpenAIModel.java index 9477b873202..1424eed9e4c 100644 --- 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/nlpmodel/llm/remote/openai/OpenAIModel.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.llm.model.openai; +package org.apache.seatunnel.transform.nlpmodel.llm.remote.openai; import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; @@ -24,7 +24,7 @@ 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.seatunnel.transform.nlpmodel.llm.remote.AbstractModel; import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.CloseableHttpResponse; diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.java new file mode 100644 index 00000000000..4d054329069 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.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.nlpmodel.embadding.EmbeddingTransformFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class EmbeddingTransformFactoryTest { + + @Test + public void testOptionRule() throws Exception { + EmbeddingTransformFactory embeddingTransformFactory = new EmbeddingTransformFactory(); + Assertions.assertNotNull(embeddingTransformFactory.optionRule()); + } +} 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 index 39b27694805..85cad8e7995 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.seatunnel.transform; -import org.apache.seatunnel.transform.llm.LLMTransformFactory; +import org.apache.seatunnel.transform.nlpmodel.llm.LLMTransformFactory; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java new file mode 100644 index 00000000000..56e37f50212 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.transform.embedding; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; + +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.custom.CustomModel; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.doubao.DoubaoModel; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.openai.OpenAIModel; +import org.apache.seatunnel.transform.nlpmodel.embadding.remote.qianfan.QianfanModel; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class EmbeddingRequestJsonTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @Test + void testOpenAIRequestJson() throws IOException { + OpenAIModel model = + new OpenAIModel( + "apikey", "modelName", "https://api.openai.com/v1/chat/completions", 1); + ObjectNode node = + model.createJsonNodeFromData( + new Object[] { + "Determine whether someone is Chinese or American by their name" + }); + Assertions.assertEquals( + "{\"model\":\"modelName\",\"input\":\"Determine whether someone is Chinese or American by their name\"}", + OBJECT_MAPPER.writeValueAsString(node)); + model.close(); + } + + @Test + void testDoubaoRequestJson() throws IOException { + DoubaoModel model = + new DoubaoModel( + "apikey", "modelName", "https://api.doubao.io/v1/chat/completions", 1); + ObjectNode node = + model.createJsonNodeFromData( + new Object[] { + "Determine whether someone is Chinese or American by their name" + }); + Assertions.assertEquals( + "{\"model\":\"modelName\",\"input\":[\"Determine whether someone is Chinese or American by their name\"]}", + OBJECT_MAPPER.writeValueAsString(node)); + model.close(); + } + + @Test + void testQianfanRequestJson() throws IOException { + QianfanModel model = + new QianfanModel( + "apikey", + "secretKey", + "modelName", + "https://api.qianfan.io/v1/chat/completions", + 1, + "xxxx", + "xxxxxxx"); + ObjectNode node = + model.createJsonNodeFromData( + new Object[] { + "Determine whether someone is Chinese or American by their name" + }); + Assertions.assertEquals( + "{\"input\":[\"Determine whether someone is Chinese or American by their name\"]}", + OBJECT_MAPPER.writeValueAsString(node)); + model.close(); + } + + @Test + void testCustomRequestJson() throws IOException { + Map header = new HashMap<>(); + header.put("Content-Type", "application/json"); + header.put("Authorization", "Bearer " + "apikey"); + Map body = new HashMap<>(); + body.put("model1", "${model}"); + body.put("input1", Lists.newArrayList("${input}", "${input}")); + + CustomModel model = + new CustomModel( + "modelName", + "https://api.custom.com/v1/chat/completions", + header, + body, + "$.data[*].embedding", + 1); + ObjectNode node = + model.createJsonNodeFromData( + new Object[] { + "Determine whether someone is Chinese or American by their name" + }); + Assertions.assertEquals( + "{\"model1\":\"modelName\",\"input1\":[\"Determine whether someone is Chinese or American by their name\"]}", + OBJECT_MAPPER.writeValueAsString(node)); + model.close(); + } + + @Test + void testCustomParseResponse() { + CustomModel customModel = + new CustomModel( + "modelName", + "https://api.custom.com/v1/chat/completions", + new HashMap<>(), + new HashMap<>(), + "$.data[*].embedding", + 1); + List> lists = + OBJECT_MAPPER.convertValue( + customModel.parseResponse( + "{\"created\":1725001256,\"id\":\"02172500125677376580aba8475a41c550bbf05104842f0405ef5\",\"data\":[{\"embedding\":[-1.625,0.07958984375,-1.5703125,-3.03125,-1.4609375,3.46875,-0.73046875,-2.578125,-0.66796875,1.71875,0.361328125,2,5.125,2.25,4.6875,1.4921875,-0.77734375,-0.466796875,0.0439453125,-2.46875,3.59375,4.96875,2.34375,-5.34375,0.11083984375,-5.875,3.0625,4.09375,3.4375,0.2265625,9,-1.9296875,2.25,0.765625,3.671875,-2.484375,-1.171875,-1.6171875,4.1875,2.390625,-6.90625,0.369140625,0.259765625,3.671875,-2.9375,-1.9140625,-0.71875,-1.6640625,0.29296875,0.396484375,-4.625,-1.9921875,5.15625,-1.3984375,3.015625,-3.203125,-1.453125,4,-8.75,-5.625,1.0546875,-3.28125,-1.2265625,0.287109375,2.09375,4.6875,0.1572265625,0.42578125,0.79296875,3.234375,-0.169921875,0.9296875,7.40625,-3.296875,5.53125,3.890625,0.62109375,1.1171875,-0.373046875,4.125,-2.78125,0.333984375,3.9375,4.59375,6,1.53125,-0.373046875,1.109375,-4.0625,1.96875,1.421875,0.57421875,-0.56640625,-2.390625,0.734375,1.1875,-2.859375,-6.09375,-5.96875,1.8359375,-3,0.80859375,-0.130859375,-5.3125,-2.859375,1.484375,-4.53125,-6.90625,-2.25,0.7734375,-1.2734375,1.1484375,3.421875,-3.484375,2.65625,1.3359375,1.1484375,-4.09375,-5.625,2.625,-0.283203125,-3.46875,2.3125,-0.220703125,4.21875,3.75,-0.37109375,0.9609375,7.25,-0.87890625,7.03125,2.34375,4.5,-1.609375,-6.46875,-6.125,-2.59375,2.234375,3.78125,1.3046875,-5.5,1.953125,-3.421875,-5.9375,3.25,-3.4375,-8.3125,-2.546875,3.640625,0.267578125,-0.220703125,0.294921875,-0.4140625,2.515625,-1.0546875,-5.21875,6.6875,3.640625,0.2314453125,-2.5,1,1.6640625,0.59765625,2.75,1.1328125,1.1328125,-4.96875,4.53125,-0.349609375,3.390625,-0.193359375,7.625,2.921875,-3.484375,4.1875,8.5,-1.9140625,6.3125,2.5625,3.0625,0.40234375,0.76953125,-4.78125,3.53125,-2.765625,0.1591796875,-0.1025390625,-3.875,2.203125,0.03076171875,1.765625,1.859375,2.15625,-1.2578125,-4.40625,-0.62890625,4.4375,-1.78125,2.671875,2.765625,-1.7890625,-8.3125,-0.02197265625,1.640625,-3.96875,-3.15625,2.796875,1.1875,2,1.15625,2.359375,1.3984375,4.21875,-2.953125,8.5,3.46875,3.578125,0.90625,-1.8828125,2.15625,3.921875,4.125,-0.9609375,-2.171875,2.328125,2.921875,1.9765625,1.0703125,4.03125,6.28125,-3.59375,-0.94921875,5.6875,-1.9140625,-5.1875,-4.25,-7.71875,1.7109375,-1.59375,3.765625,-5.3125,-3.9375,-3.796875,2.90625,2.859375,-2.203125,-1.78125,-3.796875,0.1708984375,-5.15625,0.298828125,1.828125,-7.1875,1.6953125,6.125,2.671875,0.1728515625,3.375,0.609375,-4.78125,0.40625,-3.875,-6.4375,0.6953125,1.171875,-2.140625,5.8125,-1.640625,5.90625,-0.1650390625,4.9375,-2.421875,1.609375,-3.171875,-4.71875,7.6875,-1.09375,-1.9296875,0.033447265625,-3.46875,-2.671875,-8.875,2.4375,-1.1015625,4.40625,-3.53125,1.546875,2.359375,-3.15625,10.625,7.46875,-3.0625,-0.044677734375,0.90234375,-5.28125,-3,-1.2890625,0.59375,-6.34375,-1.8203125,5.40625,5.78125,-1.578125,2.46875,-2.171875,-1.71875,-0.38671875,-2.21875,-0.150390625,4.65625,-3.46875,1.5625,4.4375,-2.609375,1.6875,-2.828125,-6.03125,5.15625,-2.296875,-1.65625,-2.3125,-4.75,-3.3125,-3.703125,-1.9296875,-6.59375,3.640625,-0.62890625,4.8125,0.11279296875,2.515625,0.9921875,-3.03125,-5.40625,7.5625,-1.765625,4.4375,4.25,-0.140625,3.671875,-2.984375,-2.734375,2.203125,-6.96875,-1.1640625,2.390625,1.3515625,-1.75,2.96875,-3.75,-0.109375,2.5,0.796875,5.21875,7.8125,-4,1.171875,0.435546875,1.2734375,-3.015625,1.90625,-1.21875,5.9375,-0.31640625,-4.03125,-3.59375,1.09375,4.65625,-0.81640625,-2.046875,0.80859375,-5.375,2,-2.265625,5.34375,-0.46875,-1.3359375,-2.953125,-4.875,-0.53515625,-3,1.8203125,-2.59375,-1.4765625,6.28125,2.09375,0.1318359375,2.40625,-0.09130859375,-2.421875,-1.78125,1.59375,0.48828125,-0.310546875,-0.2353515625,0.1748046875,0.4453125,2.078125,-1.046875,1.46875,0.6953125,-0.52734375,-0.19140625,-2.28125,-0.515625,0.47265625,-1.2421875,-8.3125,1.1875,2.015625,-4.46875,3.734375,1.453125,-2.8125,-2.78125,5.875,-0.38671875,1.171875,-6.5,1.8046875,-2.15625,4,3.375,-0.78125,0.87890625,-1.796875,-1.265625,2.59375,3.96875,1.7421875,2.296875,2.78125,-5.8125,-2.046875,-0.1201171875,-4.1875,3.96875,-3.484375,-4.125,1.21875,3.484375,0.3828125,3.8125,1.90625,-8.3125,-2.15625,2.578125,2.578125,-1.34375,-3.359375,4.71875,-1.640625,-3.484375,2.046875,3.0625,-1.03125,-2.96875,6.96875,3.703125,-0.29296875,-0.423828125,2.640625,-1.265625,3.9375,-0.314453125,-4.15625,-2.171875,0.2734375,6.375,-6.21875,-6.3125,4.6875,-0.053466796875,0.045166015625,2.765625,2.953125,1.078125,-0.453125,1.96875,-6.71875,-3.375,-4.1875,2.515625,-0.5390625,-1.9296875,-4.03125,1.1953125,8.1875,1.0078125,0.80859375,-1.15625,-1.53125,2.875,-3.921875,1.953125,4.09375,6.59375,-4.5625,-1.2109375,3.5,-8.1875,0.294921875,-3.453125,-0.9921875,-2.015625,1.5,0.6328125,-4.90625,-2.765625,1.0546875,4.25,-2.390625,-5.96875,7.15625,-5.4375,-3.953125,1.15625,-0.017822265625,2.90625,2.78125,-2.21875,3.390625,1.9453125,2.171875,1.8671875,-1.125,-3.65625,-1.359375,0.96484375,2.5625,-2.9375,1.2734375,4.15625,-6,-0.2021484375,-1.8515625,-0.56640625,-1.671875,1.546875,5.8125,-0.640625,1.140625,-5.71875,-0.40625,0.5390625,-1.6640625,0.3203125,-2.375,4.9375,-2.453125,-1.59375,0.1669921875,1.6796875,-0.81640625,1.765625,-3.125,-1.234375,0.84375,-0.96484375,0.232421875,-0.01300048828125,-6.03125,4.25,5.625,0.65625,-1.6015625,1.390625,5.65625,3.0625,0.287109375,-0.08203125,4.15625,-1.5703125,-0.609375,-6.34375,2.203125,-3.84375,-2.53125,-3.390625,3.15625,4.59375,-4.46875,5.0625,-3.09375,3.328125,-0.65625,1.8515625,-9.375,1.609375,-1.515625,-2.5625,-2.953125,0.734375,2.375,1.3515625,0.390625,1.8671875,0.07080078125,1.328125,3.6875,0.2421875,0.73828125,3.1875,1.65625,2.75,2.859375,-2.8125,-7.75,1.53125,-1.1015625,-1.6875,6.3125,3.03125,-2.46875,0.77734375,-0.34765625,-1.78125,-1.4453125,3.40625,3.140625,-3.96875,3.984375,-3.21875,5.375,-2.890625,2.90625,-2.375,-6.1875,-2.4375,0.34375,-4.46875,-2.421875,3.40625,-1.2578125,4.59375,4.125,-6,0.003936767578125,1.1484375,2.359375,4.09375,0.5703125,-1.328125,-6.03125,4.5,3.234375,-2.140625,5.03125,-2.640625,0.041748046875,-0.90234375,4.375,-2.125,-0.1630859375,2.421875,-2.078125,1.1328125,-3.53125,1.0234375,-0.2734375,-9.125,-6.03125,0.73828125,-0.87109375,6.59375,-0.65625,-2.109375,-3.359375,2.40625,-0.0157470703125,5.96875,2.390625,3.078125,5.65625,5.09375,-1.5859375,1.78125,-0.921875,-8.0625,7.0625,-5.71875,-2.375,2.359375,2.65625,-1.453125,-1.2265625,1.984375,-2.125,-5.46875,-5.25,-1.78125,-4.28125,3.375,-2.09375,1.984375,-0.75,-5.0625,1.46875,-1.8671875,-2.875,-1.859375,2.609375,-5.5,2.484375,5.65625,1.875,-0.94921875,3.890625,4.125,0.8984375,-2.796875,0.95703125,-7.9375,1.7890625,3.453125,-1.9296875,-0.69140625,-5.84375,2.171875,-3.4375,2.921875,0.890625,-2.203125,-2.375,-1.6328125,-2.65625,0.8515625,-7.28125,2.484375,1.6484375,-0.8359375,-0.859375,0.232421875,1.921875,0.73046875,-0.30078125,1.515625,4.9375,0.7109375,-0.43359375,-3.140625,-2.796875,-0.2431640625,2.265625,-2.53125,6.875,-0.54296875,-1.5625,3.96875,0.44921875,-3.640625,-4.25,4.375,-1.875,0.45703125,-1.2265625,5.65625,0.298828125,3.921875,-1.703125,-2.8125,-3.328125,1.7578125,3.3125,-1.6875,-3.234375,2.09375,2.375,5.40625,-3.234375,-7.09375,1.984375,4.125,-0.8046875,-2.71875,8.6875,-1.296875,-2.625,-3,-3.78125,1.359375,1.515625,2.875,0.11279296875,-1.5859375,1.078125,3.46875,-1.390625,0.6328125,0.24609375,-3.765625,3.515625,0.380859375,2.609375,-0.80078125,-2.484375,-2.15625,-1.3203125,0.02490234375,4.03125,8.25,-1.5234375,-1.1953125,1.2109375,0.3125,-1.7421875,5.625,-0.76953125,5.90625,1.15625,0.1640625,1.390625,0.82421875,-0.322265625,3.21875,-4.65625,-4.5,-1.765625,3.171875,-4.3125,-1.4375,-2.546875,-0.9140625,4.28125,0.609375,-3.171875,3.671875,0.48046875,-0.9140625,-4,-2.4375,-5.34375,-1.96875,0.828125,1.953125,-2.140625,-2.59375,-0.353515625,4.78125,-4.09375,-3.921875,0.03173828125,1.8359375,1.3984375,-0.65234375,-1.15625,0.1611328125,0.50390625,2.90625,-1.875,-3.40625,0.498046875,8.75,3.90625,-4.53125,0.67578125,-0.765625,1.8359375,-5.3125,-2.15625,-0.6796875,-1.8984375,-3.046875,-1.7734375,-1.390625,-2.71875,-2.015625,5.84375,-3.28125,0.55859375,0.8046875,3.984375,0.99609375,3.015625,0.458984375,5.3125,3.1875,-1.2421875,-5.84375,-1.3828125,-0.04052734375,-5.75,-1.8828125,3.234375,6,3.171875,1.5703125,-2.828125,0.033203125,-0.953125,0.640625,5.3125,-5.75,-3.78125,-1.984375,-7.9375,-6.84375,-3.859375,-2.65625,-3.15625,-6.84375,-0.9765625,-1.375,-7.1875,-1.1328125,-2.109375,-1.546875,-1,0.640625,4.625,-4.65625,2.3125,3.703125,2.6875,3.0625,-2.28125,3.34375,0.474609375,-1.46875,0.34765625,-2.03125,5.25,-1.4609375,5.875,3.984375,-0.87890625,-3.8125,4.46875,4.40625,5.90625,-4.875,-3.53125,-2.53125,-1.8125,-0.39453125,-1.2421875,2.203125,-3.828125,-3.59375,-1.0859375,-3.453125,0.1845703125,5.625,0.421875,5.3125,-1.3671875,0.30859375,1.5234375,2.953125,0.1064453125,2.59375,1.5546875,-4.46875,3.609375,-0.81640625,1.390625,0.8359375,-2.78125,2.125,-1.6875,0.365234375,2.234375,3.875,10.4375,1.15625,2.328125,-0.09326171875,-0.76171875,-2.609375,-2.96875,2.40625,1.6796875,1.4921875,-3.65625,0.74609375,-0.8828125,2.03125,-0.306640625,-16.875,-3.328125,-5.53125,-2.109375,4.625,-1.0546875,-1.984375,1.0625,3.6875,2.671875,7.09375,-1.484375,4.03125,-1.078125,-0.7265625,2.515625,-4.3125,1.578125,3.6875,1.890625,4.625,1.7734375,-1.8125,-2.828125,6.9375,5.0625,-4.5,0.193359375,5.09375,-1.3515625,-1.140625,4.40625,-2.96875,2.078125,-4.75,3.078125,7.09375,2.75,-2.953125,-4.125,-2.375,-2.0625,1.0234375,3.046875,-2.578125,1.578125,2.921875,-5.65625,2.28125,2.28125,-0.259765625,-3.484375,-0.37109375,2.71875,1.625,-0.158203125,-4.5,2.5625,0.98828125,3.84375,4.8125,-2.796875,-2.140625,2.34375,2.90625,2.1875,1.5546875,2.578125,2.8125,-1.8515625,-2.984375,0.310546875,-1.328125,-0.0234375,-1.9765625,0.83984375,3.65625,2.046875,-4.5625,2.171875,2.234375,-2.109375,-0.0439453125,-4.0625,-3.5,2.09375,-2.21875,-2.5,0.703125,-2.953125,-1.28125,3.234375,-4.6875,4.1875,-2.484375,8.75,-0.53125,-1.8203125,1.171875,-3.0625,4.78125,-2.484375,-3.453125,3.765625,-2.6875,1.5625,-3.828125,1.9296875,-1.765625,1.2421875,5.0625,-4.65625,-2.0625,0.53125,3.265625,-2.875,-2.296875,0.29296875,3.859375,0.123046875,-4.46875,4.09375,-2.796875,3.96875,-3.890625,1.875,-4.46875,-0.5078125,2.140625,0.3203125,4.84375,5.03125,-5.34375,-4.96875,-1.3203125,-5.03125,-4.875,-4.5625,5.03125,-2.625,-0.75,1.046875,2.109375,-0.130859375,1.890625,-1.8125,2.53125,6.53125,-2.09375,0.87890625,-0.41015625,-0.412109375,-4.09375,-2.421875,-4.46875,6.40625,0.43359375,1.2578125,3.734375,-1.7109375,2.953125,1.8125,-1.1171875,-1.7109375,2.15625,1.859375,-2.015625,-2.25,1.7734375,-3.578125,4.15625,-3.328125,-3.28125,-4.71875,2.953125,1.40625,-0.287109375,1.5703125,3.53125,1.578125,3.171875,-4.34375,-3.125,5.78125,3.453125,-2.046875,4.3125,-1.2265625,-1.84375,0.640625,2.625,0.12890625,-3.25,-4.6875,5.28125,2.65625,2.015625,-4.4375,-5.75,-3.625,4.0625,4.59375,-0.78125,-2.484375,-2.03125,-3.75,1.6875,-4.15625,2.734375,-1.65625,-3.453125,-0.89453125,3.71875,2.453125,-4.15625,2.09375,0.82421875,-2.03125,0.052978515625,4.4375,1.734375,-3.71875,1.375,-0.349609375,-1.75,-7,3.59375,-2.625,-0.427734375,-4.40625,-3.84375,-3.265625,-3.796875,0.74609375,2.65625,1.6171875,3.609375,-0.7890625,3.890625,2.796875,-0.8671875,-0.43359375,2.15625,-1.7578125,-3.640625,2.375,-4.65625,-3.5,1.3984375,-7.1875,-1.5,5.0625,-2.625,4.0625,-1.171875,3.796875,-1.453125,-2.9375,-4,-1.3046875,0.91796875,6.59375,0.64453125,-0.91796875,0.64453125,2.703125,2.1875,-2.296875,-1.015625,-1.9921875,5,-0.298828125,2.953125,-5.125,-5.03125,5.375,-1.1328125,-4.46875,-0.5546875,-3.09375,1.5703125,5.34375,0.765625,-4.46875,-2.421875,-6.75,2.8125,-1.6171875,3.109375,-5.59375,0.87109375,-4.875,2.53125,4.46875,-7.21875,-3.1875,2.4375,3,5.1875,1.84375,-2.625,-6.21875,2.21875,0.306640625,-1.90625,-4.09375,-2.34375,-1.3046875,-3.875,4.4375,-2.328125,2.546875,-3.875,-2.40625,0.80078125,0.34765625,1,0.828125,1.4453125,-0.859375,3.03125,1.109375,5.15625,1.1953125,-3.8125,2.734375,4.21875,0.345703125,-1.2109375,2.0625,-0.79296875,-2.8125,2.109375,2.96875,-2.90625,5.15625,3.359375,4.3125,-5.53125,-2.875,1.515625,3.515625,-2.75,1.7109375,-4.9375,0.7265625,3.71875,-0.4765625,1.34375,0.049560546875,2.796875,-1.421875,-1.7890625,1.5,2.3125,4.21875,1.6875,3.015625,3.3125,-1.1640625,3.546875,-0.375,-1.2265625,-1.59375,3.609375,-3.015625,-2.546875,-4.625,1.046875,-1.796875,4.75,2.515625,1.1484375,0.8984375,-1.4140625,-2.328125,0.037841796875,-5.78125,-1.5859375,0.11669921875,3.015625,-0.83984375,0.84375,-0.82421875,0.96484375,4.0625,0.0400390625,4.25,-2.28125,1.3515625,1,1.5625,-2.8125,3.15625,-2.609375,-0.142578125,1.578125,-2.875,3.75,-4.3125,-1.359375,-2.578125,-0.69140625,2.84375,3.75,-4.75,-5.5625,0.84765625,0.380859375,5.125,3.0625,-3.140625,-0.93359375,0.73046875,0.0303955078125,4.3125,0.85546875,2.703125,-4.28125,5.625,5.90625,0.4296875,0.76953125,-0.9140625,-1.71875,-4.5,3.828125,-0.4609375,2.21875,-1.9453125,2.5,4.15625,1.8984375,3.984375,-5.75,2.953125,0.2734375,3.890625,-0.76171875,-3.90625,0.337890625,1.96875,0.69140625,-0.70703125,3.578125,0.046142578125,0.765625,-2.734375,4.28125,4.3125,2.578125,-4.40625,1.921875,-2.90625,1.7734375,-3.90625,1.1484375,-5.625,1.65625,2.703125,5.34375,-1.9296875,-6.1875,4.5,1.5625,-0.9140625,-3.953125,4.65625,4.5625,2.484375,-5.15625,-2.375,1.625,-1.328125,-0.26171875,-5.25,3.328125,2.0625,-3.609375,-3.71875,1.6171875,1.046875,-3.1875,-3.71875,-3.34375,1.9609375,2.5625,3.609375,1.59375,-2.484375,4.125,-0.80078125,1.9140625,4.78125,-1.09375,0.140625,3.171875,-3.578125,2.640625,-0.6640625,-2.65625,-1.4375,0.47265625,-2.46875,2.6875,-2.515625,-1.0234375,-2.09375,-0.138671875,-0.5078125,1.5,4.15625,-3.09375,0.158203125,4.4375,-1.96875,-3,-1.9609375,2.09375,-1.7734375,-1.09375,-1.8984375,3.3125,1.9765625,0.8671875,0.2890625,0.66796875,-1.9765625,-3.640625,-4.90625,2.0625,-4.0625,3.59375,-0.84765625,-6.21875,1.515625,3.890625,3.640625,-0.2734375,-2.046875,0.875,3.78125,0.07470703125,-1.078125,-1.4921875,3.671875,-2.796875,-3.6875,2.75,2.78125,-5.40625,1.7890625,-4.28125,-2.265625,-0.98046875,4.46875,0.173828125,-2.25,-2.875,-3.84375,-1.7421875,-1.6171875,-3.21875,1.9140625,1.7421875,2.671875,1.09375,1.4375,-3.5,2.59375,19.125,0.0101318359375,-8.4375,1.3515625,-3.625,4.4375,4.65625,1.8125,0.423828125,-1.5,0.62890625,4.21875,0.609375,0.5390625,-2.390625,0.984375,-0.79296875,2.078125,-3.703125,-3.109375,-2.265625,-1.0234375,-0.328125,1.9765625,2.5,2.375,0.8046875,-2.265625,1.2734375,-3.390625,-4.375,-4.71875,3.765625,-2.921875,3.125,-3.171875,4.65625,-0.7890625,-3.3125,-2.984375,-3.296875,-2.796875,2.375,-0.12255859375,-3.21875,5.21875,0.1982421875,0.2138671875,-1.1796875,-0.130859375,-4.34375,-1.4453125,-2.5,6.3125,1.0625,-6.15625,-0.5703125,-3.203125,-3.546875,-1.375,2.9375,-0.53515625,1.7578125,2.71875,-1.9453125,-2.640625,-3.046875,0.49609375,1.0078125,-3,-4.84375,0.2119140625,1.2265625,1.3515625,1.609375,-4.84375,2.46875,2.140625,2.171875,1.75,0.67578125,-0.60546875,-2.46875,-2.234375,-0.9453125,1.2421875,-3.15625,0.006744384765625,3.359375,-1.765625,8.375,-8.3125,5.8125,5.15625,-2.0625,-0.470703125,1.5,-0.30859375,-2.421875,-0.2294921875,0.95703125,1.8828125,4.84375,-0.68359375,4.625,1.359375,0.373046875,0.83203125,2.640625,4.34375,0.7578125,3.109375,-0.412109375,-2,2.15625,-0.08349609375,-3.140625,-3,-3.703125,-2.5625,3.6875,1.7890625,-3.296875,0.89453125,-7.5,-5.40625,-2.25,-7.625,4.34375,-1.34375,-0.14453125,3.515625,-2.46875,-1.2109375,-2.46875,-3.921875,1.265625,3.65625,1.4375,-1.46875,-5.03125,2.59375,3.890625,-2.765625,-2.4375,0.353515625,-4.21875,4.4375,-0.376953125,3.9375,-2.09375,3.96875,3.234375,-2.203125,-6.875,5.15625,-3.6875,-4.34375,-6.625,-2.90625,-4.9375,-3.34375,0.412109375,-0.9453125,-0.5703125,-1.3046875,3.21875,-0.65234375,-1.6796875,3.171875,3.453125,-4.4375,-1.2578125,0.828125,1.1796875,-4.375,0.1787109375,4,0.53515625,1.328125,-0.546875,0.271484375,-0.5546875,-3.859375,-0.2216796875,0.86328125,-4.53125,-1.3828125,-0.60546875,-5.46875,-1.3515625,-1.2890625,-3.734375,2.9375,2.40625,-3.984375,0.875,-2.953125,-0.9765625,-1.6328125,-1.25,3.96875,1.6953125,0.0072021484375,5.875,-0.921875,-3.46875,-3.140625,-0.26953125,0.2265625,-2.09375,7.0625,-1.09375,0.30078125,-6.03125,5.34375,2.359375,1.6640625,-0.99609375,4.625,4.25,-2.484375,-4,0.89453125,3.0625,4.1875,-4.28125,3.953125,0.6328125,-0.74609375,-1.53125,2.015625,-1.1796875,1.03125,-1.6484375,-5.4375,0.3671875,1.8125,-0.326171875,1.546875,4.03125,-3.34375,0.484375,2.5,-1.4140625,3.34375,4.25,-1.7890625,1.09375,2.171875,5.34375,-1.5625,0.98828125,-5.09375,-3.625,-2.640625,-2.46875,3.109375,-2.515625,0.09033203125,0.21484375,-3.921875,3.125,-4.1875,1.2109375,1.3671875,1.1875,-5.4375,4.59375,3.890625,-2.8125,3.328125,-5.125,-1.9765625,-1.4296875,2.34375,-2.71875,-5.875,3.125,3.453125,-1.515625,3.546875,2.265625,-0.52734375,1.9375,-2.859375,2.703125,-3.359375,4.75,1.2734375,3.09375,3.65625,-0.255859375,-0.1044921875,-5.75,-0.3359375,-0.77734375,-2.234375,6.1875,-3.84375,0.19921875,4.25,6.4375,-10.5,-1.5078125,0.7265625,0.2890625,3.921875,5.0625,0.09814453125,0.68359375,3.109375,1.015625,2.671875,0.0257568359375,-0.4765625,-4,5.15625,0.2314453125,-4.6875,3.1875,3.984375,-2.609375,3.4375,-2.375,-3.734375,-0.07568359375,2.75,-5.3125,1.9296875,4.625,-1.6484375,2.875,3.734375,-1.34375,3.875,-1.9921875,-11.3125,-1.53125,3.296875,5.71875,0.80859375,1.7578125,0.48046875,-2.015625,1.4765625,-0.5546875,0.71484375,-0.7578125,-11.1875,0.9765625,-3,-0.09765625,-1.9453125,-3.8125,-2.5,4.375,1.65625,1.1015625,3.328125,2.84375,0.84375,4.5625,0.11279296875,-5.84375,1.1484375,1.7578125,-4.8125,-0.59765625,3.234375,1.125,-1.859375,-2.515625,3.78125,-1.7421875,-0.69921875,5.8125,3.765625,1.578125,-1.84375,-5.03125,0.984375,-3.375,-1.9140625,1.1953125,-0.384765625,2.8125,-2.203125,2.828125,1.1171875,-3.75,-4.15625,-2.25,-3.5625,1.5,2.671875,2.171875,-2.609375,-1.7265625,2.8125,2.5,-0.455078125,-1.546875,2.1875,-0.1884765625,-2.984375,-1.4765625,2.0625,-4.46875,-2.90625,4.0625,1.8359375,0.443359375,-0.7734375,-3.140625,2.171875,1.734375,-1.8515625,-1.84375,-1.234375,2.15625,5.34375,-2.484375,-5.6875,-1.2734375,0.1806640625,-4.375,-3.5625,0.89453125,-1.15625,0.75,3.09375,-2.25,1.1875,4.6875,-1.3359375,-3.875,3.53125,4.4375,-2.671875,-0.75,-0.458984375,-2.53125,3.8125,5,-1.2421875,-2.109375,-0.50390625,-2.734375,-4.90625,1.0234375,2.421875,-3.34375,-10.125,6.46875,3.671875,5.40625,1.546875,-2.59375,3.8125,-1.6953125,3.703125,-0.423828125,0.82421875,1.515625,-7.59375,-2.40625,-2.0625,-5.0625,0.59375,-0.345703125,-4.75,1.4921875,6.25,-2.15625,-1.8671875,-2.703125,-3.9375,4.28125,-3.484375,-5.9375,1.984375,-7.4375,1.4609375,-1.9609375,3.265625,-5.875,1.8359375,-0.017333984375,2.046875,-0.5859375,-0.671875,-2.328125,1.1953125,-2.65625,3.625,0.7890625,3.9375,-0.365234375,2.90625,-1.2421875,0.314453125,-3.265625,1.6640625,1.7109375,0.60546875,0.384765625,2.296875,-2.28125,-0.8046875,-1.0546875,1.046875,2.796875,0.61328125,-0.625,0.10693359375,4.21875,-0.6484375,2.03125,-2.3125,-0.173828125,-1.015625,-0.224609375,0.74609375,-0.86328125,0.0145263671875,0.1318359375,1.7109375,1.421875,0.486328125,-0.19921875,0.140625,1.2734375,1.015625,1.5625,-1.65625,-0.45703125,-0.435546875,-0.0206298828125,1.828125,1.734375,-2.734375,1.65625,-2.09375,-0.6875,-0.2421875,2.125,1.1015625,0.1064453125,1.59375,-1.875,1.828125,0.15234375,-1.2421875,1.25,-0.765625,-2.265625,2.34375,-2.109375,-0.921875,0.6640625,-1.2734375,-1.4765625,-0.73828125,2.21875,-0.84375,1.328125,-1.171875,-0.181640625,0.306640625,-1.171875,0.279296875,0.94140625,1.171875,-3.921875,3.15625,1.2421875,0.52734375,-0.1630859375,1.0390625,-1.46875,-0.08447265625,1.0390625,-0.37109375,0.921875,1.859375,-1.8046875,0.54296875,-0.8203125,-1.09375,1.1640625,1.515625,0.54296875,-1.65625,-1,1.5234375,1.4453125,-1.1953125,0.359375,-0.062255859375,-2.09375,3.03125,1.21875,-3.15625,-0.357421875,-0.169921875,0.546875,-0.73828125,-0.126953125,1.046875,-2.75,-0.2314453125,0.2421875,0.306640625,-1.1328125,1.8984375,0.00469970703125,3.9375,0.8515625,1.1328125,1.1875,1.3984375,2.046875,-1.3515625,0.25390625,-0.9921875,3.234375,-0.373046875,0.8828125,1.3828125,-1.921875,-0.484375,-0.81640625,0.61328125,1.4296875,-0.70703125,-0.404296875,2.53125,1.625,0.494140625,2.375,-2.03125,0.33984375,0.291015625,-0.68359375,-1.625,1.625,-0.478515625,0.349609375,-2.0625,-1.25,-0.1484375,-0.44140625,0.67578125,0.3671875,0.4921875,0.236328125,1.1953125,0.5078125,-2.375,1.3671875,-0.341796875,0.6328125,-1.7265625,-1.328125,0.84375,-0.08935546875,1.0625,0.90625,1.984375,2.828125,1.109375,-1.3671875,1.03125,1.0625,1.75,0.263671875,-1.234375,-0.09228515625,-0.13671875,0.271484375,0.58203125,-0.9375,-1.28125,0.4609375,-0.95703125,-0.1552734375,-1.5703125,3.375,-0.9609375,-1.1796875,-0.419921875,-1.5,0.58984375,-1.3125,1,-1.578125,2.484375,1.34375,3.34375,1.4296875,-0.671875,-0.984375,0.30859375,0.72265625,-0.337890625,-0.06982421875,-1.125,-0.44921875,-0.62890625,5.40625,0.263671875,1.0390625,-2.03125,3.296875,0.68359375,-0.10986328125,-1.078125,-0.2412109375,-2.078125,-0.13671875,-1.4375,-1.390625,0.29296875,-1.1484375,-4.0625,-2.703125,-0.302734375,0.77734375,-1.640625,-0.0390625,3.890625,0.375,1.2890625,1.5,2.640625,0.19140625,-1.78125,-0.5859375,1.6328125,-1.234375,2,0.8125,-1.9453125,-2.78125,-0.3671875,-2.328125,-1.9453125,-0.59375,-0.8046875,1.9921875,-0.265625,-0.03515625,-1.3125,-1.5234375,-3.03125,-0.458984375,-0.1279296875,2.375,1.53125,0.67578125,-0.55078125,-0.4296875,0.515625,-1.75,0.6640625,-1.65625,4.25,-0.326171875,-1.4296875,2.53125,0.396484375,3.140625,0.859375,-1.3671875,-1.8828125,-0.828125,0.45703125,0.7109375,3.0625,-0.2578125,0.6328125,0.57421875,-0.85546875,0.5625,1.0234375,-0.296875,-4.84375,-1.578125,-0.486328125,2.59375,-1.2109375,0.09765625,2.59375,-0.87109375,-0.7890625,-1.7421875,-2.34375,-0.2490234375,-0.82421875,0.8046875,2.078125,-0.7265625,-0.10400390625,-0.703125,-1.046875,0.46875,-1.7734375,1.09375,-0.30859375,0.0181884765625,0.2734375,-2.703125,-0.470703125,0.67578125,-1.921875,-1.0078125,1.6328125,0.2021484375,1.359375,1.6796875,-1.6015625,1.5703125,0.6484375,-2.859375,-0.63671875,-0.8359375,1.34375,0.0556640625,0.4375,1.765625,-1.1484375,-1.90625,-1.453125,0.57421875,0.84375,-0.349609375,0.251953125,-0.0927734375,0.416015625,-0.40625,-2.71875,-0.48046875,0.4140625,-0.2109375,0.96484375,1.0859375,1.453125,1.15625,1.375,-0.478515625,1.375,-1.8828125,1.6484375,0.9921875,-2.171875,0.5859375,2.03125,-2.125,0.314453125,1.1796875,-0.4921875,-0.72265625,-0.80078125,0.5546875,-0.52734375,0.58203125,-0.52734375,1.9453125,1.71875,-0.328125,1.453125,-2.203125,-2.09375,-2.625,0.2177734375,-0.82421875,0.3359375,-2.203125,1.375,-1.7578125,-0.072265625,-0.4765625,-0.38671875,-1.9453125,1.5625,1.7578125,0.4453125,0.640625,0.0255126953125,-0.5703125,3.796875,-1.0703125,-0.1201171875,0.93359375,1.15625,-2.078125,3.484375,0.5234375,2.109375,0.0037078857421875,1.3359375,-0.796875,1.25,0.1455078125,0.86328125,0.478515625,1.828125,0.31640625,-0.296875,-0.154296875,-1.53125,-1.1640625,0.6484375,1.0703125,-5.375,0.86328125,0.890625,0.48828125,0.84765625,-2.828125,1.1015625,0.4765625,3.296875,-0.00408935546875,-0.40234375,3.421875,0.61328125,-1.46875,1.1875,0.953125,0.0771484375,-2.78125,-1.171875,-0.86328125,2.9375,-1.0703125,0.1015625,-0.279296875,-0.90625,3.046875,0.6796875,-1.6640625,1.453125,0.443359375,-0.439453125,-1.453125,-3.40625,-0.1689453125,1.71875,-0.9453125,2.234375,0.158203125,0.87109375,0.66796875,-1.640625,1,0.265625,0.267578125,-0.90625,1.75,-0.2041015625,-1.59375,1.65625,-1.1484375,-1.78125,2.421875,1.6953125,-2.328125,0.027587890625,-0.494140625,-0.3203125,-0.01953125,0.58203125,-2.28125,0.546875,0.62109375,0.90625,-0.921875,-1.53125,2.484375,1.890625,2.953125,2.359375,-0.90234375,0.171875,-2.234375,0.33984375,-0.45703125,-0.87109375,0.08251953125,1.8671875,-1.0078125,1.5703125,-0.30078125,0.921875,-1.8046875,1.609375,2.703125,0.92578125,0.40625,-0.26171875,-0.322265625,-1.8671875,-0.5,-2.296875,0.62109375,0.6953125,1.1640625,0.1376953125,-1.4296875,1.5390625],\"index\":0,\"object\":\"embedding\"},{\"embedding\":[-2.28125,-0.7734375,-0.8359375,-2.3125,3.046875,4.125,-1.0390625,-2.890625,0.0103759765625,1.9296875,0.1015625,1.75,2.4375,2.015625,5.09375,1.203125,-2.140625,-2.828125,-1.328125,-4.6875,1.0078125,6.8125,0.578125,-4.71875,-0.80859375,-6.25,1.578125,4.25,4.46875,-1.0078125,8,-2.3125,2.546875,-0.00555419921875,1.5625,-1.8671875,-2.375,-2.53125,5.25,-0.69140625,-2.96875,-0.68359375,1.6171875,2.96875,-3.015625,-1.734375,0.4140625,-2.9375,2.53125,-1.6640625,-4.5625,-1.9296875,3.234375,-2.734375,2.359375,-4.125,-3.046875,4.5,-5.875,-2.984375,-1.8515625,-2.8125,-0.7734375,0.46484375,1.3984375,5.28125,0.68359375,-1.3359375,0.51171875,8.625,-0.055908203125,3.578125,6.5,-2.390625,6.34375,5.5625,0.7265625,1.578125,-2.921875,4.90625,-2.953125,-0.62890625,2.453125,3.46875,4.5625,2.671875,-1.9140625,0.859375,-3.03125,1.703125,1.96875,0.59375,-1.4140625,-3.140625,-1.2109375,1.2890625,-3.21875,-6.5625,-6.78125,2.765625,-0.78515625,-0.3515625,1.8125,-4.53125,-5.03125,2.171875,-1.8515625,-5.46875,-1.78125,0.380859375,2.640625,1.65625,3.640625,-2.140625,2.46875,1.21875,4.28125,-2.796875,-4.40625,2.796875,-2.0625,-1.9765625,4.28125,-0.6796875,4.4375,4.28125,-4.03125,-0.01416015625,5.53125,-1.4609375,7.25,3.578125,3.6875,-2.375,-8.0625,-4.71875,-1.9453125,3.71875,4.3125,4.40625,-5.03125,3.21875,-3.734375,-6.625,4.1875,-3.4375,-6.4375,-3.15625,3.859375,-1.9140625,-1.78125,1.8046875,0.5,2.3125,-1.2421875,-4.375,4.0625,3.875,0.1259765625,-1.0546875,2.015625,3.328125,1.1484375,1.7265625,1.8046875,-0.462890625,-5.625,3.6875,-1.0390625,2.5625,0.90625,10.4375,4.28125,-4.5625,1.9765625,8.625,-1.328125,8.625,1.4609375,2.203125,0.81640625,-0.640625,-2.90625,4.53125,-2.15625,1.5,0.12255859375,-5.6875,3.140625,1.2890625,1.578125,1.5625,2.71875,-1,-4.84375,-1.8671875,3.484375,-2.578125,3.4375,0.1025390625,-1.40625,-7.375,1.4921875,1.5546875,-4.71875,-3.765625,2.703125,-1.71875,3.078125,-0.380859375,2.265625,0.24609375,3.21875,-2.0625,7.65625,2.640625,2.734375,2.046875,1.8359375,2.46875,4.53125,3.484375,1.8359375,-2.078125,-0.83984375,2.03125,5.8125,0.439453125,3.75,8.6875,0.251953125,0.408203125,6.84375,-2.515625,-1.78125,-3.578125,-3.78125,1.6015625,-0.279296875,2.671875,-5.65625,-4.0625,-2.328125,2.984375,3.515625,-3.359375,-2.34375,-2.703125,-0.51171875,-6.4375,1.484375,3.671875,-9.0625,1.8828125,5.625,3.96875,1.984375,1.265625,-0.33203125,-4.125,0.333984375,-2.4375,-5.875,-0.58203125,1.890625,-2.390625,5.09375,-1.5546875,3.515625,-0.7421875,5.1875,-2.28125,-0.0927734375,-3.046875,-4.3125,8.8125,-0.232421875,-1.90625,1.0703125,-3.078125,-3.5625,-10.25,2.5,1.1171875,4.96875,-2.921875,1.40625,0.40234375,-3.640625,12.75,3.90625,-1.8203125,1.9921875,-0.63671875,-6.03125,-1.984375,-2.046875,2.046875,-5.59375,1.84375,3.6875,4.5,-1.9296875,3.4375,-1.7421875,-0.9296875,-1.109375,-4.5625,-1.9375,2.671875,-3.765625,2.34375,9.625,-4.75,2.03125,-2.109375,-6.1875,4.75,-0.03662109375,-0.11376953125,-2.140625,-5.125,-1.9921875,-2.78125,-1.4296875,-6.65625,4.96875,-0.984375,5.375,0.97265625,3,3.296875,-4.1875,-5.03125,8.4375,-1.5,3.296875,5.71875,0.55078125,0.68359375,-3.515625,-4.6875,2.46875,-5.46875,0.953125,5.71875,3.328125,-1.640625,1.0234375,-6.21875,2.40625,2.328125,-0.68359375,6.53125,6.90625,-2.265625,2.78125,1.9140625,-0.71484375,-2.28125,-0.2294921875,-1.078125,6.34375,1.1875,-3.890625,-3.796875,-0.5859375,5.03125,-2.375,0.7734375,-1.21875,-4.15625,2.59375,-1.15625,3.6875,0.91796875,0.90625,-1.8046875,-5.125,0.087890625,-2.625,0.29296875,-1.7734375,-3.28125,4.25,1.515625,-0.484375,1.59375,0.67578125,-3.53125,-0.46484375,0.59765625,-1.15625,0.65625,2.5625,-0.5703125,-0.984375,1.5546875,-0.3828125,-2.21875,1.0546875,-1.2734375,2.40625,-6.9375,-0.6484375,-0.2490234375,-2.125,-8.375,-0.4765625,1.0703125,-3.78125,2.71875,1.96875,-1.2578125,-3.0625,4.4375,1.421875,1.8671875,-6.90625,2.15625,-1.8828125,3.328125,2.140625,-1.7421875,0.59375,-1.4296875,-2.765625,4.375,3.546875,-0.69921875,3.453125,0.68359375,-3.265625,-3.625,0.1630859375,-4.90625,4.75,-0.236328125,-1.859375,5.21875,2.203125,-1.5,1.625,0.98828125,-6.28125,-4.78125,2.96875,3.171875,-3.078125,-3.96875,0.470703125,-1.4296875,-4.4375,3.078125,3.84375,-1.1171875,-2.8125,3.40625,4.375,-2.203125,0.0830078125,1.1171875,0.52734375,2.703125,-1.9375,-3.140625,-0.1103515625,0.130859375,4.71875,-5.8125,-6.84375,3.015625,-2.875,0.2001953125,1.15625,4.5625,0.46875,-1.8984375,-1.9296875,-3.0625,-3.46875,-2.828125,3.53125,-1.078125,-2.53125,-2.90625,0.29296875,8.3125,1.90625,0.369140625,-2.375,-0.11572265625,2.453125,-1.71875,0.50390625,4.4375,7.90625,-4.03125,-0.63671875,3.53125,-8.125,0.94921875,-1.375,-1.15625,-0.94921875,2.3125,2.1875,-6.25,-0.7890625,0.0115966796875,5.03125,-3.453125,-3.828125,5.15625,-4.8125,-3.09375,1.859375,-0.6875,4.0625,1.296875,-1.34375,2.875,2.984375,2.65625,1.8203125,-2.53125,-3.640625,-3.3125,1.2890625,2.265625,-2.234375,2.296875,4,-5.4375,0.90234375,-2.25,-0.6953125,-0.212890625,-0.515625,5.90625,2.125,2.25,-6.09375,1.2578125,0.50390625,-0.416015625,-0.7421875,-1.1484375,6.71875,-0.5,-0.2294921875,0.94921875,2.09375,-1.1953125,1.640625,-3.796875,-2.453125,-3.109375,-1.796875,-1.0234375,-4.03125,-5.5,4.4375,6,-1.234375,-1.6796875,2.171875,5.5,3.984375,-0.84375,1.515625,3.421875,-2.5,0.23828125,-5.40625,2.609375,-7.84375,-2.53125,-1.6875,2.921875,3.75,-4.15625,3.765625,-2.578125,2.4375,-1.4375,4.4375,-10.5625,2.046875,-2.15625,-2.796875,-2.28125,-0.57421875,3.171875,-0.44921875,2.109375,1.3671875,-0.75,3.953125,5.46875,-1.5,1.765625,2.1875,2.46875,-0.5859375,2.515625,-2.125,-8.25,1.3125,-1.1484375,1.09375,7.5625,1.9375,-1.7734375,2.46875,0.88671875,-1.5703125,-1.7265625,4.0625,3.015625,-1.546875,4.25,-3.90625,5.40625,-3.28125,1.7265625,-3.265625,-6.15625,0.279296875,1.9296875,-5.5625,-4.09375,2.859375,0.216796875,5.78125,3.421875,-5.375,1.21875,-0.41796875,1.109375,2,0.30078125,-0.03759765625,-4.75,3.921875,4.1875,-2.40625,7.03125,-1.5703125,-1.6484375,-1.1171875,2.40625,-1.7734375,0.373046875,1.84375,0.287109375,-0.78125,-3.484375,0.96484375,0.5703125,-6.625,-7.21875,1.7265625,-1.7734375,7.0625,0.73046875,-0.859375,-3.15625,2,1.5546875,6.375,3.3125,3.765625,4.5,3.765625,-2.390625,2.671875,-3.6875,-6.09375,7,-6.53125,-1.8515625,1.015625,0.859375,-0.2578125,-1.0234375,-0.3515625,-0.71484375,-3.484375,-6.09375,-2.359375,-1.875,2.015625,-1.6484375,2.203125,0.57421875,-4.09375,-0.5703125,-1.6484375,-1.6875,-1.6640625,4.15625,-5.625,1.484375,5.71875,2.046875,-1.5234375,4.15625,3.09375,-0.47265625,-4.78125,0.7109375,-6.875,1.6015625,1.46875,-0.6015625,0.50390625,-8,2.03125,-2.4375,3.5,-0.671875,-0.05078125,-1.265625,-3.296875,-1.3984375,-0.91796875,-5.40625,-0.171875,1.6953125,1.125,-1.8359375,0.671875,3.078125,-0.52734375,0.384765625,-1.125,2.046875,0.40625,2.34375,-4.78125,-2.90625,1.28125,0.9140625,-2.03125,6.53125,0.91796875,0.79296875,3.546875,1.7265625,-5.5,-5.78125,3.921875,-2.8125,-1.796875,-3.25,2.421875,-1.359375,6.53125,-2.21875,-5.53125,-3.703125,1.6484375,3.15625,-2.609375,-3.09375,4.78125,1.8359375,2.765625,-2.15625,-7.5,1.609375,0.98828125,-0.146484375,-1.140625,8.625,-1.9296875,-0.4765625,-4.4375,-3.234375,2.046875,0.875,2.046875,-0.76171875,-1.2734375,0.69921875,0.4765625,-2.34375,-0.55078125,0.6015625,-2.546875,1.75,0.07177734375,4.875,-2.53125,0.3984375,-1.2734375,-0.50390625,-0.10009765625,4.3125,8.75,-1.765625,-0.96875,0.35546875,2.984375,-3.59375,6.6875,1.3515625,7.75,-1.1640625,0.25,1.03125,0.375,-2.171875,4.59375,-5.25,-2.84375,-1.890625,1.21875,-2.5625,0.671875,-3.984375,-0.498046875,4.40625,-0.455078125,-0.007568359375,2.609375,0.79296875,-0.201171875,-3.09375,-1.3125,-4.71875,-2.515625,-0.14453125,2.03125,-3.03125,-0.4921875,-0.33984375,5.84375,-0.357421875,-1.4453125,-2.59375,1.53125,1.859375,1.171875,-0.8046875,0.255859375,0.58984375,3.3125,-1.015625,-4.34375,-0.94921875,8.4375,4.21875,-6.875,1.5703125,-0.43359375,1.4453125,-4.8125,-1.4609375,-2.15625,-1.4921875,-4.1875,1.1328125,0.419921875,-3,-0.06494140625,4.5,-1.2890625,-0.15625,3.46875,4.0625,0.478515625,2.96875,-2.125,4.375,2.21875,-2.09375,-5.96875,-1.703125,0.48046875,-2.75,-1.4140625,2.03125,6.15625,0.55859375,2.625,-1.0625,2.28125,-1.6953125,3.78125,5.125,-4.59375,-2.703125,-2.3125,-9.5625,-4.03125,-1.7421875,-2.921875,-5.34375,-4.25,-0.86328125,-1.2421875,-8,0.0966796875,-2.234375,-3.265625,1.4453125,2.953125,1.7578125,-5.75,3.125,4.125,2.578125,2.546875,0.84765625,5.46875,-0.050537109375,-2.96875,1.4453125,-3.4375,4.15625,-1.03125,3.546875,6.25,-0.453125,-4.96875,4.78125,2.96875,5.53125,-7.375,-2.625,-0.337890625,-1.671875,-0.458984375,-1.7578125,2.546875,-4.5,-5.5,1.078125,-3.203125,1.2265625,4.6875,-0.8046875,6.78125,1.6328125,0.419921875,2.140625,2.71875,0.62109375,0.169921875,1.7421875,-5.9375,3.234375,-2.171875,3.265625,-0.296875,-1.5234375,2.734375,-0.7578125,-0.310546875,2.8125,2.734375,10.3125,0.515625,4,-2.3125,0.63671875,-1.7265625,-0.2392578125,2.25,2.015625,0.79296875,-1.4765625,0.7890625,-0.44921875,0.478515625,-0.4609375,-13.25,-1.9609375,-7.25,-1.9296875,7.0625,-2.1875,-1.9921875,1.4296875,2.6875,3.484375,5.125,-0.58984375,3.375,-0.60546875,0.80859375,5.96875,-4.25,1.03125,3.359375,2.546875,5.21875,0.154296875,-0.44921875,-3.203125,8,2.25,-1.4140625,0.8359375,2.796875,-1.3046875,-2.34375,3.09375,-3.171875,2.96875,-4.9375,0.5859375,4.15625,0.65625,-3.890625,-3.4375,-2,-0.62890625,1.3828125,1.375,-2.59375,0.18359375,0.94921875,-4.1875,3.328125,-0.59375,0.140625,-5.53125,1.03125,4.65625,0.703125,-0.109375,-1.8515625,1.4453125,-0.8984375,4.3125,2.78125,-2.734375,0.2734375,2.21875,1.7421875,-0.125,1.03125,1.1328125,2.921875,-3.09375,-0.353515625,-0.44140625,-1.625,1.4765625,-3.1875,1.6640625,3.203125,1.3984375,-3.984375,2.21875,0.79296875,-0.11669921875,2.96875,-5.125,-1.9921875,-1.1015625,-0.71484375,-4.0625,-0.9140625,-4.375,-0.1455078125,5.46875,-5,3.4375,-2.515625,8.1875,0.1298828125,-1.421875,1.2890625,-2.828125,2.59375,-3.390625,-1.234375,3.484375,-0.92578125,2.125,-3.546875,1.8984375,-2.078125,-0.46484375,6.09375,-3.953125,-1.9765625,0.7421875,3.21875,-5.0625,-3.296875,0.1611328125,0.8515625,0.009765625,-1.8984375,1.4765625,-2.03125,4.4375,-4.75,3.390625,-4.65625,-3.90625,0.28125,0.07568359375,7.90625,4.25,-3.796875,-3.421875,-0.6015625,-7.0625,-3.421875,-3.859375,6.65625,-0.52734375,0.96875,2.078125,2.390625,-0.01031494140625,1.46875,-2.96875,3.203125,5.28125,0.294921875,3.046875,2.1875,-1.125,-4.40625,0.3125,-3.171875,7.0625,3.0625,0.404296875,3,-1.8984375,1.484375,-1.03125,-1.0625,-2.828125,2.171875,1.71875,-2.5,-3.28125,1.046875,-3.859375,0.72265625,-5.40625,-2.578125,-5.3125,2.765625,2.3125,-0.81640625,-0.7578125,4.4375,0.318359375,3.328125,-5.53125,-3.890625,3.8125,0.9765625,0.333984375,2.84375,-0.6796875,-5.03125,-0.9375,0.201171875,1.9140625,-4.1875,-3.609375,3.328125,2.46875,0.283203125,-3.9375,-4.40625,-3.453125,2.390625,4.1875,-0.96484375,0.353515625,0.06005859375,-1.53125,2.171875,-2.65625,4.5,-3.109375,-4.15625,-0.47265625,0.734375,3.578125,-3.203125,-1.0703125,1.4296875,-3.4375,0.7578125,1.2734375,-0.11279296875,-1.9453125,3.171875,-2,-3.65625,-5.4375,5.78125,-2.0625,0.45703125,-3.875,-2.65625,-3.1875,-1.421875,-0.6640625,1.7421875,0.0703125,5.78125,-0.63671875,2.8125,0.478515625,-0.8828125,0.0712890625,3.453125,-0.271484375,-2.90625,1.8359375,-4.59375,-4.65625,0.7578125,-8.0625,-2.0625,2.90625,-2.40625,2.671875,-2.671875,2.375,-1.1015625,-2.21875,-1.8203125,-0.8203125,0.83984375,5.375,2.171875,0.2216796875,0.38671875,1.8984375,0.859375,-1.109375,-1.8515625,-0.25,5.34375,0.62109375,2.765625,-3.359375,-2.34375,4.46875,-0.59375,-3.75,0.8984375,-0.357421875,0.6640625,4.5625,0.9609375,-3.796875,-2.9375,-6.15625,4.03125,0.73828125,1.828125,-4.625,1.5,-3.0625,0.1748046875,2.03125,-6.5625,-2.546875,3.328125,2.828125,5.46875,1.328125,-2.421875,-4.53125,2.203125,-0.396484375,-1.6171875,-2.234375,-1.7265625,-0.96875,-3.765625,4.125,-2.515625,4.25,-1.3359375,-2.8125,-0.8671875,0.61328125,-0.203125,0.47265625,-0.353515625,-0.88671875,4.0625,-0.3515625,7,2.171875,-4.0625,4.59375,2.515625,0.412109375,-1.5625,3.75,-1.109375,-2.3125,3.921875,2.890625,-4.0625,4.96875,2.125,3.375,-3.46875,-2.1875,-0.9921875,4.5625,0.287109375,1.28125,-4.34375,0.1630859375,4.0625,-0.1884765625,0.8671875,-1.765625,0.3046875,0.65234375,0.52734375,2,1.921875,3.4375,-0.52734375,1,-0.92578125,-1.2265625,2.328125,-0.1328125,-0.703125,-1.8828125,3.21875,-1.6953125,-1.875,-6,1.2421875,-3.46875,2.21875,3.1875,2.875,2.234375,-2.828125,-1.625,-2.640625,-5.25,-3.140625,1.75,1.09375,-1.75,1.875,-0.1181640625,2.546875,5.84375,0.130859375,4.6875,-3.109375,2.5,1.140625,0.875,0.046630859375,4.3125,-1.8203125,-2.21875,3.640625,-4.46875,3.71875,-4.53125,-3.078125,-0.63671875,-0.10986328125,2.640625,6.625,-4.5625,-3.953125,5.21875,1.328125,4.59375,3.78125,-2.078125,-1.484375,0.79296875,1.3515625,5.46875,0.93359375,2.953125,-2.734375,6.9375,5.65625,0.90625,2.359375,0.166015625,-2.6875,-6.4375,5.125,1.3984375,1.984375,-2.375,1.6875,3.109375,0.1533203125,3.640625,-5.5,0.8671875,1.2109375,0.90625,0.5234375,-3.15625,0.103515625,2.640625,0.33203125,-1.6875,5.84375,0.97265625,4.125,-0.72265625,3.34375,2.328125,3.703125,-2.03125,1.5234375,-3.46875,3.578125,-1.3984375,2.15625,-5.5,1.0546875,3.640625,4.3125,-1.625,-3.5625,2.21875,0.275390625,-0.5,-4.46875,4.21875,3.59375,2.5625,-6.9375,-3.328125,-0.05029296875,0.2060546875,1.234375,-3.484375,1.171875,1.6796875,-4.625,-3.265625,1.296875,1.625,-5.65625,-6.0625,-3.203125,1.65625,1.3203125,3.1875,3.21875,-0.8203125,3.40625,-0.55078125,3.046875,4.28125,-1.1328125,1.5546875,0.9375,-2.75,4.125,-0.263671875,-2.671875,1.5546875,-0.50390625,-2.140625,0.50390625,-2.296875,-1.0703125,-4.21875,-0.85546875,2.328125,-1.09375,5.125,-3.96875,0.30078125,3.609375,-1.4375,-2.28125,-2.65625,0.5703125,-2.921875,-2.578125,-1.9140625,3.609375,2.984375,2.046875,0.58203125,-0.6015625,-3.265625,-6.40625,-5.65625,3.578125,-2.515625,2.859375,0.439453125,-4.25,2.078125,2.8125,1.78125,-0.1640625,-0.55859375,2.765625,4.59375,0.455078125,-1.7265625,-0.466796875,3.609375,-4.5625,-3.78125,0.515625,1,-3.171875,2.28125,-3.125,-1.8359375,0.79296875,4.5,-0.5078125,-2.859375,-1.75,-2.40625,-2.875,-3.03125,-2.859375,2.5625,1.859375,3.296875,0.1689453125,-0.421875,-5,3.71875,16.875,0.9375,-4.71875,2.421875,-3.140625,2.65625,3.171875,4.8125,-1.7109375,-1.96875,-2.1875,1.765625,0.01031494140625,1.4140625,-2.140625,1.7421875,1.9921875,-0.48828125,-4.125,-1.9765625,-1.328125,0.84765625,-0.7578125,2.96875,0.408203125,2.265625,-0.734375,-0.259765625,0.2333984375,-3.234375,-4.46875,-4.4375,2.265625,-1.7578125,4.75,-4.25,5.375,0.1845703125,-2.9375,-2.09375,-3.296875,-3.171875,1.0234375,-0.75,-1.9453125,4.34375,-0.72265625,1.09375,0.37890625,-0.337890625,-3.546875,-3.046875,-2.6875,7.25,0.62890625,-5.71875,-1.546875,-4.84375,-4.5625,0.58984375,2.796875,-2.328125,1.6328125,1.453125,-1.828125,-2.171875,-1.953125,0.85546875,3,-5.125,-5.625,0.13671875,1.5546875,3.359375,2.796875,-4.0625,1.5703125,5.3125,2.6875,0.69140625,-0.75,1.4453125,-1.3828125,-2.5,-0.91015625,1.4609375,-4.03125,1.109375,1.4453125,-4.875,11.25,-8.625,4.8125,4.0625,-4.75,-0.1865234375,2.796875,1.796875,-1.6796875,-0.169921875,2.953125,2.453125,3.359375,-0.306640625,6.09375,1.5234375,0.388671875,0.73828125,2.9375,3.578125,2.4375,2.9375,-0.828125,-1.9609375,1.3046875,1.7734375,-2.484375,-3.46875,-1.4609375,-4.4375,6,1.6171875,-2.765625,-1.2578125,-10.5,-3.421875,-2.328125,-5.84375,4.5,-2.65625,2.46875,3.421875,-0.609375,-1.078125,-2.53125,-5,2.296875,4.0625,0.208984375,-0.3984375,-6.0625,2.84375,3.546875,-3.984375,-2.09375,1.4453125,-3.265625,3.296875,-0.1923828125,4.9375,-3.578125,3.9375,2.03125,-2.546875,-5.8125,3.171875,-3.765625,-2.234375,-5.3125,-2.453125,-2.078125,-3.328125,-0.6171875,-0.35546875,-2.078125,-1.03125,1.6171875,-0.60546875,-3.15625,2.921875,2.96875,-4.375,-2.625,0.58203125,0.73046875,-4.28125,1.1875,5.1875,-0.54296875,1.5,0.55078125,0.078125,-0.3203125,-4.34375,0.81640625,1.71875,-4.03125,-0.71875,-1.359375,-2.828125,-2.4375,-2.78125,-3.375,3.875,3.59375,-5.0625,1.9609375,-0.34765625,0.014892578125,-1.4453125,-1.546875,6.4375,2.234375,-1.6484375,5.59375,1.03125,-4.15625,-2,-2.046875,-1.1484375,-1.2734375,6.3125,1.2578125,2.375,-5.90625,7.53125,2.453125,1.7265625,-0.43359375,2.34375,1.6796875,-3.71875,-5.40625,2.46875,2.75,3.84375,-4.59375,0.6328125,0.53515625,0.53125,-4.28125,1.90625,-0.259765625,0.482421875,-3.140625,-7.59375,-0.109375,0.90625,-1.8828125,1.5234375,4.25,-2.96875,1.3828125,0.95703125,-0.58984375,3.640625,3.28125,-2.828125,1.90625,-0.1904296875,2.625,-2.34375,1.4921875,-3.71875,-4.96875,-3.109375,-1.765625,1.8828125,-2.625,0.67578125,-0.357421875,-4.1875,2.109375,-2.25,1.125,1.09375,0.2578125,-6.25,3.984375,5.1875,-4.15625,4.4375,-5.53125,-2.4375,-1.640625,2.21875,-1.9140625,-6.46875,2.0625,4.5,-3.390625,2.203125,3.546875,-1.625,-0.4453125,-2.25,5.3125,-1.015625,4.78125,-0.6953125,3.953125,3.9375,-1.28125,-0.061279296875,-5.125,0.470703125,-2.28125,-3.84375,5.53125,-1.921875,2.46875,5.21875,4.9375,-9,-1.96875,0.54296875,-0.1845703125,3.578125,3.109375,-1.3671875,1.0234375,0.028076171875,-0.30859375,4.4375,-0.9296875,-1.46875,-3.65625,4.96875,-0.1728515625,-4.0625,2.984375,2.609375,-4.15625,4.34375,-2.75,-2.6875,-0.6875,-0.1396484375,-5.625,1.8046875,2.6875,-0.92578125,3.4375,3.109375,1.203125,3.59375,-2.640625,-10.0625,0.0703125,2.75,5.3125,1.7265625,2.3125,0.0859375,-1.0625,3.640625,-4.5625,0.46875,-1.484375,-9.5,0.255859375,-4.15625,-1.609375,-3.453125,-1.4921875,-1.9453125,3.90625,1.3984375,-0.8515625,3.5,2.921875,0.453125,4.15625,-0.361328125,-3.578125,1.2734375,1.75,-5.28125,-1.90625,4.8125,3.578125,-2.203125,-2.0625,3.84375,-4.28125,-0.70703125,4.3125,4.28125,2.15625,-0.828125,-3.234375,2.84375,-2.546875,-2.828125,1.703125,-3.421875,2.453125,-1.4375,2.578125,1.296875,-2.640625,-2.03125,-4.15625,-2.71875,3.484375,0.28515625,0.9765625,-2.265625,-1.1171875,3.234375,3.5625,-2.359375,-2.109375,2.796875,-1.3515625,-4.28125,-1.0859375,1.0859375,-5.90625,-2.609375,2.734375,3.4375,-2.5625,-3.5625,-2.125,1.6171875,1.3046875,-0.8984375,-0.1318359375,-3.53125,2.65625,5.0625,-2.9375,-3.75,-1.6171875,-0.486328125,-5.03125,-3.609375,-0.1767578125,1.140625,-0.73046875,3.890625,-1.40625,0.47265625,4.4375,-3.65625,-3.21875,3.96875,3.359375,-3.203125,-1.46875,2.25,-3.375,1.03125,5.4375,-2.390625,-2.234375,0.41796875,-2.171875,-4.28125,2.34375,1.2265625,-3.734375,-7.875,5.96875,1.0703125,4.34375,4.125,-3.90625,4.0625,-4.6875,1.8828125,-1.265625,1.015625,1.3828125,-5.65625,-1.1875,-2.5,-3.5,0.5390625,-1.734375,-3.5625,0.66015625,8.0625,-1.328125,-2.59375,-2.953125,-3.515625,3.3125,-4.15625,-7.625,0.1181640625,-7.34375,1.734375,-2.1875,1.75,-5.59375,1.9140625,-1.078125,1.734375,-2.984375,0.27734375,-0.384765625,1.21875,0.54296875,4.6875,1.2109375,1.984375,-0.1484375,2.71875,0.0791015625,1.875,-1.453125,-0.4921875,1.21875,-1.234375,0.33203125,0.69921875,-2.734375,0.1708984375,-1.7578125,-0.263671875,-1.015625,1.7578125,2.9375,-0.640625,-0.291015625,-1.6875,1.703125,-4.5,1.3125,-1.796875,0.859375,-0.78515625,-1.0078125,1.9609375,-2.328125,1.6640625,1.015625,1.640625,0.01068115234375,-1.5,2.234375,2.6875,-0.031982421875,-2.328125,-1.8046875,-0.55859375,-1.7421875,1.7421875,0.55078125,-2.0625,2.9375,-1.640625,-0.41015625,0.890625,1.7265625,0.44140625,-1.6484375,2.40625,-1.8671875,1.2890625,1.0859375,-1.5234375,2.609375,0.63671875,1.03125,1.2734375,0.9765625,-2,0.64453125,0.2578125,-1.4375,-0.291015625,3.484375,-1.7265625,0.31640625,-1.078125,-0.5625,1.0859375,-0.8671875,1.2109375,0.15625,-0.396484375,-2.75,2.640625,-2.125,-1.2578125,-0.42578125,0.29296875,-0.5703125,0.8984375,0.08935546875,1.2109375,-0.29296875,2.28125,-0.73828125,2.171875,-0.020263671875,-0.2060546875,1.3359375,3.421875,-1.984375,0.7421875,-2.0625,-1.1328125,1.3203125,-0.3046875,1.15625,-0.93359375,-2,1.2421875,1.1328125,-2.984375,-0.734375,2.265625,-0.189453125,-1.1328125,-0.609375,1.2265625,-0.75390625,-0.38671875,0.419921875,-0.89453125,2,3.265625,-1.0625,2.5,-1.453125,0.396484375,0.73046875,1.046875,2.3125,0.07958984375,-2.34375,-0.9296875,2.71875,-1.4375,0.37109375,0.890625,-1.53125,-0.1396484375,1.3359375,0.5703125,1.640625,-0.06982421875,-1.859375,-0.330078125,-0.6796875,1.609375,1.65625,-1.6875,0.68359375,-1.8359375,-0.53125,-1.015625,2.765625,-1.7578125,-2.140625,-0.78515625,-1.1015625,-0.83203125,-0.498046875,0.11962890625,-0.1298828125,0.60546875,1.125,1.5,0.4296875,-0.609375,1.4375,-0.08056640625,0.68359375,-1.1875,-1.5234375,1.484375,1.2421875,2.34375,-1.359375,1.34375,0.9296875,0.8828125,-1.1796875,1.9453125,-0.5234375,0.314453125,0.010986328125,-0.1181640625,1.40625,2.21875,0.318359375,0.5859375,-0.1328125,1.40625,0.69921875,1.375,-1.3046875,-2.203125,-1.0078125,-1.4296875,-2.125,0.361328125,-0.0615234375,-1.3046875,-0.1904296875,0.034912109375,-0.86328125,1.375,1.1796875,1.5390625,-0.828125,-0.58203125,0.1787109375,-0.328125,0.25390625,0.8828125,-0.8046875,-0.78125,-1.1171875,-2.0625,1.578125,0.88671875,-1.09375,-0.2890625,2.0625,-1.5,1.0078125,-2.78125,0.55078125,-1.828125,-0.341796875,0.0859375,-3.265625,0.34765625,-0.12451171875,-2.15625,-3.078125,-1.75,-0.85546875,-2.375,-0.3203125,4,-0.81640625,-1.21875,2.03125,0.08203125,-1.0078125,-0.94921875,1.7578125,2.84375,-0.8203125,3.859375,0.349609375,-0.16015625,-1.3984375,-1.265625,0.52734375,-1.2890625,0.294921875,-0.84765625,-0.8046875,-1.6796875,-3.109375,0.05859375,-4.1875,-2.125,0.1337890625,0.90625,1.890625,-0.08447265625,-0.7421875,-0.56640625,-0.96875,2.796875,-0.267578125,0.18359375,1.4375,0.27734375,0.46875,-1.4140625,0.92578125,-0.84375,2.953125,-1.171875,-0.50390625,-2.65625,-1.5546875,-4.1875,1.453125,2.484375,0.421875,2.96875,1.3671875,-0.5546875,-2.5625,0.07421875,0.00909423828125,-4.75,-0.373046875,-0.7265625,0.07275390625,-1.4140625,-0.7109375,-0.1318359375,-0.609375,-1.328125,-0.51953125,-1.828125,-0.271484375,-2.28125,2.984375,1.7890625,1.875,2.3125,0.3125,-0.31640625,1.1875,2.359375,1.1484375,0.6953125,0.255859375,0.408203125,-1.09375,2.09375,0.337890625,0.4609375,-1.2265625,0.2275390625,1.1875,2.5625,1.734375,-0.76171875,0.85546875,0.328125,-1.9140625,-1.40625,0.31640625,0.296875,1.140625,0.333984375,1.03125,-1.2890625,0.416015625,-0.6875,0.9453125,1.7578125,-1.953125,1.109375,-0.134765625,0.1787109375,-1.5,1.203125,1.15625,1.8203125,-0.48046875,2.140625,1.1640625,0.48828125,1.8515625,2.609375,-0.361328125,1.421875,-0.86328125,1.953125,0.51953125,-2.484375,3.15625,-0.34375,-0.47265625,-0.56640625,1.2890625,1.359375,-0.60546875,-0.25,-0.38671875,2.015625,0.52734375,0.14453125,1.8828125,0.67578125,-0.546875,-0.77734375,-0.6015625,-1.09375,-2.328125,-1.0078125,-3.0625,-0.37109375,-0.9375,1.765625,-0.828125,-1.484375,-0.142578125,1.390625,-0.02099609375,1.3203125,1.6171875,-1.0859375,2.09375,0.154296875,0.1962890625,0.89453125,-0.97265625,-1.2421875,1.15625,0.82421875,-0.59765625,4.625,0.1962890625,2.28125,-0.65625,-1.0390625,-0.78515625,3.59375,-0.44921875,-0.4375,-1.6953125,1.140625,-0.296875,-1.25,-0.76953125,-1.3984375,-0.9765625,1.78125,-0.87109375,-3.234375,-2.171875,0.330078125,-1.875,0.48828125,-1.859375,-1.0390625,2.40625,1.734375,-0.63671875,0.216796875,1.125,-1.0234375,0.58984375,-0.4296875,0.3515625,1.6015625,-1.2109375,1.765625,0.5859375,2.796875,-3.921875,-0.298828125,2.171875,1.578125,-0.458984375,-1.015625,-0.51171875,2.109375,0.369140625,-0.018798828125,-0.50390625,-4.46875,0.0135498046875,-0.043212890625,-3.21875,-0.09423828125,0.4921875,1.2421875,0.6640625,-3.15625,0.73046875,-1.5078125,-1.6328125,3.46875,-0.55078125,-0.41796875,0.58203125,1.1640625,-0.83203125,-0.84765625,1.53125,0.17578125,-3.484375,-1.1015625,-0.1591796875,-0.875,0.59765625,0.01373291015625,0.099609375,0.546875,-0.36328125,-1.171875,-1.1328125,-0.33984375,-0.08056640625,1.015625,4,1.1484375,1.265625,1.2109375,-2.125,4.5625,-2.515625,-0.96484375,1.1015625,1.3515625,-1.1796875,3.921875,1.109375,0.2265625,-2,0.55859375,2.96875,0.765625,0.9453125,0.671875,1.28125,1.7421875,1.78125,-1,-1.8671875,1.5,-0.35546875,-2.5,0.012451171875,0.2578125],\"index\":1,\"object\":\"embedding\"}],\"model\":\"doubao-embedding-text-240715\",\"object\":\"list\",\"usage\":{\"prompt_tokens\":7,\"total_tokens\":7}}"), + new TypeReference>>() {}); + Assertions.assertEquals(2, lists.size()); + Assertions.assertEquals(2560, lists.get(0).size()); + } +} 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 index f32cc870559..2de785a1a8b 100644 --- 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 @@ -24,12 +24,17 @@ 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.apache.seatunnel.transform.nlpmodel.llm.remote.custom.CustomModel; +import org.apache.seatunnel.transform.nlpmodel.llm.remote.openai.OpenAIModel; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class LLMRequestJsonTest { @@ -58,4 +63,50 @@ void testOpenAIRequestJson() throws IOException { OBJECT_MAPPER.writeValueAsString(node)); model.close(); } + + @Test + void testCustomRequestJson() throws IOException { + SeaTunnelRowType rowType = + new SeaTunnelRowType( + new String[] {"id", "name"}, + new SeaTunnelDataType[] {BasicType.INT_TYPE, BasicType.STRING_TYPE}); + + Map header = new HashMap<>(); + header.put("Content-Type", "application/json"); + header.put("Authorization", "Bearer " + "apikey"); + + List> messagesList = new ArrayList<>(); + + Map systemMessage = new HashMap<>(); + systemMessage.put("role", "system"); + systemMessage.put("content", "${prompt}"); + messagesList.add(systemMessage); + + Map userMessage = new HashMap<>(); + userMessage.put("role", "user"); + userMessage.put("content", "${input}"); + messagesList.add(userMessage); + + Map resultMap = new HashMap<>(); + resultMap.put("model", "${model}"); + resultMap.put("messages", messagesList); + + CustomModel model = + new CustomModel( + rowType, + SqlType.STRING, + "Determine whether someone is Chinese or American by their name", + "custom-model", + "https://api.custom.com/v1/chat/completions", + header, + resultMap, + "{\"model\":\"${model}\",\"messages\":[{\"role\":\"system\",\"content\":\"${prompt}\"},{\"role\":\"user\",\"content\":\"${data}\"}]}"); + ObjectNode node = + model.createJsonNodeFromData( + "Determine whether someone is Chinese or American by their name", + "{\"id\":1, \"name\":\"John\"}"); + Assertions.assertEquals( + "{\"messages\":[{\"role\":\"system\",\"content\":\"Determine whether someone is Chinese or American by their name\"},{\"role\":\"user\",\"content\":\"{\\\"id\\\":1, \\\"name\\\":\\\"John\\\"}\"}],\"model\":\"custom-model\"}", + OBJECT_MAPPER.writeValueAsString(node)); + } } diff --git a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java index 11b240dd993..4e5d864369f 100644 --- a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java +++ b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java @@ -24,6 +24,10 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; +import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; +import org.apache.seatunnel.api.source.event.ReaderCloseEvent; +import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; import lombok.extern.slf4j.Slf4j; @@ -136,6 +140,7 @@ public void open() throws Exception { ThreadPoolExecutorFactory.createScheduledThreadPoolExecutor( parallelism, "parallel-split-enumerator-executor"); splitEnumerator.open(); + coordinatedEnumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); restoredSplitStateMap.forEach( (subtaskId, splits) -> { splitEnumerator.addSplitsBack(splits, subtaskId); @@ -147,6 +152,10 @@ public void open() throws Exception { entry -> { try { entry.getValue().open(); + readerContextMap + .get(entry.getKey()) + .getEventListener() + .onEvent(new ReaderOpenEvent()); splitEnumerator.registerReader(entry.getKey()); } catch (Exception e) { throw new RuntimeException(e); @@ -203,6 +212,7 @@ public void close() throws IOException { for (Map.Entry> entry : readerMap.entrySet()) { readerRunningMap.get(entry.getKey()).set(false); entry.getValue().close(); + readerContextMap.get(entry.getKey()).getEventListener().onEvent(new ReaderCloseEvent()); } if (executorService != null) { @@ -211,6 +221,7 @@ public void close() throws IOException { try (SourceSplitEnumerator closed = splitEnumerator) { // just close the resources + coordinatedEnumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); } } diff --git a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java index 4cc1bfd1418..ed794a5b6cb 100644 --- a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java +++ b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java @@ -23,6 +23,10 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; +import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; +import org.apache.seatunnel.api.source.event.ReaderCloseEvent; +import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; import org.slf4j.Logger; @@ -115,7 +119,9 @@ public void open() throws Exception { splitEnumerator.addSplitsBack(restoredSplitState, subtaskId); } reader.open(); + readerContext.getEventListener().onEvent(new ReaderOpenEvent()); parallelEnumeratorContext.register(); + parallelEnumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); splitEnumerator.registerReader(subtaskId); } @@ -170,6 +176,8 @@ public void close() throws IOException { if (reader != null) { LOG.debug("Close the data reader for the Apache SeaTunnel source."); reader.close(); + readerContext.getEventListener().onEvent(new ReaderCloseEvent()); + parallelEnumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java index 4a720e347b2..2ebbcba4f91 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java @@ -66,10 +66,7 @@ public SinkWriter, FlinkWriterState> if (states == null || states.isEmpty()) { return new FlinkSinkWriter<>( - sink.createWriter(stContext), - 1, - catalogTable.getSeaTunnelRowType(), - stContext.getMetricsContext()); + sink.createWriter(stContext), 1, catalogTable.getSeaTunnelRowType(), stContext); } else { List restoredState = states.stream().map(FlinkWriterState::getState).collect(Collectors.toList()); @@ -77,7 +74,7 @@ public SinkWriter, FlinkWriterState> sink.restoreWriter(stContext, restoredState), states.get(0).getCheckpointId() + 1, catalogTable.getSeaTunnelRowType(), - stContext.getMetricsContext()); + stContext); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java index 725bf606f93..8de831aee17 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SupportResourceShare; +import org.apache.seatunnel.api.sink.event.WriterCloseEvent; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -53,6 +54,8 @@ public class FlinkSinkWriter private final org.apache.seatunnel.api.sink.SinkWriter sinkWriter; + private final org.apache.seatunnel.api.sink.SinkWriter.Context context; + private final Counter sinkWriteCount; private final Counter sinkWriteBytes; @@ -67,9 +70,11 @@ public class FlinkSinkWriter org.apache.seatunnel.api.sink.SinkWriter sinkWriter, long checkpointId, SeaTunnelDataType dataType, - MetricsContext metricsContext) { + org.apache.seatunnel.api.sink.SinkWriter.Context context) { + this.context = context; this.sinkWriter = sinkWriter; this.checkpointId = checkpointId; + MetricsContext metricsContext = context.getMetricsContext(); this.sinkWriteCount = metricsContext.counter(MetricNames.SINK_WRITE_COUNT); this.sinkWriteBytes = metricsContext.counter(MetricNames.SINK_WRITE_BYTES); this.sinkWriterQPS = metricsContext.meter(MetricNames.SINK_WRITE_QPS); @@ -118,6 +123,7 @@ public List> snapshotState() throws IOException { @Override public void close() throws Exception { sinkWriter.close(); + context.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java index e457d69f27c..7d8052bfd18 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java @@ -19,6 +19,8 @@ import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; +import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; @@ -49,6 +51,7 @@ public class FlinkSourceEnumerator private final SplitEnumeratorContext> enumeratorContext; + private final SourceSplitEnumerator.Context context; private final int parallelism; private final Object lock = new Object(); @@ -62,12 +65,14 @@ public FlinkSourceEnumerator( SplitEnumeratorContext> enumContext) { this.sourceSplitEnumerator = enumerator; this.enumeratorContext = enumContext; + this.context = new FlinkSourceSplitEnumeratorContext<>(enumeratorContext); this.parallelism = enumeratorContext.currentParallelism(); } @Override public void start() { sourceSplitEnumerator.open(); + context.getEventListener().onEvent(new EnumeratorOpenEvent()); } @Override @@ -106,6 +111,7 @@ public EnumStateT snapshotState(long checkpointId) throws Exception { @Override public void close() throws IOException { sourceSplitEnumerator.close(); + context.getEventListener().onEvent(new EnumeratorCloseEvent()); } @Override diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java index c2f9cde5005..fb1dc85174e 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.event.ReaderCloseEvent; +import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.flink.api.connector.source.ReaderOutput; @@ -66,6 +68,7 @@ public FlinkSourceReader( public void start() { try { sourceReader.open(); + context.getEventListener().onEvent(new ReaderOpenEvent()); } catch (Exception e) { throw new RuntimeException(e); } @@ -121,6 +124,7 @@ public void handleSourceEvents(SourceEvent sourceEvent) { @Override public void close() throws Exception { sourceReader.close(); + context.getEventListener().onEvent(new ReaderCloseEvent()); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java index 434b1ef9799..a9eac500629 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java @@ -21,6 +21,7 @@ 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.event.WriterCloseEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; @@ -47,16 +48,19 @@ public class SparkDataWriter implements DataWriter sinkWriter, @Nullable SinkCommitter sinkCommitter, MultiTableManager multiTableManager, - long epochId) { + long epochId, + org.apache.seatunnel.api.sink.SinkWriter.Context context) { this.sinkWriter = sinkWriter; this.sinkCommitter = sinkCommitter; this.epochId = epochId == 0 ? 1 : epochId; this.multiTableManager = multiTableManager; + this.context = context; initResourceManger(); } @@ -97,6 +101,7 @@ public WriterCommitMessage commit() throws IOException { new SparkWriterCommitMessage<>(latestCommitInfoT); cleanCommitInfo(); sinkWriter.close(); + context.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java index 3a646f3aca2..b684654103a 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java @@ -63,6 +63,6 @@ public DataWriter createDataWriter(int partitionId, long taskId, lo throw new RuntimeException("Failed to create SinkCommitter.", e); } return new SparkDataWriter<>( - writer, committer, new MultiTableManager(catalogTables), epochId); + writer, committer, new MultiTableManager(catalogTables), epochId, context); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java index 59f931e38f1..c2c24aa9147 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java @@ -21,6 +21,7 @@ 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.event.WriterCloseEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; @@ -47,16 +48,19 @@ public class SeaTunnelSparkDataWriter implements DataWriter protected volatile MultiTableResourceManager resourceManager; private final MultiTableManager multiTableManager; + private final SinkWriter.Context context; public SeaTunnelSparkDataWriter( SinkWriter sinkWriter, @Nullable SinkCommitter sinkCommitter, MultiTableManager multiTableManager, - long epochId) { + long epochId, + SinkWriter.Context context) { this.sinkWriter = sinkWriter; this.sinkCommitter = sinkCommitter; this.multiTableManager = multiTableManager; this.epochId = epochId == 0 ? 1 : epochId; + this.context = context; initResourceManger(); } @@ -89,6 +93,7 @@ public WriterCommitMessage commit() throws IOException { new SeaTunnelSparkWriterCommitMessage<>(latestCommitInfoT); cleanCommitInfo(); sinkWriter.close(); + context.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java index b83787cac1e..255a9cd339f 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java @@ -64,7 +64,7 @@ public DataWriter createWriter(int partitionId, long taskId) { throw new RuntimeException("Failed to create SinkCommitter.", e); } return new SeaTunnelSparkDataWriter<>( - writer, committer, new MultiTableManager(catalogTables), 0); + writer, committer, new MultiTableManager(catalogTables), 0, context); } @Override diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 7f95aaf01c4..c2f431836b6 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -46,4 +46,10 @@ accessors-smart-2.4.7.jar asm-9.1.jar avro-1.11.1.jar groovy-4.0.16.jar -seatunnel-janino-2.3.8-SNAPSHOT-optional.jar \ No newline at end of file +seatunnel-janino-2.3.8-SNAPSHOT-optional.jar +protobuf-java-util-3.25.3.jar +protobuf-java-3.25.3.jar +protoc-jar-3.11.4.jar +error_prone_annotations-2.18.0.jar +gson-2.8.9.jar +j2objc-annotations-2.8.jar From c7b60632a99da9c142d2c8d7220b5496c4b4b782 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 14:11:50 +0800 Subject: [PATCH 11/19] init --- .../seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java | 5 ----- .../connectors/seatunnel/hbase/sink/HbaseSinkFactory.java | 5 ----- .../connectors/seatunnel/hbase/sink/HbaseSinkWriter.java | 1 - 3 files changed, 11 deletions(-) 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 6f350e60e77..0a46b1baefa 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 @@ -23,8 +23,6 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SchemaSaveMode; import org.apache.seatunnel.api.sink.SeaTunnelSink; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSink; import org.apache.seatunnel.api.sink.SupportSaveMode; @@ -32,12 +30,9 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.factory.CatalogFactory; -import org.apache.seatunnel.api.sink.SupportMultiTableSink; -import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseAggregatedCommitInfo; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java index f7620190b23..0992b11d710 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java @@ -21,15 +21,10 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.connector.TableSink; -import org.apache.seatunnel.api.sink.SinkCommonOptions; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; -import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; -import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import com.google.auto.service.AutoService; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java index ac2b8414a1b..73ee19f9369 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; -import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; From 65e75c40ba56975cca9cae30d538e3240ce5c0aa Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 14:23:16 +0800 Subject: [PATCH 12/19] init --- .../connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java index 6894fba17fc..b9a3fc25fd4 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java @@ -34,7 +34,7 @@ public class HbaseCatalogFactory implements CatalogFactory { public Catalog createCatalog(String catalogName, ReadonlyConfig options) { // Create an instance of HbaseCatalog, passing in the catalog name, namespace, and Hbase // parameters - HbaseParameters hbaseParameters = HbaseParameters.buildWithSinkConfig(options.toConfig()); + HbaseParameters hbaseParameters = HbaseParameters.buildWithConfig(options); return new HbaseCatalog(catalogName, hbaseParameters.getNamespace(), hbaseParameters); } From c7fd0e5e5ccc7a90e7991126876990bdec0f205f Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 22:47:26 +0800 Subject: [PATCH 13/19] Revert "init" This reverts commit 8b4b926c02c13a683aa95306c52f0039b4b5defc. --- .../workflows/labeler/label-scope-conf.yml | 15 +- .github/workflows/publish-docker.yaml | 6 +- config/log4j2.properties | 6 +- config/plugin_config | 4 +- docs/en/concept/sql-config.md | 5 +- docs/en/connector-v2/sink/Clickhouse.md | 6 +- docs/en/connector-v2/sink/Hbase.md | 73 -- docs/en/connector-v2/sink/Http.md | 6 +- docs/en/connector-v2/sink/Jdbc.md | 2 +- docs/en/connector-v2/sink/Kafka.md | 88 +- docs/en/connector-v2/sink/MongoDB.md | 6 +- docs/en/connector-v2/sink/Mysql.md | 2 +- docs/en/connector-v2/sink/Paimon.md | 2 +- docs/en/connector-v2/sink/Qdrant.md | 70 -- docs/en/connector-v2/sink/Rabbitmq.md | 39 - docs/en/connector-v2/sink/Typesense.md | 93 -- docs/en/connector-v2/source/Clickhouse.md | 6 +- docs/en/connector-v2/source/Elasticsearch.md | 150 +-- docs/en/connector-v2/source/Http.md | 6 +- docs/en/connector-v2/source/Jdbc.md | 27 +- docs/en/connector-v2/source/MongoDB-CDC.md | 6 +- docs/en/connector-v2/source/MongoDB.md | 6 +- docs/en/connector-v2/source/MyHours.md | 2 +- docs/en/connector-v2/source/Mysql.md | 8 +- docs/en/connector-v2/source/Opengauss-CDC.md | 170 ---- docs/en/connector-v2/source/Oracle.md | 6 +- docs/en/connector-v2/source/PostgreSQL.md | 21 +- docs/en/connector-v2/source/Qdrant.md | 81 -- docs/en/connector-v2/source/Sls.md | 6 +- docs/en/connector-v2/source/SqlServer.md | 6 +- docs/en/connector-v2/source/Tablestore.md | 102 -- docs/en/connector-v2/source/Typesense.md | 79 -- docs/en/connector-v2/source/kafka.md | 92 +- docs/en/faq.md | 21 + .../en/seatunnel-engine/checkpoint-storage.md | 1 + docs/en/seatunnel-engine/rest-api.md | 135 --- docs/en/start-v2/docker/docker.md | 48 +- docs/en/start-v2/locally/deployment.md | 39 +- docs/en/transform-v2/embedding.md | 392 -------- docs/en/transform-v2/llm.md | 168 +--- docs/sidebars.js | 24 +- docs/zh/concept/sql-config.md | 5 +- docs/zh/connector-v2/sink/Clickhouse.md | 6 +- docs/zh/connector-v2/sink/Hbase.md | 72 -- docs/zh/connector-v2/sink/Http.md | 6 +- docs/zh/connector-v2/sink/Jdbc.md | 2 +- docs/zh/connector-v2/sink/Kafka.md | 87 +- docs/zh/connector-v2/sink/Paimon.md | 31 +- docs/zh/connector-v2/sink/Qdrant.md | 68 -- docs/zh/connector-v2/sink/Typesense.md | 95 -- docs/zh/connector-v2/source/Elasticsearch.md | 247 ----- docs/zh/connector-v2/source/Kafka.md | 288 ------ docs/zh/connector-v2/source/Opengauss-CDC.md | 169 ---- docs/zh/connector-v2/source/Qdrant.md | 79 -- docs/zh/connector-v2/source/Sls.md | 6 +- docs/zh/connector-v2/source/Typesense.md | 79 -- docs/zh/faq.md | 21 + .../zh/seatunnel-engine/checkpoint-storage.md | 1 + docs/zh/seatunnel-engine/rest-api.md | 155 +-- docs/zh/start-v2/locally/deployment.md | 41 +- docs/zh/start-v2/locally/quick-start-flink.md | 2 +- .../locally/quick-start-seatunnel-engine.md | 2 +- docs/zh/start-v2/locally/quick-start-spark.md | 2 +- docs/zh/transform-v2/embedding.md | 382 ------- docs/zh/transform-v2/llm.md | 164 +--- plugin-mapping.properties | 9 +- pom.xml | 6 +- release-note.md | 2 - .../sink/multitablesink/MultiTableSink.java | 9 +- .../multitablesink/MultiTableSinkWriter.java | 73 +- .../catalog/schema/ReadonlyConfigParser.java | 3 +- .../api/table/type/SeaTunnelRow.java | 3 - .../seatunnel/api/tracing/MDCCallable.java | 50 - .../seatunnel/api/tracing/MDCContext.java | 122 --- .../seatunnel/api/tracing/MDCExecutor.java | 36 - .../api/tracing/MDCExecutorService.java | 121 --- .../seatunnel/api/tracing/MDCRunnable.java | 44 - .../tracing/MDCScheduledExecutorService.java | 63 -- .../seatunnel/api/tracing/MDCTracer.java | 126 --- .../seatunnel/api/tracing/MDCTracerTest.java | 149 --- .../common/exception/CommonError.java | 4 +- .../utils/TemporaryClassLoaderContext.java | 53 - .../assertion/excecutor/AssertExecutor.java | 8 - .../connector-cdc-opengauss/pom.xml | 91 -- .../connection/PostgresConnection.java | 815 --------------- .../PostgresReplicationConnection.java | 928 ------------------ .../OpengaussIncrementalSourceFactory.java | 110 --- seatunnel-connectors-v2/connector-cdc/pom.xml | 1 - .../console/sink/ConsoleSinkWriter.java | 5 +- .../catalog/ElasticSearchCatalog.java | 3 +- .../elasticsearch/client/EsRestClient.java | 28 +- .../elasticsearch/config/SourceConfig.java | 41 +- .../dto/source/SourceIndexInfo.java} | 15 +- .../ElasticsearchConnectorErrorCode.java | 6 +- .../DefaultSeaTunnelRowDeserializer.java | 4 +- .../serialize/source/ElasticsearchRecord.java | 2 - .../source/ElasticsearchSource.java | 116 +-- .../source/ElasticsearchSourceFactory.java | 8 +- .../source/ElasticsearchSourceReader.java | 32 +- .../source/ElasticsearchSourceSplit.java | 9 +- .../ElasticsearchSourceSplitEnumerator.java | 61 +- .../fake/source/FakeDataGenerator.java | 72 +- .../fake/source/FakeSourceReader.java | 10 +- .../source/FakeSourceSplitEnumerator.java | 10 +- .../fake/utils/FakeDataRandomUtils.java | 93 +- .../fake/source/FakeDataGeneratorTest.java | 55 -- .../src/test/resources/complex.schema.conf | 2 +- .../src/test/resources/fake-data.column.conf | 97 -- .../hbase/config/HbaseParameters.java | 53 +- .../seatunnel/hbase/sink/HbaseSink.java | 2 +- .../hbase/sink/HbaseSinkFactory.java | 2 - .../seatunnel/hbase/sink/HbaseSinkWriter.java | 11 +- .../iceberg/IcebergCatalogLoader.java | 11 +- .../iceberg/catalog/IcebergCatalog.java | 37 +- .../seatunnel/iceberg/data/RowConverter.java | 22 +- .../seatunnel/iceberg/sink/IcebergSink.java | 6 +- .../iceberg/sink/IcebergSinkWriter.java | 10 +- .../sink/writer/IcebergRecordWriter.java | 12 +- .../sink/writer/IcebergWriterFactory.java | 4 + .../iceberg/source/IcebergSource.java | 10 +- .../seatunnel/iceberg/utils/SchemaUtils.java | 43 +- .../jdbc/catalog/JdbcCatalogOptions.java | 6 +- .../jdbc/catalog/oracle/OracleCatalog.java | 25 +- .../catalog/oracle/OracleCatalogFactory.java | 4 +- .../jdbc/catalog/utils/CatalogUtils.java | 19 +- .../jdbc/config/JdbcConnectionConfig.java | 15 - .../seatunnel/jdbc/config/JdbcOptions.java | 7 - .../jdbc/config/JdbcSourceConfig.java | 5 +- .../jdbc/config/JdbcSourceTableConfig.java | 5 +- .../internal/dialect/mysql/MySqlVersion.java | 37 +- .../dialect/oracle/OracleTypeConverter.java | 24 +- .../dialect/oracle/OracleTypeMapper.java | 13 +- .../BufferReducedBatchStatementExecutor.java | 14 +- .../BufferedBatchStatementExecutor.java | 12 +- .../InsertOrUpdateBatchStatementExecutor.java | 17 +- .../jdbc/utils/JdbcCatalogUtils.java | 5 - .../dialect/mysql/MysqlVersionTest.java | 59 -- .../oracle/OracleTypeConverterTest.java | 239 ++--- .../connector-kafka/pom.xml | 5 - .../seatunnel/kafka/config/Config.java | 14 - .../seatunnel/kafka/config/MessageFormat.java | 3 +- .../DefaultSeaTunnelRowSerializer.java | 52 +- .../seatunnel/kafka/sink/KafkaSinkWriter.java | 12 +- .../kafka/source/ConsumerMetadata.java | 1 + .../kafka/source/KafkaConsumerThread.java | 111 +++ .../source/KafkaPartitionSplitReader.java | 458 --------- .../kafka/source/KafkaRecordEmitter.java | 112 --- .../seatunnel/kafka/source/KafkaSource.java | 33 +- .../kafka/source/KafkaSourceConfig.java | 21 +- .../kafka/source/KafkaSourceReader.java | 343 +++++-- .../source/KafkaSourceSplitEnumerator.java | 8 +- .../kafka/source/KafkaSourceSplitState.java | 45 - .../fetch/KafkaSourceFetcherManager.java | 99 -- .../serde/BsonToRowDataConverters.java | 2 +- .../source/split/SamplingSplitStrategy.java | 7 +- .../serde/BsonToRowDataConvertersTest.java | 55 -- .../paimon/catalog/PaimonCatalog.java | 30 - .../exception/PaimonConnectorErrorCode.java | 3 +- .../seatunnel/paimon/utils/RowConverter.java | 63 +- .../seatunnel/paimon/utils/SchemaUtil.java | 15 +- .../paimon/utils/RowConverterTest.java | 225 ++--- .../connector-qdrant/pom.xml | 63 -- .../seatunnel/qdrant/config/QdrantConfig.java | 50 - .../qdrant/config/QdrantParameters.java | 47 - .../exception/QdrantConnectorException.java | 36 - .../qdrant/sink/QdrantBatchWriter.java | 190 ---- .../seatunnel/qdrant/sink/QdrantSink.java | 50 - .../qdrant/sink/QdrantSinkFactory.java | 55 -- .../qdrant/sink/QdrantSinkWriter.java | 60 -- .../seatunnel/qdrant/source/QdrantSource.java | 63 -- .../qdrant/source/QdrantSourceFactory.java | 63 -- .../qdrant/source/QdrantSourceReader.java | 181 ---- .../rabbitmq/client/RabbitmqClient.java | 11 +- .../rabbitmq/config/RabbitmqConfig.java | 36 - .../tablestore/config/TablestoreOptions.java | 19 - .../DefaultSeaTunnelRowDeserializer.java | 38 - .../serialize/SeaTunnelRowDeserializer.java | 26 - .../tablestore/source/TableStoreDBSource.java | 102 -- .../source/TableStoreDBSourceReader.java | 175 ---- .../source/TableStoreDBSourceSplit.java | 38 - .../TableStoreDBSourceSplitEnumerator.java | 166 ---- .../source/TableStoreDBSourceState.java | 34 - .../source/TableStoreDbSourceFactory.java | 64 -- .../source/TableStoreProcessor.java | 95 -- .../connector-typesense/pom.xml | 90 -- .../typesense/catalog/TypesenseCatalog.java | 214 ---- .../catalog/TypesenseCatalogFactory.java | 45 - .../catalog/TypesenseTypeConverter.java | 94 -- .../typesense/client/TypesenseClient.java | 286 ------ .../typesense/client/TypesenseType.java | 38 - .../typesense/config/SinkConfig.java | 78 -- .../typesense/config/SourceConfig.java | 42 - .../config/TypesenseConnectionConfig.java | 45 - .../typesense/dto/CollectionInfo.java | 40 - .../typesense/dto/SourceCollectionInfo.java | 33 - .../TypesenseConnectorErrorCode.java | 50 - .../TypesenseConnectorException.java | 32 - .../typesense/serialize/KeyExtractor.java | 98 -- .../sink/SeaTunnelRowSerializer.java | 26 - .../sink/TypesenseRowSerializer.java | 119 --- .../sink/collection/CollectionSerializer.java | 24 - .../FixedValueCollectionSerializer.java | 34 - .../DefaultSeaTunnelRowDeserializer.java | 253 ----- .../source/SeaTunnelRowDeserializer.java | 25 - .../serialize/source/TypesenseRecord.java | 31 - .../typesense/sink/TypesenseSink.java | 96 -- .../typesense/sink/TypesenseSinkFactory.java | 72 -- .../typesense/sink/TypesenseSinkWriter.java | 139 --- .../typesense/source/TypesenseSource.java | 87 -- .../source/TypesenseSourceFactory.java | 59 -- .../source/TypesenseSourceReader.java | 132 --- .../source/TypesenseSourceSplit.java | 41 - .../TypesenseSourceSplitEnumerator.java | 187 ---- .../state/TypesenseAggregatedCommitInfo.java | 22 - .../typesense/state/TypesenseCommitInfo.java | 22 - .../typesense/util/URLParamsConverter.java | 71 -- .../TypesenseRowSerializerTest.java | 64 -- .../typesense/sink/TypesenseFactoryTest.java | 32 - .../util/URLParamsConverterTest.java | 35 - seatunnel-connectors-v2/pom.xml | 2 - .../src/main/bin/seatunnel-cluster.cmd | 15 +- .../src/main/bin/seatunnel-cluster.sh | 13 +- .../src/main/bin/seatunnel.cmd | 21 +- .../src/main/bin/seatunnel.sh | 19 +- .../src/main/resources/log4j2.properties | 4 +- seatunnel-dist/pom.xml | 13 - seatunnel-dist/release-docs/LICENSE | 3 - .../licenses/LICENSE-protoc-jar.txt | 201 ---- .../connector-cdc-opengauss-e2e/pom.xml | 75 -- .../cdc/postgres/OpengaussCDCIT.java | 727 -------------- .../src/test/resources/ddl/inventory.sql | 199 ---- .../resources/opengausscdc_to_opengauss.conf | 62 -- ...ngausscdc_to_opengauss_test_add_Filed.conf | 62 -- ..._to_opengauss_with_custom_primary_key.conf | 69 -- ...gauss_with_multi_table_mode_one_table.conf | 63 -- ...gauss_with_multi_table_mode_two_table.conf | 63 -- ...scdc_to_opengauss_with_no_primary_key.conf | 63 -- .../elasticsearch/ElasticsearchIT.java | 355 +------ ...earch_multi_source_and_sink_by_filter.conf | 92 -- .../e2e/connector/file/s3/S3Utils.java | 14 + .../e2e/connector/hbase/HbaseIT.java | 47 +- .../fake-to-hbase-with-multipletable.conf | 86 -- .../hbase-to-assert-with-multipletable.conf | 129 --- .../seatunnel/jdbc/AbstractJdbcIT.java | 48 +- .../jdbc/InsecureURLClassLoader.java | 59 -- .../connectors/seatunnel/jdbc/JdbcCase.java | 4 - .../seatunnel/jdbc/JdbcMysqlIT.java | 1 - .../seatunnel/jdbc/JdbcOracleIT.java | 26 - .../jdbc/JdbcOracleMultipleTablesIT.java | 307 ------ .../resources/jdbc_oracle_source_to_sink.conf | 4 +- ...dbc_oracle_source_to_sink_use_select1.conf | 4 +- ...dbc_oracle_source_to_sink_use_select2.conf | 2 +- ...dbc_oracle_source_to_sink_use_select3.conf | 2 +- ...o_sink_without_decimal_type_narrowing.conf | 82 -- ...e_source_with_multiple_tables_to_sink.conf | 65 -- .../seatunnel/jdbc/JdbcStarRocksdbIT.java | 2 - .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 4 +- .../seatunnel/jdbc/JdbcSqlServerIT.java | 1 - .../connectors/seatunnel/jdbc/JdbcDmIT.java | 1 - .../seatunnel/jdbc/JdbcGBase8aIT.java | 2 +- .../seatunnel/jdbc/JdbcGreenplumIT.java | 1 - .../connectors/seatunnel/jdbc/JdbcXuguIT.java | 1 - .../connector-kafka-e2e/pom.xml | 6 - .../e2e/connector/kafka/KafkaIT.java | 352 +------ .../kafka_multi_source_to_pg.conf | 2 +- .../protobuf/fake_to_kafka_protobuf.conf | 100 -- .../protobuf/kafka_protobuf_to_assert.conf | 177 ---- .../e2e/connector/paimon/PaimonRecord.java | 1 - .../e2e/connector/paimon/PaimonSinkCDCIT.java | 48 +- .../connector/paimon/PaimonSinkHdfsIT.java | 129 --- ..._sink_paimon_truncate_with_hdfs_case1.conf | 80 -- ..._sink_paimon_truncate_with_hdfs_case2.conf | 65 -- ..._sink_paimon_truncate_with_hive_case1.conf | 82 -- ..._sink_paimon_truncate_with_hive_case2.conf | 67 -- ...sink_paimon_truncate_with_local_case1.conf | 71 -- ...sink_paimon_truncate_with_local_case2.conf | 56 -- .../connector-qdrant-e2e/pom.xml | 68 -- .../e2e/connector/v2/qdrant/QdrantIT.java | 145 --- .../src/test/resources/qdrant-to-qdrant.conf | 51 - .../e2e/connector/rabbitmq/RabbitmqIT.java | 9 - .../test/resources/rabbitmq-to-rabbitmq.conf | 6 - .../connector-typesense-e2e/pom.xml | 51 - .../e2e/connector/typesense/TypesenseIT.java | 265 ----- .../fake_to_typesense_with_append_data.conf | 52 - ...typesense_with_create_when_not_exists.conf | 51 - .../fake_to_typesense_with_drop_data.conf | 52 - ...typesense_with_error_when_data_exists.conf | 52 - ..._typesense_with_error_when_not_exists.conf | 51 - .../fake_to_typesense_with_primary_keys.conf | 50 - ...ake_to_typesense_with_recreate_schema.conf | 51 - .../resources/typesense_source_and_sink.conf | 54 - .../resources/typesense_to_typesense.conf | 64 -- .../typesense_to_typesense_with_query.conf | 65 -- .../seatunnel-connector-v2-e2e/pom.xml | 3 - .../src/test/resources/log4j2.properties | 4 +- .../FakeSourceToConsoleWithEventReportIT.java | 2 +- .../connector-seatunnel-e2e-base/pom.xml | 10 - .../engine/e2e/ClusterFaultToleranceIT.java | 1 + .../engine/e2e/ClusterSeaTunnelContainer.java | 186 ---- .../WorkerTagClusterTest.java | 161 --- .../log4j2-test.properties | 4 +- .../log4j2.properties | 4 +- .../log4j2-test.properties | 4 +- .../log4j2.properties | 4 +- .../log4j2-test.properties | 4 +- .../log4j2.properties | 4 +- .../src/test/resources/log4j2-test.properties | 4 +- .../src/test/resources/log4j2.properties | 4 +- .../e2e/transform/TestEmbeddingIT.java | 102 -- .../seatunnel/e2e/transform/TestLLMIT.java | 7 - .../test/resources/embedding_transform.conf | 295 ------ .../resources/embedding_transform_custom.conf | 152 --- .../test/resources/llm_transform_custom.conf | 94 -- .../src/test/resources/mock-embedding.json | 75 -- .../engine/client/SeaTunnelClient.java | 37 +- .../src/test/resources/log4j2-test.properties | 4 +- .../seatunnel/engine/common/Constant.java | 2 - .../SeaTunnelChildFirstClassLoader.java | 2 - .../engine/server/CoordinatorService.java | 15 +- .../seatunnel/engine/server/EventService.java | 100 -- .../server/LiteNodeDropOutTcpIpJoiner.java | 4 - .../engine/server/SeaTunnelServer.java | 10 +- .../engine/server/TaskExecutionService.java | 76 +- .../checkpoint/CheckpointCoordinator.java | 2 - .../CheckpointBarrierTriggerOperation.java | 2 +- .../operation/CheckpointEndOperation.java | 2 +- .../CheckpointErrorReportOperation.java | 2 +- .../CheckpointFinishedOperation.java | 2 +- .../operation/NotifyTaskRestoreOperation.java | 2 +- .../operation/NotifyTaskStartOperation.java | 2 +- .../operation/TaskAcknowledgeOperation.java | 9 +- .../operation/TaskReportStatusOperation.java | 9 +- ...rSchemaChangeAfterCheckpointOperation.java | 8 +- ...SchemaChangeBeforeCheckpointOperation.java | 6 +- .../dag/execution/ExecutionPlanGenerator.java | 4 +- .../engine/server/execution/Task.java | 1 - .../opeartion/ReleaseSlotOperation.java | 8 +- .../opeartion/RequestSlotOperation.java | 8 +- .../resourcemanager/worker/WorkerProfile.java | 2 - .../engine/server/rest/RestConstant.java | 4 +- .../rest/RestHttpGetCommandProcessor.java | 4 +- .../rest/RestHttpPostCommandProcessor.java | 80 +- .../service/slot/DefaultSlotService.java | 1 - .../task/SourceSplitEnumeratorTask.java | 4 - .../server/task/flow/SinkFlowLifeCycle.java | 17 +- .../server/task/flow/SourceFlowLifeCycle.java | 23 +- .../task/operation/CancelTaskOperation.java | 5 +- .../CheckTaskGroupIsExecutingOperation.java | 5 +- .../CleanTaskGroupContextOperation.java | 5 +- .../task/operation/DeployTaskOperation.java | 5 +- .../GetTaskGroupAddressOperation.java | 6 +- .../operation/NotifyTaskStatusOperation.java | 6 +- .../server/task/operation/TaskOperation.java | 5 +- .../task/operation/TracingOperation.java | 63 -- .../checkpoint/BarrierFlowOperation.java | 2 +- .../checkpoint/CloseRequestOperation.java | 6 +- .../sink/SinkPrepareCommitOperation.java | 2 +- .../operation/sink/SinkRegisterOperation.java | 6 +- .../source/AssignSplitOperation.java | 8 +- .../source/CloseIdleReaderOperation.java | 9 +- .../source/LastCheckpointNotifyOperation.java | 9 +- .../source/RequestSplitOperation.java | 6 +- .../source/RestoredSplitOperation.java | 2 +- .../source/SourceNoMoreElementOperation.java | 7 +- .../source/SourceReaderEventOperation.java | 2 +- .../source/SourceRegisterOperation.java | 7 +- .../engine/server/utils/RestUtil.java | 21 - .../resourcemanager/FixSlotResourceTest.java | 17 +- .../src/test/resources/log4j2-test.properties | 4 +- .../storage/hdfs/HdfsStorageFactory.java | 1 + .../storage/hdfs/OssFileCheckpointTest.java | 3 + .../src/test/resources/log4j2-test.properties | 4 +- .../storage/file/IMapFileOSSStorageTest.java | 6 + .../src/main/resources/log4j2.properties | 4 +- .../pom.xml | 6 + seatunnel-formats/pom.xml | 1 - .../seatunnel-format-protobuf/pom.xml | 59 -- .../format/protobuf/CompileDescriptor.java | 107 -- .../ProtobufDeserializationSchema.java | 68 -- .../protobuf/ProtobufSerializationSchema.java | 51 - .../protobuf/ProtobufToRowConverter.java | 185 ---- .../protobuf/RowToProtobufConverter.java | 152 --- .../exception/ProtobufFormatErrorCode.java | 43 - .../SeaTunnelProtobufFormatException.java | 29 - .../protobuf/ProtobufConverterTest.java | 155 --- .../{nlpmodel => }/llm/LLMTransform.java | 62 +- .../llm/LLMTransformConfig.java | 35 +- .../llm/LLMTransformFactory.java | 26 +- .../transform/llm/ModelProvider.java | 8 +- .../remote => llm/model}/AbstractModel.java | 2 +- .../llm/remote => llm/model}/Model.java | 2 +- .../model}/openai/OpenAIModel.java | 4 +- .../nlpmodel/CustomConfigPlaceholder.java | 56 -- .../transform/nlpmodel/ModelProvider.java | 52 - .../nlpmodel/ModelTransformConfig.java | 115 --- .../embadding/EmbeddingTransform.java | 212 ---- .../embadding/EmbeddingTransformConfig.java | 41 - .../embadding/EmbeddingTransformFactory.java | 72 -- .../embadding/remote/AbstractModel.java | 70 -- .../nlpmodel/embadding/remote/Model.java | 30 - .../embadding/remote/custom/CustomModel.java | 152 --- .../embadding/remote/doubao/DoubaoModel.java | 112 --- .../embadding/remote/openai/OpenAIModel.java | 116 --- .../remote/qianfan/QianfanModel.java | 173 ---- .../llm/remote/custom/CustomModel.java | 152 --- .../EmbeddingTransformFactoryTest.java | 32 - .../transform/LLMTransformFactoryTest.java | 2 +- .../embedding/EmbeddingRequestJsonTest.java | 143 --- .../transform/llm/LLMRequestJsonTest.java | 53 +- .../translation/source/CoordinatedSource.java | 11 - .../translation/source/ParallelSource.java | 8 - .../translation/flink/sink/FlinkSink.java | 7 +- .../flink/sink/FlinkSinkWriter.java | 8 +- .../flink/source/FlinkSourceEnumerator.java | 6 - .../flink/source/FlinkSourceReader.java | 4 - .../spark/sink/writer/SparkDataWriter.java | 7 +- .../sink/writer/SparkDataWriterFactory.java | 2 +- .../sink/write/SeaTunnelSparkDataWriter.java | 7 +- .../SeaTunnelSparkDataWriterFactory.java | 2 +- tools/dependencies/known-dependencies.txt | 8 +- 420 files changed, 1643 insertions(+), 22587 deletions(-) delete mode 100644 docs/en/connector-v2/sink/Qdrant.md delete mode 100644 docs/en/connector-v2/sink/Typesense.md delete mode 100644 docs/en/connector-v2/source/Opengauss-CDC.md delete mode 100644 docs/en/connector-v2/source/Qdrant.md delete mode 100644 docs/en/connector-v2/source/Tablestore.md delete mode 100644 docs/en/connector-v2/source/Typesense.md delete mode 100644 docs/en/transform-v2/embedding.md delete mode 100644 docs/zh/connector-v2/sink/Qdrant.md delete mode 100644 docs/zh/connector-v2/sink/Typesense.md delete mode 100644 docs/zh/connector-v2/source/Elasticsearch.md delete mode 100644 docs/zh/connector-v2/source/Kafka.md delete mode 100644 docs/zh/connector-v2/source/Opengauss-CDC.md delete mode 100644 docs/zh/connector-v2/source/Qdrant.md delete mode 100644 docs/zh/connector-v2/source/Typesense.md delete mode 100644 docs/zh/transform-v2/embedding.md delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.java delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java delete mode 100644 seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java delete mode 100644 seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java rename seatunnel-connectors-v2/{connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java => connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java} (74%) delete mode 100644 seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf delete mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java create mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java delete mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java delete mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java delete mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java delete mode 100644 seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java delete mode 100644 seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/pom.xml delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java delete mode 100644 seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java delete mode 100644 seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/pom.xml delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java delete mode 100644 seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java delete mode 100644 seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleMultipleTablesIT.java delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_without_decimal_type_narrowing.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf delete mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java delete mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java delete mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf delete mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform_custom.conf delete mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_transform_custom.conf delete mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mock-embedding.json delete mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java delete mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/pom.xml delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.java delete mode 100644 seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{nlpmodel => }/llm/LLMTransform.java (57%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{nlpmodel => }/llm/LLMTransformConfig.java (55%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{nlpmodel => }/llm/LLMTransformFactory.java (67%) rename seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java => seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java (83%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{nlpmodel/llm/remote => llm/model}/AbstractModel.java (98%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{nlpmodel/llm/remote => llm/model}/Model.java (94%) rename seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/{nlpmodel/llm/remote => llm/model}/openai/OpenAIModel.java (96%) delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java delete mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.java delete mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java diff --git a/.github/workflows/labeler/label-scope-conf.yml b/.github/workflows/labeler/label-scope-conf.yml index b417d53e72a..b0a89dfd354 100644 --- a/.github/workflows/labeler/label-scope-conf.yml +++ b/.github/workflows/labeler/label-scope-conf.yml @@ -257,19 +257,6 @@ activemq: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-activemq/** - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(activemq)/**' - -qdrant: - - all: - - changed-files: - - any-glob-to-any-file: seatunnel-connectors-v2/connector-qdrant/** - - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(qdrant)/**' - -typesense: - - all: - - changed-files: - - any-glob-to-any-file: seatunnel-connectors-v2/connector-typesense/** - - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(typesense)/**' - Zeta Rest API: - changed-files: - any-glob-to-any-file: seatunnel-engine/**/server/rest/** @@ -292,4 +279,4 @@ sls: - all: - changed-files: - any-glob-to-any-file: seatunnel-connectors-v2/connector-sls/** - - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(sls)/**' + - all-globs-to-all-files: '!seatunnel-connectors-v2/connector-!(sls)/**' \ No newline at end of file diff --git a/.github/workflows/publish-docker.yaml b/.github/workflows/publish-docker.yaml index 111aabf8511..e1041bca2c9 100644 --- a/.github/workflows/publish-docker.yaml +++ b/.github/workflows/publish-docker.yaml @@ -20,6 +20,8 @@ on: push: tags: - '*' + branches: + - dev paths-ignore: - 'docs/**' - '**/*.md' @@ -70,8 +72,8 @@ jobs: MAVEN_OPTS: -Xmx4096m run: | ./mvnw -B clean install \ - -Dmaven.test.skip=true \ - -Dmaven.javadoc.skip=true \ + -Dmaven.test.skip \ + -Dmaven.javadoc.skip \ -Dlicense.skipAddThirdParty=true \ -D"docker.build.skip"=false \ -D"docker.verify.skip"=false \ diff --git a/config/log4j2.properties b/config/log4j2.properties index 57dfa5fdf08..d7ff7f99a3f 100644 --- a/config/log4j2.properties +++ b/config/log4j2.properties @@ -45,7 +45,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -55,7 +55,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT @@ -67,7 +67,7 @@ appender.file.fileName = ${file_path}/${file_name}.log appender.file.filePattern = ${file_path}/${file_name}.log.%d{yyyy-MM-dd}-%i appender.file.append = true appender.file.layout.type = PatternLayout -appender.file.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.file.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.file.policies.type = Policies appender.file.policies.time.type = TimeBasedTriggeringPolicy appender.file.policies.time.modulate = true diff --git a/config/plugin_config b/config/plugin_config index 26eb4cab4a6..e3ac0f1d046 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -88,6 +88,4 @@ connector-web3j connector-milvus connector-activemq connector-sls -connector-qdrant -connector-typesense -connector-cdc-opengauss +--end-- \ No newline at end of file diff --git a/docs/en/concept/sql-config.md b/docs/en/concept/sql-config.md index dd24b57032e..fe148a6f726 100644 --- a/docs/en/concept/sql-config.md +++ b/docs/en/concept/sql-config.md @@ -120,10 +120,7 @@ CREATE TABLE sink_table WITH ( INSERT INTO sink_table SELECT id, name, age, email FROM source_table; ``` -* The `SELECT FROM` part is the table name of the source-mapped table. If the select field has keyword([refrence](https://github.com/JSQLParser/JSqlParser/blob/master/src/main/jjtree/net/sf/jsqlparser/parser/JSqlParserCC.jjt)),you should use it like \`filedName\`. -```sql -INSERT INTO sink_table SELECT id, name, age, email,`output` FROM source_table; -``` +* The `SELECT FROM` part is the table name of the source-mapped table. * The `INSERT INTO` part is the table name of the target-mapped table. * Note: This syntax does **not support** specifying fields in `INSERT`, like this: `INSERT INTO sink_table (id, name, age, email) SELECT id, name, age, email FROM source_table;` diff --git a/docs/en/connector-v2/sink/Clickhouse.md b/docs/en/connector-v2/sink/Clickhouse.md index 15d92f8c5fc..1528a6d161b 100644 --- a/docs/en/connector-v2/sink/Clickhouse.md +++ b/docs/en/connector-v2/sink/Clickhouse.md @@ -24,9 +24,9 @@ Used to write data to Clickhouse. In order to use the Clickhouse connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------------| -| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-clickhouse) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------------------------------| +| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | ## Data Type Mapping diff --git a/docs/en/connector-v2/sink/Hbase.md b/docs/en/connector-v2/sink/Hbase.md index 3ceba0982d4..0f808f5e52c 100644 --- a/docs/en/connector-v2/sink/Hbase.md +++ b/docs/en/connector-v2/sink/Hbase.md @@ -116,79 +116,6 @@ Hbase { all_columns = seatunnel } } - -``` - -### Multiple Table - -```hocon -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - tables_configs = [ - { - schema = { - table = "hbase_sink_1" - fields { - name = STRING - c_string = STRING - c_double = DOUBLE - c_bigint = BIGINT - c_float = FLOAT - c_int = INT - c_smallint = SMALLINT - c_boolean = BOOLEAN - time = BIGINT - } - } - rows = [ - { - kind = INSERT - fields = ["label_1", "sink_1", 4.3, 200, 2.5, 2, 5, true, 1627529632356] - } - ] - }, - { - schema = { - table = "hbase_sink_2" - fields { - name = STRING - c_string = STRING - c_double = DOUBLE - c_bigint = BIGINT - c_float = FLOAT - c_int = INT - c_smallint = SMALLINT - c_boolean = BOOLEAN - time = BIGINT - } - } - rows = [ - { - kind = INSERT - fields = ["label_2", "sink_2", 4.3, 200, 2.5, 2, 5, true, 1627529632357] - } - ] - } - ] - } -} - -sink { - Hbase { - zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" - table = "${table_name}" - rowkey_column = ["name"] - family_name { - all_columns = info - } - } -} ``` ## Writes To The Specified Column Family diff --git a/docs/en/connector-v2/sink/Http.md b/docs/en/connector-v2/sink/Http.md index 75753b3bf1c..d756f06aea0 100644 --- a/docs/en/connector-v2/sink/Http.md +++ b/docs/en/connector-v2/sink/Http.md @@ -27,9 +27,9 @@ Used to launch web hooks using data. In order to use the Http connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------| -| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-http) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------------------------| +| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-http) | ## Sink Options diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md index 1ddbdd507d9..99f06891a14 100644 --- a/docs/en/connector-v2/sink/Jdbc.md +++ b/docs/en/connector-v2/sink/Jdbc.md @@ -235,7 +235,7 @@ there are some reference value for params above. | SQL Server | com.microsoft.sqlserver.jdbc.SQLServerDriver | jdbc:sqlserver://localhost:1433 | com.microsoft.sqlserver.jdbc.SQLServerXADataSource | https://mvnrepository.com/artifact/com.microsoft.sqlserver/mssql-jdbc | | Oracle | oracle.jdbc.OracleDriver | jdbc:oracle:thin:@localhost:1521/xepdb1 | oracle.jdbc.xa.OracleXADataSource | https://mvnrepository.com/artifact/com.oracle.database.jdbc/ojdbc8 | | sqlite | org.sqlite.JDBC | jdbc:sqlite:test.db | / | https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc | -| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar | +| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar | | StarRocks | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | / | https://mvnrepository.com/artifact/mysql/mysql-connector-java | | db2 | com.ibm.db2.jcc.DB2Driver | jdbc:db2://localhost:50000/testdb | com.ibm.db2.jcc.DB2XADataSource | https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc/db2jcc4 | | saphana | com.sap.db.jdbc.Driver | jdbc:sap://localhost:39015 | / | https://mvnrepository.com/artifact/com.sap.cloud.db.jdbc/ngdbc | diff --git a/docs/en/connector-v2/sink/Kafka.md b/docs/en/connector-v2/sink/Kafka.md index 9868e44f602..6447faa159a 100644 --- a/docs/en/connector-v2/sink/Kafka.md +++ b/docs/en/connector-v2/sink/Kafka.md @@ -24,28 +24,25 @@ Write Rows to a Kafka topic. In order to use the Kafka connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Maven | -|------------|--------------------|-------------------------------------------------------------------------------------| -| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | +| Datasource | Supported Versions | Maven | +|------------|--------------------|-------------------------------------------------------------------------------------------------------------| +| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | ## Sink Options -| Name | Type | Required | Default | Description | -|-----------------------|--------|----------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| topic | String | Yes | - | When the table is used as sink, the topic name is the topic to write data to. | -| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | -| kafka.config | Map | No | - | In addition to the above parameters that must be specified by the `Kafka producer` client, the user can also specify multiple non-mandatory parameters for the `producer` client, covering [all the producer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#producerconfigs). | -| semantics | String | No | NON | Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. | -| partition_key_fields | Array | No | - | Configure which fields are used as the key of the kafka message. | -| partition | Int | No | - | We can specify the partition, all messages will be sent to this partition. | -| assign_partitions | Array | No | - | We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. | -| transaction_prefix | String | No | - | If semantic is specified as EXACTLY_ONCE, the producer will write all messages in a Kafka transaction,kafka distinguishes different transactions by different transactionId. This parameter is prefix of kafka transactionId, make sure different job use different prefix. | -| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, ogg_json and avro.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | -| field_delimiter | String | No | , | Customize the field delimiter for data format. | -| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../sink-common-options.md) for details | -| protobuf_message_name | String | No | - | Effective when the format is set to protobuf, specifies the Message name | -| protobuf_schema | String | No | - | Effective when the format is set to protobuf, specifies the Schema definition | - +| Name | Type | Required | Default | Description | +|----------------------|--------|----------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | Yes | - | When the table is used as sink, the topic name is the topic to write data to. | +| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | +| kafka.config | Map | No | - | In addition to the above parameters that must be specified by the `Kafka producer` client, the user can also specify multiple non-mandatory parameters for the `producer` client, covering [all the producer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#producerconfigs). | +| semantics | String | No | NON | Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. | +| partition_key_fields | Array | No | - | Configure which fields are used as the key of the kafka message. | +| partition | Int | No | - | We can specify the partition, all messages will be sent to this partition. | +| assign_partitions | Array | No | - | We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. | +| transaction_prefix | String | No | - | If semantic is specified as EXACTLY_ONCE, the producer will write all messages in a Kafka transaction,kafka distinguishes different transactions by different transactionId. This parameter is prefix of kafka transactionId, make sure different job use different prefix. | +| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, ogg_json and avro.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | +| field_delimiter | String | No | , | Customize the field delimiter for data format. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../sink-common-options.md) for details | ## Parameter Interpretation @@ -216,56 +213,3 @@ sink { } ``` - -### Protobuf Configuration - -Set the `format` to `protobuf` and configure the `protobuf` data structure using the `protobuf_message_name` and `protobuf_schema` parameters. - -Example Usage: - -```hocon -sink { - kafka { - topic = "test_protobuf_topic_fake_source" - bootstrap.servers = "kafkaCluster:9092" - format = protobuf - kafka.request.timeout.ms = 60000 - kafka.config = { - acks = "all" - request.timeout.ms = 60000 - buffer.memory = 33554432 - } - protobuf_message_name = Person - protobuf_schema = """ - syntax = "proto3"; - - package org.apache.seatunnel.format.protobuf; - - option java_outer_classname = "ProtobufE2E"; - - message Person { - int32 c_int32 = 1; - int64 c_int64 = 2; - float c_float = 3; - double c_double = 4; - bool c_bool = 5; - string c_string = 6; - bytes c_bytes = 7; - - message Address { - string street = 1; - string city = 2; - string state = 3; - string zip = 4; - } - - Address address = 8; - - map attributes = 9; - - repeated string phone_numbers = 10; - } - """ - } -} -``` diff --git a/docs/en/connector-v2/sink/MongoDB.md b/docs/en/connector-v2/sink/MongoDB.md index 3d08dea4b43..e7e1fd7f877 100644 --- a/docs/en/connector-v2/sink/MongoDB.md +++ b/docs/en/connector-v2/sink/MongoDB.md @@ -27,9 +27,9 @@ This document describes how to set up the MongoDB connector to run data writers In order to use the Mongodb connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|---------------------------------------------------------------------------------------| -| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-mongodb) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|---------------------------------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-mongodb) | ## Data Type Mapping diff --git a/docs/en/connector-v2/sink/Mysql.md b/docs/en/connector-v2/sink/Mysql.md index 6151394b809..1b579aadd9f 100644 --- a/docs/en/connector-v2/sink/Mysql.md +++ b/docs/en/connector-v2/sink/Mysql.md @@ -4,7 +4,7 @@ ## Support Mysql Version -- 5.5/5.6/5.7/8.0/8.1/8.2/8.3/8.4 +- 5.5/5.6/5.7/8.0/8.4 ## Support Those Engines diff --git a/docs/en/connector-v2/sink/Paimon.md b/docs/en/connector-v2/sink/Paimon.md index 57ba427430e..6637644bd27 100644 --- a/docs/en/connector-v2/sink/Paimon.md +++ b/docs/en/connector-v2/sink/Paimon.md @@ -43,7 +43,7 @@ libfb303-xxx.jar | data_save_mode | Enum | No | APPEND_DATA | The data save mode | | paimon.table.primary-keys | String | No | - | Default comma-separated list of columns (primary key) that identify a row in tables.(Notice: The partition field needs to be included in the primary key fields) | | paimon.table.partition-keys | String | No | - | Default comma-separated list of partition fields to use when creating tables. | -| paimon.table.write-props | Map | No | - | Properties passed through to paimon table initialization, [reference](https://paimon.apache.org/docs/master/maintenance/configurations/#coreoptions). | +| paimon.table.write-props | Map | No | - | Properties passed through to paimon table initialization, [reference](https://paimon.apache.org/docs/0.8/maintenance/configurations/#coreoptions). | | paimon.hadoop.conf | Map | No | - | Properties in hadoop conf | | paimon.hadoop.conf-path | String | No | - | The specified loading path for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files | diff --git a/docs/en/connector-v2/sink/Qdrant.md b/docs/en/connector-v2/sink/Qdrant.md deleted file mode 100644 index e94598d2911..00000000000 --- a/docs/en/connector-v2/sink/Qdrant.md +++ /dev/null @@ -1,70 +0,0 @@ -# Qdrant - -> Qdrant Sink Connector - -## Description - -[Qdrant](https://qdrant.tech/) is a high-performance vector search engine and vector database. - -This connector can be used to write data into a Qdrant collection. - -## Data Type Mapping - -| SeaTunnel Data Type | Qdrant Data Type | -|---------------------|------------------| -| TINYINT | INTEGER | -| SMALLINT | INTEGER | -| INT | INTEGER | -| BIGINT | INTEGER | -| FLOAT | DOUBLE | -| DOUBLE | DOUBLE | -| BOOLEAN | BOOL | -| STRING | STRING | -| ARRAY | LIST | -| FLOAT_VECTOR | DENSE_VECTOR | -| BINARY_VECTOR | DENSE_VECTOR | -| FLOAT16_VECTOR | DENSE_VECTOR | -| BFLOAT16_VECTOR | DENSE_VECTOR | -| SPARSE_FLOAT_VECTOR | SPARSE_VECTOR | - -The value of the primary key column will be used as point ID in Qdrant. If no primary key is present, a random UUID will be used. - -## Options - -| name | type | required | default value | -|-----------------|--------|----------|---------------| -| collection_name | string | yes | - | -| batch_size | int | no | 64 | -| host | string | no | localhost | -| port | int | no | 6334 | -| api_key | string | no | - | -| use_tls | int | no | false | -| common-options | | no | - | - -### collection_name [string] - -The name of the Qdrant collection to read data from. - -### batch_size [int] - -The batch size of each upsert request to Qdrant. - -### host [string] - -The host name of the Qdrant instance. Defaults to "localhost". - -### port [int] - -The gRPC port of the Qdrant instance. - -### api_key [string] - -The API key to use for authentication if set. - -### use_tls [bool] - -Whether to use TLS(SSL) connection. Required if using Qdrant cloud(https). - -### common options - -Sink plugin common parameters, please refer to [Source Common Options](../sink-common-options.md) for details. diff --git a/docs/en/connector-v2/sink/Rabbitmq.md b/docs/en/connector-v2/sink/Rabbitmq.md index c7963525fba..489287249e0 100644 --- a/docs/en/connector-v2/sink/Rabbitmq.md +++ b/docs/en/connector-v2/sink/Rabbitmq.md @@ -57,21 +57,6 @@ convenience method for setting the fields in an AMQP URI: host, port, username, the queue to write the message to -### durable [boolean] - -true: The queue will survive a server restart. -false: The queue will be deleted on server restart. - -### exclusive [boolean] - -true: The queue is used only by the current connection and will be deleted when the connection closes. -false: The queue can be used by multiple connections. - -### auto_delete [boolean] - -true: The queue will be deleted automatically when the last consumer unsubscribes. -false: The queue will not be automatically deleted. - ### schema [Config] #### fields [Config] @@ -127,30 +112,6 @@ sink { } ``` -### Example 2 - -queue with durable, exclusive, auto_delete: - -```hocon -sink { - RabbitMQ { - host = "rabbitmq-e2e" - port = 5672 - virtual_host = "/" - username = "guest" - password = "guest" - queue_name = "test1" - durable = "true" - exclusive = "false" - auto_delete = "false" - rabbitmq.config = { - requested-heartbeat = 10 - connection-timeout = 10 - } - } -} -``` - ## Changelog ### next version diff --git a/docs/en/connector-v2/sink/Typesense.md b/docs/en/connector-v2/sink/Typesense.md deleted file mode 100644 index 8700d68dc77..00000000000 --- a/docs/en/connector-v2/sink/Typesense.md +++ /dev/null @@ -1,93 +0,0 @@ -# Typesense - -## Description - -Outputs data to `Typesense`. - -## Key Features - -- [ ] [Exactly Once](../../concept/connector-v2-features.md) -- [x] [CDC](../../concept/connector-v2-features.md) - -## Options - -| Name | Type | Required | Default Value | -|------------------|--------|----------|------------------------------| -| hosts | array | Yes | - | -| collection | string | Yes | - | -| schema_save_mode | string | Yes | CREATE_SCHEMA_WHEN_NOT_EXIST | -| data_save_mode | string | Yes | APPEND_DATA | -| primary_keys | array | No | | -| key_delimiter | string | No | `_` | -| api_key | string | No | | -| max_retry_count | int | No | 3 | -| max_batch_size | int | No | 10 | -| common-options | | No | - | - -### hosts [array] - -The access address for Typesense, formatted as `host:port`, e.g., `["typesense-01:8108"]`. - -### collection [string] - -The name of the collection to write to, e.g., "seatunnel". - -### primary_keys [array] - -Primary key fields used to generate the document `id`. - -### key_delimiter [string] - -Sets the delimiter for composite keys (default is `_`). - -### api_key [config] - -The `api_key` for secure access to Typesense. - -### max_retry_count [int] - -The maximum number of retry attempts for batch requests. - -### max_batch_size [int] - -The maximum size of document batches. - -### common options - -Common parameters for Sink plugins. Refer to [Common Sink Options](../source-common-options.md) for more details. - -### schema_save_mode - -Choose how to handle the target-side schema before starting the synchronization task: -- `RECREATE_SCHEMA`: Creates the table if it doesn’t exist, and deletes and recreates it if it does. -- `CREATE_SCHEMA_WHEN_NOT_EXIST`: Creates the table if it doesn’t exist, skips creation if it does. -- `ERROR_WHEN_SCHEMA_NOT_EXIST`: Throws an error if the table doesn’t exist. - -### data_save_mode - -Choose how to handle existing data on the target side before starting the synchronization task: -- `DROP_DATA`: Retains the database structure but deletes the data. -- `APPEND_DATA`: Retains both the database structure and the data. -- `ERROR_WHEN_DATA_EXISTS`: Throws an error if data exists. - -## Example - -Simple example: - -```bash -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["localhost:8108"] - collection = "typesense_to_typesense_sink_with_query" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","id"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "APPEND_DATA" - } -} -``` - diff --git a/docs/en/connector-v2/source/Clickhouse.md b/docs/en/connector-v2/source/Clickhouse.md index 47907bd3025..d867dbb701b 100644 --- a/docs/en/connector-v2/source/Clickhouse.md +++ b/docs/en/connector-v2/source/Clickhouse.md @@ -28,9 +28,9 @@ Used to read data from Clickhouse. In order to use the Clickhouse connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------------| -| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-clickhouse) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------------------------------| +| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | ## Data Type Mapping diff --git a/docs/en/connector-v2/source/Elasticsearch.md b/docs/en/connector-v2/source/Elasticsearch.md index 932f2c5a823..9fa1faa5d04 100644 --- a/docs/en/connector-v2/source/Elasticsearch.md +++ b/docs/en/connector-v2/source/Elasticsearch.md @@ -19,27 +19,24 @@ support version >= 2.x and <= 8.x. ## Options -| name | type | required | default value | -| ----------------------- | ------- | -------- | ------------------------------------------------------------ | -| hosts | array | yes | - | -| username | string | no | - | -| password | string | no | - | -| index | string | no | If the index list does not exist, the index must be configured | -| index_list | array | no | used to define a multiple table task | -| source | array | no | - | -| query | json | no | {"match_all": {}} | -| scroll_time | string | no | 1m | -| scroll_size | int | no | 100 | -| tls_verify_certificate | boolean | no | true | -| tls_verify_hostnames | boolean | no | true | -| array_column | map | no | | -| tls_keystore_path | string | no | - | -| tls_keystore_password | string | no | - | -| tls_truststore_path | string | no | - | -| tls_truststore_password | string | no | - | -| common-options | | no | - | - - +| name | type | required | default value | +|-------------------------|---------|----------|-------------------| +| hosts | array | yes | - | +| username | string | no | - | +| password | string | no | - | +| index | string | yes | - | +| source | array | no | - | +| query | json | no | {"match_all": {}} | +| scroll_time | string | no | 1m | +| scroll_size | int | no | 100 | +| tls_verify_certificate | boolean | no | true | +| tls_verify_hostnames | boolean | no | true | +| array_column | map | no | | +| tls_keystore_path | string | no | - | +| tls_keystore_password | string | no | - | +| tls_truststore_path | string | no | - | +| tls_truststore_password | string | no | - | +| common-options | | no | - | ### hosts [array] @@ -81,10 +78,6 @@ Amount of time Elasticsearch will keep the search context alive for scroll reque Maximum number of hits to be returned with each Elasticsearch scroll request. -### index_list [array] - -The `index_list` is used to define multi-index synchronization tasks. It is an array that contains the parameters required for single-table synchronization, such as `query`, `source/schema`, `scroll_size`, and `scroll_time`. It is recommended that `index_list` and `query` should not be configured at the same level simultaneously. Please refer to the upcoming multi-table synchronization example for more details. - ### tls_verify_certificate [boolean] Enable certificates validation for HTTPS endpoints @@ -115,94 +108,46 @@ Source plugin common parameters, please refer to [Source Common Options](../sour ## Examples -Demo 1 - -> This case will read data from indices matching the seatunnel-* pattern based on a query. The query will only return documents containing the id, name, age, tags, and phones fields. In this example, the source field configuration is used to specify which fields should be read, and the array_column is used to indicate that tags and phones should be treated as arrays. +simple ```hocon Elasticsearch { hosts = ["localhost:9200"] index = "seatunnel-*" - array_column = {tags = "array",phones = "array"} - source = ["_id","name","age","tags","phones"] + source = ["_id","name","age"] query = {"range":{"firstPacket":{"gte":1669225429990,"lte":1669225429990}}} } ``` -Demo 2 : Multi-table synchronization - -> This example demonstrates how to read different data from ``read_index1`` and ``read_index2`` and write separately to ``read_index1_copy``,``read_index2_copy``. -> in `read_index1`,I used source to specify the fields to be read and specify which fields are array fields using the 'array_column'. +complex ```hocon -source { - Elasticsearch { - hosts = ["https://elasticsearch:9200"] - username = "elastic" - password = "elasticsearch" - tls_verify_certificate = false - tls_verify_hostname = false - index_list = [ - { - index = "read_index1" - query = {"range": {"c_int": {"gte": 10, "lte": 20}}} - source = [ - c_map, - c_array, - c_string, - c_boolean, - c_tinyint, - c_smallint, - c_bigint, - c_float, - c_double, - c_decimal, - c_bytes, - c_int, - c_date, - c_timestamp] - array_column = { - c_array = "array" - } - } - { - index = "read_index2" - query = {"match_all": {}} - source = [ - c_int2, - c_date2, - c_null - ] - - } - - ] - - } -} - -transform { -} - -sink { - Elasticsearch { - hosts = ["https://elasticsearch:9200"] - username = "elastic" - password = "elasticsearch" - tls_verify_certificate = false - tls_verify_hostname = false - - index = "${table_name}_copy" - index_type = "st" - "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" - "data_save_mode"="APPEND_DATA" - } +Elasticsearch { + hosts = ["elasticsearch:9200"] + index = "st_index" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_decimal = "decimal(2, 1)" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + query = {"range":{"firstPacket":{"gte":1669225429990,"lte":1669225429990}}} } ``` - - -Demo 3 : SSL (Disable certificates validation) +SSL (Disable certificates validation) ```hocon source { @@ -216,7 +161,7 @@ source { } ``` -Demo 4 :SSL (Disable hostname validation) +SSL (Disable hostname validation) ```hocon source { @@ -230,7 +175,7 @@ source { } ``` -Demo 5 :SSL (Enable certificates validation) +SSL (Enable certificates validation) ```hocon source { @@ -251,4 +196,5 @@ source { - Add Elasticsearch Source Connector - [Feature] Support https protocol & compatible with opensearch ([3997](https://github.com/apache/seatunnel/pull/3997)) -- [Feature] Support DSL \ No newline at end of file +- [Feature] Support DSL + diff --git a/docs/en/connector-v2/source/Http.md b/docs/en/connector-v2/source/Http.md index 9c60b4c9aa4..318b8cf00a1 100644 --- a/docs/en/connector-v2/source/Http.md +++ b/docs/en/connector-v2/source/Http.md @@ -36,9 +36,9 @@ Supported DataSource Info In order to use the Http connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|------------------------------------------------------------------------------------| -| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-http) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------------------------| +| Http | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-http) | ## Source Options diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md index 27b3d875580..31257d85b12 100644 --- a/docs/en/connector-v2/source/Jdbc.md +++ b/docs/en/connector-v2/source/Jdbc.md @@ -39,7 +39,7 @@ supports query SQL and can achieve projection effect. ## Options -| name | type | required | default value | description | +| 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`. | @@ -52,7 +52,6 @@ supports query SQL and can achieve projection effect. | 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. | -| decimal_type_narrowing | Boolean | No | true | Decimal type narrowing, if true, the decimal type will be narrowed to the int or long type if without loss of precision. Only support for Oracle at now. Please refer to `decimal_type_narrowing` below | | 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. | @@ -67,28 +66,6 @@ supports query SQL and can achieve projection effect. | 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](../source-common-options.md) for details. | -### decimal_type_narrowing - -Decimal type narrowing, if true, the decimal type will be narrowed to the int or long type if without loss of precision. Only support for Oracle at now. - -eg: - -decimal_type_narrowing = true - -| Oracle | SeaTunnel | -|---------------|-----------| -| NUMBER(1, 0) | Boolean | -| NUMBER(6, 0) | INT | -| NUMBER(10, 0) | BIGINT | - -decimal_type_narrowing = false - -| Oracle | SeaTunnel | -|---------------|----------------| -| NUMBER(1, 0) | Decimal(1, 0) | -| NUMBER(6, 0) | Decimal(6, 0) | -| NUMBER(10, 0) | Decimal(10, 0) | - ## Parallel Reader The JDBC Source connector supports parallel reading of data from tables. SeaTunnel will use certain rules to split the data in the table, which will be handed over to readers for reading. The number of readers is determined by the `parallelism` option. @@ -122,7 +99,7 @@ there are some reference value for params above. | sqlserver | com.microsoft.sqlserver.jdbc.SQLServerDriver | jdbc:sqlserver://localhost:1433 | https://mvnrepository.com/artifact/com.microsoft.sqlserver/mssql-jdbc | | oracle | oracle.jdbc.OracleDriver | jdbc:oracle:thin:@localhost:1521/xepdb1 | https://mvnrepository.com/artifact/com.oracle.database.jdbc/ojdbc8 | | sqlite | org.sqlite.JDBC | jdbc:sqlite:test.db | https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc | -| gbase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar | +| gbase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar | | starrocks | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | https://mvnrepository.com/artifact/mysql/mysql-connector-java | | db2 | com.ibm.db2.jcc.DB2Driver | jdbc:db2://localhost:50000/testdb | https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc/db2jcc4 | | tablestore | com.alicloud.openservices.tablestore.jdbc.OTSDriver | "jdbc:ots:http s://myinstance.cn-hangzhou.ots.aliyuncs.com/myinstance" | https://mvnrepository.com/artifact/com.aliyun.openservices/tablestore-jdbc | diff --git a/docs/en/connector-v2/source/MongoDB-CDC.md b/docs/en/connector-v2/source/MongoDB-CDC.md index 301d7075738..a35c1966663 100644 --- a/docs/en/connector-v2/source/MongoDB-CDC.md +++ b/docs/en/connector-v2/source/MongoDB-CDC.md @@ -25,9 +25,9 @@ The MongoDB CDC connector allows for reading snapshot data and incremental data In order to use the Mongodb CDC connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|-------------------------------------------------------------------------------------------| -| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-cdc-mongodb) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|-------------------------------------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-cdc-mongodb) | ## Availability Settings diff --git a/docs/en/connector-v2/source/MongoDB.md b/docs/en/connector-v2/source/MongoDB.md index 6605aacce5f..694cce53525 100644 --- a/docs/en/connector-v2/source/MongoDB.md +++ b/docs/en/connector-v2/source/MongoDB.md @@ -27,9 +27,9 @@ This document describes how to set up the MongoDB connector to run data reads ag In order to use the Mongodb connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Dependency | -|------------|--------------------|---------------------------------------------------------------------------------------| -| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-mongodb) | +| Datasource | Supported Versions | Dependency | +|------------|--------------------|---------------------------------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-mongodb) | ## Data Type Mapping diff --git a/docs/en/connector-v2/source/MyHours.md b/docs/en/connector-v2/source/MyHours.md index fc2805b45c5..62dffbd352f 100644 --- a/docs/en/connector-v2/source/MyHours.md +++ b/docs/en/connector-v2/source/MyHours.md @@ -37,7 +37,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor | Datasource | Supported Versions | Dependency | |------------|--------------------|---------------------------------------------------------------------------------------------| -| My Hours | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel) | +| My Hours | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2) | ## Source Options diff --git a/docs/en/connector-v2/source/Mysql.md b/docs/en/connector-v2/source/Mysql.md index 57f40ac7b19..45a6bce18d7 100644 --- a/docs/en/connector-v2/source/Mysql.md +++ b/docs/en/connector-v2/source/Mysql.md @@ -8,7 +8,7 @@ Read external data source data through JDBC. ## Support Mysql Version -- 5.5/5.6/5.7/8.0/8.1/8.2/8.3/8.4 +- 5.5/5.6/5.7/8.0/8.4 ## Support Those Engines @@ -67,7 +67,7 @@ Read external data source data through JDBC. ## Source Options -| Name | Type | Required | Default | Description | +| Name | Type | Required | Default | Description | |--------------------------------------------|------------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/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`. | @@ -81,8 +81,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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" | -| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| table_path | Int | No | 0 | 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" | +| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/Opengauss-CDC.md b/docs/en/connector-v2/source/Opengauss-CDC.md deleted file mode 100644 index 81691ea1ff4..00000000000 --- a/docs/en/connector-v2/source/Opengauss-CDC.md +++ /dev/null @@ -1,170 +0,0 @@ -# Opengauss CDC - -> Opengauss CDC source connector - -## Support Those Engines - -> SeaTunnel Zeta
-> Flink
- -## Key features - -- [ ] [batch](../../concept/connector-v2-features.md) -- [x] [stream](../../concept/connector-v2-features.md) -- [x] [exactly-once](../../concept/connector-v2-features.md) -- [ ] [column projection](../../concept/connector-v2-features.md) -- [x] [parallelism](../../concept/connector-v2-features.md) -- [x] [support user-defined split](../../concept/connector-v2-features.md) - -## Description - -The Opengauss CDC connector allows for reading snapshot data and incremental data from Opengauss database. This document -describes how to set up the Opengauss CDC connector to run SQL queries against Opengauss databases. - -## Using steps - -> Here are the steps to enable CDC (Change Data Capture) in Opengauss: - -1. Ensure the wal_level is set to logical, you can use SQL commands to modify the configuration directly: - -```sql -ALTER SYSTEM SET wal_level TO 'logical'; -SELECT pg_reload_conf(); -``` - -2. Change the REPLICA policy of the specified table to FULL - -```sql -ALTER TABLE your_table_name REPLICA IDENTITY FULL; -``` - -If you have multi tables,you can use the result of this sql to change the REPLICA policy of all tables to FULL - -```sql -select 'ALTER TABLE ' || schemaname || '.' || tablename || ' REPLICA IDENTITY FULL;' from pg_tables where schemaname = 'YourTableSchema' -``` - -## Data Type Mapping - -| Opengauss Data type | SeaTunnel Data type | -|-----------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------| -| BOOL
| BOOLEAN | -| BYTEA
| BYTES | -| INT2
SMALLSERIAL
INT4
SERIAL
| INT | -| INT8
BIGSERIAL
| BIGINT | -| FLOAT4
| FLOAT | -| FLOAT8
| DOUBLE | -| NUMERIC(Get the designated column's specified column size>0) | DECIMAL(Get the designated column's specified column size,Gets the number of digits in the specified column to the right of the decimal point) | -| NUMERIC(Get the designated column's specified column size<0) | DECIMAL(38, 18) | -| BPCHAR
CHARACTER
VARCHAR
TEXT
GEOMETRY
GEOGRAPHY
JSON
JSONB | STRING | -| TIMESTAMP
| TIMESTAMP | -| TIME
| TIME | -| DATE
| DATE | -| OTHER DATA TYPES | NOT SUPPORTED YET | - -## Source Options - -| Name | Type | Required | Default | Description | -|------------------------------------------------|----------|----------|----------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| base-url | String | Yes | - | The URL of the JDBC connection. Refer to a case: `jdbc:postgresql://localhost:5432/postgres_cdc?loggerLevel=OFF`. | -| username | String | Yes | - | Username of the database to use when connecting to the database server. | -| password | String | Yes | - | Password to use when connecting to the database server. | -| database-names | List | No | - | Database name of the database to monitor. | -| table-names | List | Yes | - | Table name of the database to monitor. The table name needs to include the database name, for example: `database_name.table_name` | -| table-names-config | List | No | - | Table config list. for example: [{"table": "db1.schema1.table1","primaryKeys":["key1"]}] | -| startup.mode | Enum | No | INITIAL | Optional startup mode for Opengauss CDC consumer, valid enumerations are `initial`, `earliest`, `latest` and `specific`.
`initial`: Synchronize historical data at startup, and then synchronize incremental data.
`earliest`: Startup from the earliest offset possible.
`latest`: Startup from the latest offset.
`specific`: Startup from user-supplied specific offsets. | -| snapshot.split.size | Integer | No | 8096 | The split size (number of rows) of table snapshot, captured tables are split into multiple splits when read the snapshot of table. | -| snapshot.fetch.size | Integer | No | 1024 | The maximum fetch size for per poll when read table snapshot. | -| slot.name | String | No | - | The name of the Opengauss logical decoding slot that was created for streaming changes from a particular plug-in for a particular database/schema. The server uses this slot to stream events to the connector that you are configuring. Default is seatunnel. | -| decoding.plugin.name | String | No | pgoutput | The name of the Postgres logical decoding plug-in installed on the server,Supported values are decoderbufs, wal2json, wal2json_rds, wal2json_streaming,wal2json_rds_streaming and pgoutput. | -| server-time-zone | String | No | UTC | The session time zone in database server. If not set, then ZoneId.systemDefault() is used to determine the server time zone. | -| connect.timeout.ms | Duration | No | 30000 | The maximum time that the connector should wait after trying to connect to the database server before timing out. | -| connect.max-retries | Integer | No | 3 | The max retry times that the connector should retry to build database server connection. | -| connection.pool.size | Integer | No | 20 | The jdbc connection pool size. | -| chunk-key.even-distribution.factor.upper-bound | Double | No | 100 | 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. | -| chunk-key.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | -| 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. | -| format | Enum | No | DEFAULT | Optional output format for Opengauss 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/postgresql.adoc#connector-configuration-properties) to Debezium Embedded Engine which is used to capture data changes from Opengauss server. | -| common-options | | no | - | Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details | - -## Task Example - -### Simple - -> Support multi-table reading - -``` - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1","opengauss_cdc.inventory.opengauss_cdc_table_2"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" - decoding.plugin.name = "pgoutput" - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = "opengauss_cdc" - schema = "inventory" - tablePrefix = "sink_" - primary_keys = ["id"] - } -} - -``` - -### Support custom primary key for table - -``` -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - exactly_once = true - table-names-config = [ - { - table = "opengauss_cdc.inventory.full_types_no_primary_key" - primaryKeys = ["id"] - } - ] - } -} -``` - diff --git a/docs/en/connector-v2/source/Oracle.md b/docs/en/connector-v2/source/Oracle.md index 1756a76db1d..b7a28225206 100644 --- a/docs/en/connector-v2/source/Oracle.md +++ b/docs/en/connector-v2/source/Oracle.md @@ -81,7 +81,7 @@ Read external data source data through JDBC. | 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 toimprove 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. | -| Name | Type | Required | Default | Description | +| Name | Type | Required | Default | Description | |--------------------------------------------|------------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/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`. | @@ -95,8 +95,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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" | -| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| table_path | Int | No | 0 | 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" | +| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/PostgreSQL.md b/docs/en/connector-v2/source/PostgreSQL.md index 101902d3618..b6e95c8ad7d 100644 --- a/docs/en/connector-v2/source/PostgreSQL.md +++ b/docs/en/connector-v2/source/PostgreSQL.md @@ -74,9 +74,24 @@ Read external data source data through JDBC. ## Options +| Name | Type | Required | Default | Description | +|------------------------------|------------|----------|-----------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost:5432/test | +| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use PostgreSQL the value is `org.postgresql.Driver`. | +| user | String | No | - | Connection instance user name | +| password | String | No | - | Connection instance password | +| query | String | Yes | - | Query statement | +| 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 parallelism's partition, only support numeric type,Only support numeric type primary key, and only can config one column. | +| partition_lower_bound | BigDecimal | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | +| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. | +| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | +| 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 toimprove 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. | + | Name | Type | Required | Default | Description | |--------------------------------------------|------------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:postgresql://localhost:5432/test | +| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:mysql://localhost:3306:3306/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 | - | Connection instance user name | | password | String | No | - | Connection instance password | @@ -88,8 +103,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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" | -| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| table_path | Int | No | 0 | 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" | +| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/Qdrant.md b/docs/en/connector-v2/source/Qdrant.md deleted file mode 100644 index c523cf1da69..00000000000 --- a/docs/en/connector-v2/source/Qdrant.md +++ /dev/null @@ -1,81 +0,0 @@ -# Qdrant - -> Qdrant source connector - -## Description - -[Qdrant](https://qdrant.tech/) is a high-performance vector search engine and vector database. - -This connector can be used to read data from a Qdrant collection. - -## Options - -| name | type | required | default value | -|-----------------|--------|----------|---------------| -| collection_name | string | yes | - | -| schema | config | yes | - | -| host | string | no | localhost | -| port | int | no | 6334 | -| api_key | string | no | - | -| use_tls | int | no | false | -| common-options | | no | - | - -### collection_name [string] - -The name of the Qdrant collection to read data from. - -### schema [config] - -The schema of the table to read data into. - -Eg: - -```hocon -schema = { - fields { - age = int - address = string - some_vector = float_vector - } -} -``` - -Each entry in Qdrant is called a point. - -The `float_vector` type columns are read from the vectors of each point, others are read from the JSON payload associated with the point. - -If a column is marked as primary key, the ID of the Qdrant point is written into it. It can be of type `"string"` or `"int"`. Since Qdrant only [allows](https://qdrant.tech/documentation/concepts/points/#point-ids) positive integers and UUIDs as point IDs. - -If the collection was created with a single default/unnamed vector, use `default_vector` as the vector name. - -```hocon -schema = { - fields { - age = int - address = string - default_vector = float_vector - } -} -``` - -The ID of the point in Qdrant will be written into the column which is marked as the primary key. It can be of type `int` or `string`. - -### host [string] - -The host name of the Qdrant instance. Defaults to "localhost". - -### port [int] - -The gRPC port of the Qdrant instance. - -### api_key [string] - -The API key to use for authentication if set. - -### use_tls [bool] - -Whether to use TLS(SSL) connection. Required if using Qdrant cloud(https). - -### common options - -Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details. diff --git a/docs/en/connector-v2/source/Sls.md b/docs/en/connector-v2/source/Sls.md index 609676e8ff9..6468f397ab7 100644 --- a/docs/en/connector-v2/source/Sls.md +++ b/docs/en/connector-v2/source/Sls.md @@ -26,9 +26,9 @@ Source connector for Aliyun Sls. In order to use the Sls connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Maven | -|------------|--------------------|-----------------------------------------------------------------------------------| -| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-sls) | +| Datasource | Supported Versions | Maven | +|------------|--------------------|-----------------------------------------------------------------------------------------------------------| +| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-sls) | ## Source Options diff --git a/docs/en/connector-v2/source/SqlServer.md b/docs/en/connector-v2/source/SqlServer.md index 2905f45fbdb..bbcd6751a21 100644 --- a/docs/en/connector-v2/source/SqlServer.md +++ b/docs/en/connector-v2/source/SqlServer.md @@ -67,7 +67,7 @@ Read external data source data through JDBC. ## Source Options -| name | type | required | default | Description | +| name | type | required | default | Description | |--------------------------------------------|--------|----------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:sqlserver://127.0.0.1:1434;database=TestDB | | driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use SQLserver the value is `com.microsoft.sqlserver.jdbc.SQLServerDriver`. | @@ -81,8 +81,8 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | 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 toimprove 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" | -| table_list | Array | No | - | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | +| table_path | Int | No | 0 | 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" | +| table_list | Array | No | 0 | The list of tables to be read, you can use this configuration instead of `table_path` example: ```[{ table_path = "testdb.table1"}, {table_path = "testdb.table2", query = "select * id, name from testdb.table2"}]``` | | 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 | The split size (number of rows) of table, captured tables are split into multiple splits when read of table. | | split.even-distribution.factor.lower-bound | Double | No | 0.05 | 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. | diff --git a/docs/en/connector-v2/source/Tablestore.md b/docs/en/connector-v2/source/Tablestore.md deleted file mode 100644 index 8e0d1aeebc7..00000000000 --- a/docs/en/connector-v2/source/Tablestore.md +++ /dev/null @@ -1,102 +0,0 @@ -# Tablestore - -> Tablestore source connector - -## Description - -Read data from Alicloud Tablestore,support full and CDC. - - -## Key features - -- [ ] [batch](../../concept/connector-v2-features.md) -- [X] [stream](../../concept/connector-v2-features.md) -- [ ] [exactly-once](../../concept/connector-v2-features.md) -- [ ] [column projection](../../concept/connector-v2-features.md) -- [ ] [parallelism](../../concept/connector-v2-features.md) -- [ ] [support user-defined split](../../concept/connector-v2-features.md) - -## Options - -| name | type | required | default value | -|-----------------------|--------|----------|---------------| -| end_point | string | yes | - | -| instance_name | string | yes | - | -| access_key_id | string | yes | - | -| access_key_secret | string | yes | - | -| table | string | yes | - | -| primary_keys | array | yes | - | -| schema | config | yes | - | - - -### end_point [string] - -The endpoint of Tablestore. - -### instance_name [string] - -The intance name of Tablestore. - -### access_key_id [string] - -The access id of Tablestore. - -### access_key_secret [string] - -The access secret of Tablestore. - -### table [string] - -The table name of Tablestore. - -### primary_keys [array] - -The primarky key of table,just add a unique primary key. - -### schema [Config] - - - -## Example - -```bash -env { - parallelism = 1 - job.mode = "STREAMING" -} - -source { - # This is a example source plugin **only for test and demonstrate the feature source plugin** - Tablestore { - end_point = "https://****.cn-zhangjiakou.tablestore.aliyuncs.com" - instance_name = "****" - access_key_id="***************2Ag5" - access_key_secret="***********2Dok" - table="test" - primary_keys=["id"] - schema={ - fields { - id = string - name = string - } - } - } -} - - -sink { - MongoDB{ - uri = "mongodb://localhost:27017" - database = "test" - collection = "test" - primary-key = ["id"] - schema = { - fields { - id = string - name = string - } - } - } -} -``` - diff --git a/docs/en/connector-v2/source/Typesense.md b/docs/en/connector-v2/source/Typesense.md deleted file mode 100644 index 9f88a58d4a5..00000000000 --- a/docs/en/connector-v2/source/Typesense.md +++ /dev/null @@ -1,79 +0,0 @@ -# Typesense - -> Typesense Source Connector - -## Description - -Reads data from Typesense. - -## Key Features - -- [x] [Batch Processing](../../concept/connector-v2-features.md) -- [ ] [Stream Processing](../../concept/connector-v2-features.md) -- [ ] [Exactly-Once](../../concept/connector-v2-features.md) -- [x] [Schema](../../concept/connector-v2-features.md) -- [x] [Parallelism](../../concept/connector-v2-features.md) -- [ ] [User-Defined Splits Support](../../concept/connector-v2-features.md) - -## Options - -| Name | Type | Required | Default | -|------------|--------|----------|---------| -| hosts | array | yes | - | -| collection | string | yes | - | -| schema | config | yes | - | -| api_key | string | no | - | -| query | string | no | - | -| batch_size | int | no | 100 | - -### hosts [array] - -The access address of Typesense, for example: `["typesense-01:8108"]`. - -### collection [string] - -The name of the collection to write to, for example: `"seatunnel"`. - -### schema [config] - -The columns to be read from Typesense. For more information, please refer to the [guide](../../concept/schema-feature.md#how-to-declare-type-supported). - -### api_key [config] - -The `api_key` for Typesense security authentication. - -### batch_size - -The number of records to query per batch when reading data. - -### Common Options - -For common parameters of Source plugins, please refer to [Source Common Options](../source-common-options.md). - -## Example - -```bash -source { - Typesense { - hosts = ["localhost:8108"] - collection = "companies" - api_key = "xyz" - query = "q=*&filter_by=num_employees:>9000" - schema = { - fields { - company_name_list = array - company_name = string - num_employees = long - country = string - id = string - c_row = { - c_int = int - c_string = string - c_array_int = array - } - } - } - } -} -``` - diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index e9259fae484..42106fc7d5f 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -26,32 +26,30 @@ Source connector for Apache Kafka. In order to use the Kafka connector, the following dependencies are required. They can be downloaded via install-plugin.sh or from the Maven central repository. -| Datasource | Supported Versions | Maven | -|------------|--------------------|-------------------------------------------------------------------------------------| -| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | +| Datasource | Supported Versions | Maven | +|------------|--------------------|-------------------------------------------------------------------------------------------------------------| +| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | ## Source Options -| Name | Type | Required | Default | Description | -|-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| topic | String | Yes | - | Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by comma like 'topic-1,topic-2'. | -| table_list | Map | No | - | Topic list config You can configure only one `table_list` and one `topic` at the same time | -| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | -| pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | -| consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | -| commit_on_checkpoint | Boolean | No | true | If true the consumer's offset will be periodically committed in the background. | -| kafka.config | Map | No | - | In addition to the above necessary parameters that must be specified by the `Kafka consumer` client, users can also specify multiple `consumer` client non-mandatory parameters, covering [all consumer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#consumerconfigs). | -| schema | Config | No | - | The structure of the data, including field names and field types. | -| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, maxwell_json, ogg_json, avro and protobuf. If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. Some format details please refer [formats](../formats) | -| format_error_handle_way | String | No | fail | The processing method of data format error. The default value is fail, and the optional value is (fail, skip). When fail is selected, data format error will block and an exception will be thrown. When skip is selected, data format error will skip this line data. | -| field_delimiter | String | No | , | Customize the field delimiter for data format. | -| start_mode | StartMode[earliest],[group_offsets],[latest],[specific_offsets],[timestamp] | No | group_offsets | The initial consumption pattern of consumers. | -| start_mode.offsets | Config | No | - | The offset required for consumption mode to be specific_offsets. | -| start_mode.timestamp | Long | No | - | The time required for consumption mode to be "timestamp". | -| partition-discovery.interval-millis | Long | No | -1 | The interval for dynamically discovering topics and partitions. | -| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details | -| protobuf_message_name | String | No | - | Effective when the format is set to protobuf, specifies the Message name | -| protobuf_schema | String | No | - | Effective when the format is set to protobuf, specifies the Schema definition | +| Name | Type | Required | Default | Description | +|-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | Yes | - | Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by comma like 'topic-1,topic-2'. | +| table_list | Map | No | - | Topic list config You can configure only one `table_list` and one `topic` at the same time | +| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | +| pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | +| consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | +| commit_on_checkpoint | Boolean | No | true | If true the consumer's offset will be periodically committed in the background. | +| kafka.config | Map | No | - | In addition to the above necessary parameters that must be specified by the `Kafka consumer` client, users can also specify multiple `consumer` client non-mandatory parameters, covering [all consumer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#consumerconfigs). | +| schema | Config | No | - | The structure of the data, including field names and field types. | +| format | String | No | json | Data format. The default format is json. Optional text format, canal_json, debezium_json, ogg_json and avro.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | +| format_error_handle_way | String | No | fail | The processing method of data format error. The default value is fail, and the optional value is (fail, skip). When fail is selected, data format error will block and an exception will be thrown. When skip is selected, data format error will skip this line data. | +| field_delimiter | String | No | , | Customize the field delimiter for data format. | +| start_mode | StartMode[earliest],[group_offsets],[latest],[specific_offsets],[timestamp] | No | group_offsets | The initial consumption pattern of consumers. | +| start_mode.offsets | Config | No | - | The offset required for consumption mode to be specific_offsets. | +| start_mode.timestamp | Long | No | - | The time required for consumption mode to be "timestamp". | +| partition-discovery.interval-millis | Long | No | -1 | The interval for dynamically discovering topics and partitions. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details | ## Task Example @@ -244,51 +242,3 @@ sink { } ``` -### Protobuf configuration - -Set `format` to `protobuf`, configure `protobuf` data structure, `protobuf_message_name` and `protobuf_schema` parameters - -Example: - -```hocon -source { - Kafka { - topic = "test_protobuf_topic_fake_source" - format = protobuf - protobuf_message_name = Person - protobuf_schema = """ - syntax = "proto3"; - - package org.apache.seatunnel.format.protobuf; - - option java_outer_classname = "ProtobufE2E"; - - message Person { - int32 c_int32 = 1; - int64 c_int64 = 2; - float c_float = 3; - double c_double = 4; - bool c_bool = 5; - string c_string = 6; - bytes c_bytes = 7; - - message Address { - string street = 1; - string city = 2; - string state = 3; - string zip = 4; - } - - Address address = 8; - - map attributes = 9; - - repeated string phone_numbers = 10; - } - """ - bootstrap.servers = "kafkaCluster:9092" - start_mode = "earliest" - result_table_name = "kafka_table" - } -} -``` \ No newline at end of file diff --git a/docs/en/faq.md b/docs/en/faq.md index 02c125ad4fd..2e50c9d4618 100644 --- a/docs/en/faq.md +++ b/docs/en/faq.md @@ -203,6 +203,23 @@ spark { } ``` +## How do I specify a different JDK version for SeaTunnel on YARN? + +For example, if you want to set the JDK version to JDK8, there are two cases: + +- The YARN cluster has deployed JDK8, but the default JDK is not JDK8. Add two configurations to the SeaTunnel config file: + + ``` + env { + ... + spark.executorEnv.JAVA_HOME="/your/java_8_home/directory" + spark.yarn.appMasterEnv.JAVA_HOME="/your/java_8_home/directory" + ... + } + ``` +- YARN cluster does not deploy JDK8. At this time, start SeaTunnel attached with JDK8. For detailed operations, see: + https://www.cnblogs.com/jasondan/p/spark-specific-jdk-version.html + ## What should I do if OOM always appears when running SeaTunnel in Spark local[*] mode? If you run in local mode, you need to modify the `start-seatunnel.sh` startup script. After `spark-submit`, add a parameter `--driver-memory 4g` . Under normal circumstances, local mode is not used in the production environment. Therefore, this parameter generally does not need to be set during On YARN. See: [Application Properties](https://spark.apache.org/docs/latest/configuration.html#application-properties) for details. @@ -317,6 +334,10 @@ spark-submit --verbose ... ``` +## How do I use SeaTunnel to synchronize data across HDFS clusters? + +Just configure hdfs-site.xml properly. Refer to: https://www.cnblogs.com/suanec/p/7828139.html. + ## I want to learn the source code of SeaTunnel. Where should I start? SeaTunnel has a completely abstract and structured code implementation, and many people have chosen SeaTunnel As a way to learn Spark. You can learn the source code from the main program entry: SeaTunnel.java diff --git a/docs/en/seatunnel-engine/checkpoint-storage.md b/docs/en/seatunnel-engine/checkpoint-storage.md index 7027f8067fb..f5dd44e3af6 100644 --- a/docs/en/seatunnel-engine/checkpoint-storage.md +++ b/docs/en/seatunnel-engine/checkpoint-storage.md @@ -67,6 +67,7 @@ seatunnel: fs.oss.accessKeyId: your-access-key fs.oss.accessKeySecret: your-secret-key fs.oss.endpoint: endpoint address + fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` For additional reading on the Hadoop Credential Provider API, you can see: [Credential Provider API](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/CredentialProviderAPI.html). diff --git a/docs/en/seatunnel-engine/rest-api.md b/docs/en/seatunnel-engine/rest-api.md index c2c3e06b79c..8b6cdb231b5 100644 --- a/docs/en/seatunnel-engine/rest-api.md +++ b/docs/en/seatunnel-engine/rest-api.md @@ -389,106 +389,6 @@ When we can't get the job info, the response will be: ------------------------------------------------------------------------------------------ -### Batch Submit Jobs - -
-POST /hazelcast/rest/maps/submit-jobs (Returns jobId and jobName if the job is successfully submitted.) - -#### Parameters (add in the `params` field in the request body) - -> | Parameter Name | Required | Type | Description | -> |----------------------|--------------|---------|---------------------------------------| -> | jobId | optional | string | job id | -> | jobName | optional | string | job name | -> | isStartWithSavePoint | optional | string | if the job is started with save point | - -#### Request Body - -```json -[ - { - "params":{ - "jobId":"123456", - "jobName":"SeaTunnel-01" - }, - "env": { - "job.mode": "batch" - }, - "source": [ - { - "plugin_name": "FakeSource", - "result_table_name": "fake", - "row.num": 1000, - "schema": { - "fields": { - "name": "string", - "age": "int", - "card": "int" - } - } - } - ], - "transform": [ - ], - "sink": [ - { - "plugin_name": "Console", - "source_table_name": ["fake"] - } - ] - }, - { - "params":{ - "jobId":"1234567", - "jobName":"SeaTunnel-02" - }, - "env": { - "job.mode": "batch" - }, - "source": [ - { - "plugin_name": "FakeSource", - "result_table_name": "fake", - "row.num": 1000, - "schema": { - "fields": { - "name": "string", - "age": "int", - "card": "int" - } - } - } - ], - "transform": [ - ], - "sink": [ - { - "plugin_name": "Console", - "source_table_name": ["fake"] - } - ] - } -] -``` - -#### Response - -```json -[ - { - "jobId": "123456", - "jobName": "SeaTunnel-01" - },{ - "jobId": "1234567", - "jobName": "SeaTunnel-02" - } -] -``` - -
- ------------------------------------------------------------------------------------------- - ### Stop A Job
@@ -514,42 +414,7 @@ When we can't get the job info, the response will be:
------------------------------------------------------------------------------------------ -### Batch Stop Jobs - -
-POST /hazelcast/rest/maps/stop-jobs (Returns jobId if the job is successfully stopped.) - -#### Request Body - -```json -[ - { - "jobId": 881432421482889220, - "isStopWithSavePoint": false - }, - { - "jobId": 881432456517910529, - "isStopWithSavePoint": false - } -] -``` - -#### Response - -```json -[ - { - "jobId": 881432421482889220 - }, - { - "jobId": 881432456517910529 - } -] -``` - -
------------------------------------------------------------------------------------------- ### Encrypt Config
diff --git a/docs/en/start-v2/docker/docker.md b/docs/en/start-v2/docker/docker.md index 67f57c590ae..5c2c8422224 100644 --- a/docs/en/start-v2/docker/docker.md +++ b/docs/en/start-v2/docker/docker.md @@ -2,7 +2,7 @@ sidebar_position: 3 --- -# Set Up With Docker In Local Mode +# Set Up with Docker in local mode ## Zeta Engine @@ -15,53 +15,17 @@ docker pull apache/seatunnel: How to submit job in local mode ```shell -# Run fake source to console sink -docker run --rm -it apache/seatunnel: ./bin/seatunnel.sh -m local -c config/v2.batch.config.template +docker run --rm -it apache/seatunnel bash ./bin/seatunnel.sh -e local -c -# Run job with custom config file -docker run --rm -it -v //:/config apache/seatunnel: ./bin/seatunnel.sh -m local -c /config/fake_to_console.conf -# Example -# If you config file is in /tmp/job/fake_to_console.conf -docker run --rm -it -v /tmp/job/:/config apache/seatunnel: ./bin/seatunnel.sh -m local -c /config/fake_to_console.conf +# eg: a fake source to console sink +docker run --rm -it apache/seatunnel bash ./bin/seatunnel.sh -e local -c config/v2.batch.config.template -# Set JVM options when running -docker run --rm -it -v /tmp/job/:/config apache/seatunnel: ./bin/seatunnel.sh -DJvmOption="-Xms4G -Xmx4G" -m local -c /config/fake_to_console.conf ``` ### Build Image By Yourself -Build from source code. The way of downloading the source code is the same as the way of downloading the binary package. -You can download the source code from the [download page](https://seatunnel.apache.org/download/) or clone the source code from the [GitHub repository](https://github.com/apache/seatunnel/releases) - -#### Build With One Command -```shell -cd seatunnel -# Use already sett maven profile -sh ./mvnw -B clean install -Dmaven.test.skip=true -Dmaven.javadoc.skip=true -Dlicense.skipAddThirdParty=true -D"docker.build.skip"=false -D"docker.verify.skip"=false -D"docker.push.skip"=true -D"docker.tag"=2.3.8 -Dmaven.deploy.skip --no-snapshot-updates -Pdocker,seatunnel - -# Check the docker image -docker images | grep apache/seatunnel -``` - -#### Build Step By Step -```shell -# Build binary package from source code -sh ./mvnw clean package -DskipTests -Dskip.spotless=true - -# Build docker image -cd seatunnel-dist -docker build -f src/main/docker/Dockerfile --build-arg VERSION=2.3.8 -t apache/seatunnel:2.3.8 . - -# If you build from dev branch, you should add SNAPSHOT suffix to the version -docker build -f src/main/docker/Dockerfile --build-arg VERSION=2.3.8-SNAPSHOT -t apache/seatunnel:2.3.8-SNAPSHOT . - -# Check the docker image -docker images | grep apache/seatunnel -``` - -The Dockerfile is like this: -```dockerfile +```Dockerfile FROM openjdk:8 ARG VERSION @@ -100,7 +64,7 @@ docker run \ Or you can change the `SPARK_HOME`, `FLINK_HOME` environment variable in Dockerfile and re-build your and mount the spark/flink to related path. -```dockerfile +```Dockerfile FROM apache/seatunnel ENV SPARK_HOME= diff --git a/docs/en/start-v2/locally/deployment.md b/docs/en/start-v2/locally/deployment.md index db5dbdd0fd1..7940001b7f2 100644 --- a/docs/en/start-v2/locally/deployment.md +++ b/docs/en/start-v2/locally/deployment.md @@ -5,17 +5,15 @@ sidebar_position: 2 import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -# Deployment +# Download and Make Installation Packages -## Preparation +## Step 1: Preparation Before starting to download SeaTunnel, you need to ensure that you have installed the following software required by SeaTunnel: * Install [Java](https://www.java.com/en/download/) (Java 8 or 11, and other versions higher than Java 8 can theoretically work) and set `JAVA_HOME`. -## Download SeaTunnel Release Package - -### Download The Binary Package +## Step 2: Download SeaTunnel Visit the [SeaTunnel Download Page](https://seatunnel.apache.org/download) to download the latest binary package `seatunnel--bin.tar.gz`. @@ -27,7 +25,7 @@ wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${ve tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` -### Download The Connector Plugins +## Step 3: Download The Connector Plugins Starting from version 2.2.0-beta, the binary package no longer provides connector dependencies by default. Therefore, the first time you use it, you need to run the following command to install the connectors (Alternatively, you can manually download the connectors from the [Apache Maven Repository](https://repo.maven.apache.org/maven2/org/apache/seatunnel/) and move them to the `connectors/` directory. For versions before 2.3.5, place them in the `connectors/seatunnel` directory): @@ -58,33 +56,10 @@ If you want to install connector plugins by manually downloading connectors, you ::: -## Build SeaTunnel From Source Code - -### Download The Source Code - -Build from source code. The way of downloading the source code is the same as the way of downloading the binary package. -You can download the source code from the [download page](https://seatunnel.apache.org/download/) or clone the source code from the [GitHub repository](https://github.com/apache/seatunnel/releases) - -### Build The Source Code - -```shell -cd seatunnel -sh ./mvnw clean package -DskipTests -Dskip.spotless=true -# get the binary package -cp seatunnel-dist/target/apache-seatunnel-2.3.8-bin.tar.gz /The-Path-You-Want-To-Copy - -cd /The-Path-You-Want-To-Copy -tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" -``` - -When built from the source code, all the connector plugins and some necessary dependencies (eg: mysql driver) are included in the binary package. You can directly use the connector plugins without the need to install them separately. - -# Run SeaTunnel - Now you have downloaded the SeaTunnel binary package and the connector plugins. Next, you can choose different engine option to run synchronization tasks. -If you use Flink to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start With Flink](quick-start-flink.md) to run your synchronization task. +If you use Flink to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start of SeaTunnel Flink Engine](quick-start-flink.md) to run your synchronization task. -If you use Spark to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start With Spark](quick-start-spark.md) to run your synchronization task. +If you use Spark to run the synchronization task, there is no need to deploy the SeaTunnel Engine service cluster. You can refer to [Quick Start of SeaTunnel Spark Engine](quick-start-spark.md) to run your synchronization task. -If you use the builtin SeaTunnel Engine (Zeta) to run tasks, you need to deploy the SeaTunnel Engine service first. Refer to [Quick Start With SeaTunnel Engine](quick-start-seatunnel-engine.md). +If you use the builtin SeaTunnel Engine (Zeta) to run tasks, you need to deploy the SeaTunnel Engine service first. Refer to [Deployment of SeaTunnel Engine (Zeta) Service](quick-start-seatunnel-engine.md). diff --git a/docs/en/transform-v2/embedding.md b/docs/en/transform-v2/embedding.md deleted file mode 100644 index 046f72789ac..00000000000 --- a/docs/en/transform-v2/embedding.md +++ /dev/null @@ -1,392 +0,0 @@ -# Embedding - -> Embedding Transform Plugin - -## Description - -The `Embedding` transform plugin leverages embedding models to convert text data into vectorized representations. This -transformation can be applied to various fields. The plugin supports multiple model providers and can be integrated with -different API endpoints. - -## Options - -| Name | Type | Required | Default Value | Description | -|--------------------------------|--------|----------|---------------|-------------------------------------------------------------------------------------------------------------| -| model_provider | enum | yes | - | The model provider for embedding. Options may include `QIANFAN`, `OPENAI`, etc. | -| api_key | string | yes | - | The API key required to authenticate with the embedding service. | -| secret_key | string | yes | - | The secret key required for additional authentication with the embedding service. | -| single_vectorized_input_number | int | no | 1 | The number of inputs vectorized in one request. Default is 1. | -| vectorization_fields | map | yes | - | A mapping between input fields and their corresponding output vector fields. | -| model | string | yes | - | The specific model to use for embedding (e.g: `text-embedding-3-small` for OPENAI). | -| api_path | string | no | - | The API endpoint for the embedding service. Typically provided by the model provider. | -| oauth_path | string | no | - | The API endpoint for the oauth service. | -| custom_config | map | no | | Custom configurations for the model. | -| custom_response_parse | string | no | | Specifies how to parse the response from the model using JsonPath. Example: `$.choices[*].message.content`. | -| custom_request_headers | map | no | | Custom headers for the request to the model. | -| custom_request_body | map | no | | Custom body for the request. Supports placeholders like `${model}`, `${input}`. | - -### model_provider - -The providers for generating embeddings include common options such as `DOUBAO`, `QIANFAN`, and `OPENAI`. Additionally, -you can choose `CUSTOM` to implement requests and retrievals for custom embedding models. - -### api_key - -The API key for authenticating requests to the embedding service. This is typically provided by the model provider when -you register for their service. - -### secret_key - -The secret key used for additional authentication. Some providers may require this for secure API requests. - -### single_vectorized_input_number - -Specifies how many inputs are processed in a single vectorization request. The default is 1. Adjust based on your -processing -capacity and the model provider's API limitations. - -### vectorization_fields - -A mapping between input fields and their respective output vector fields. This allows the plugin to understand which -text fields to vectorize and how to store the resulting vectors. - -```hocon -vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography -} -``` - -### model - -The specific embedding model to use. This depends on the `embedding_model_provider`. For example, if using OPENAI, you -might specify `text-embedding-3-small`. - -### api_path - -The API endpoint to use for making requests to the embedding service. This might vary based on the provider and model -used. Generally, this is provided by the model provider. - -### oauth_path - -The API endpoint for the oauth service. Get certification information. This might vary based on the provider and model -used. Generally, this is provided by the model provider. - -### custom_config - -The `custom_config` option allows you to provide additional custom configurations for the model. This is a map where you -can define various settings that might be required by the specific model you're using. - -### custom_response_parse - -The `custom_response_parse` option allows you to specify how to parse the model's response. You can use JsonPath to -extract the specific data you need from the response. For example, by using `$.data[*].embedding`, you can extract -the `embedding` field values from the following JSON and obtain a `List` of nested `List` results. For more details on -using JsonPath, please refer to -the [JsonPath Getting Started guide](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started). - -```json -{ - "object": "list", - "data": [ - { - "object": "embedding", - "index": 0, - "embedding": [ - -0.006929283495992422, - -0.005336422007530928, - -0.00004547132266452536, - -0.024047505110502243 - ] - } - ], - "model": "text-embedding-3-small", - "usage": { - "prompt_tokens": 5, - "total_tokens": 5 - } -} -``` - -### custom_request_headers - -The `custom_request_headers` option allows you to define custom headers that should be included in the request sent to -the model's API. This is useful if the API requires additional headers beyond the standard ones, such as authorization -tokens, content types, etc. - -### custom_request_body - -The `custom_request_body` option supports placeholders: - -- `${model}`: Placeholder for the model name. -- `${input}`: Placeholder to determine input value and define request body request type based on the type of body - value. Example: `["${input}"]` -> ["input"] (list) - -### common options - -Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details. - -## Example Configuration - -```hocon -env { - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - schema = { - fields { - book_id = "int" - book_name = "string" - book_intro = "string" - author_biography = "string" - } - } - rows = [ - {fields = [1, "To Kill a Mockingbird", - "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", - "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." - ], kind = INSERT} - {fields = [2, "1984", - "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", - "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." - ], kind = INSERT} - {fields = [3, "Pride and Prejudice", - "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", - "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." - ], kind = INSERT} - {fields = [4, "The Great GatsbyThe Great Gatsby", - "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", - "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." - ], kind = INSERT} - {fields = [5, "Moby-Dick", - "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", - "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." - ], kind = INSERT} - ] - result_table_name = "fake" - } -} - -transform { - Embedding { - source_table_name = "fake" - embedding_model_provider = QIANFAN - model = bge_large_en - api_key = xxxxxxxxxx - secret_key = xxxxxxxxxx - api_path = xxxxxxxxxx - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - result_table_name = "embedding_output" - } -} - -sink { - Assert { - source_table_name = "embedding_output" - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } -} -``` - -### Customize the embedding model - -```hocon - -env { - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - schema = { - fields { - book_id = "int" - book_name = "string" - book_intro = "string" - author_biography = "string" - } - } - rows = [ - {fields = [1, "To Kill a Mockingbird", - "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", - "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." - ], kind = INSERT} - {fields = [2, "1984", - "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", - "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." - ], kind = INSERT} - {fields = [3, "Pride and Prejudice", - "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", - "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." - ], kind = INSERT} - {fields = [4, "The Great GatsbyThe Great Gatsby", - "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", - "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." - ], kind = INSERT} - {fields = [5, "Moby-Dick", - "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", - "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." - ], kind = INSERT} - ] - result_table_name = "fake" - } -} - -transform { - Embedding { - source_table_name = "fake" - model_provider = CUSTOM - model = text-embedding-3-small - api_key = xxxxxxxx - api_path = "http://mockserver:1080/v1/doubao/embedding" - single_vectorized_input_number = 2 - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - custom_config={ - custom_response_parse = "$.data[*].embedding" - custom_request_headers = { - "Content-Type"= "application/json" - "Authorization"= "Bearer xxxxxxx - } - custom_request_body ={ - modelx = "${model}" - inputx = ["${input}"] - } - } - result_table_name = "embedding_output_1" - } -} - -sink { - Assert { - source_table_name = "embedding_output_1" - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } -} - -``` diff --git a/docs/en/transform-v2/llm.md b/docs/en/transform-v2/llm.md index 8caaad00a0e..d03b8226f06 100644 --- a/docs/en/transform-v2/llm.md +++ b/docs/en/transform-v2/llm.md @@ -10,23 +10,19 @@ 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 | | -| api_path | string | no | | -| custom_config | map | no | | -| custom_response_parse | string | no | | -| custom_request_headers | map | no | | -| custom_request_body | map | no | | +| 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、DOUBAO、CUSTOM +OPENAI ### output_data_type @@ -40,7 +36,7 @@ The prompt to send to the LLM. This parameter defines how LLM will process and r The data read from source is a table like this: -| name | age | +| name | age | |---------------|-----| | Jia Fan | 20 | | Hailin Wang | 20 | @@ -55,7 +51,7 @@ Determine whether someone is Chinese or American by their name The result will be: -| name | age | llm_output | +| name | age | llm_output | |---------------|-----|------------| | Jia Fan | 20 | Chinese | | Hailin Wang | 20 | Chinese | @@ -65,73 +61,16 @@ The result will be: ### 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. +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. -### api_path - -The API path to use for the 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. - -### custom_config - -The `custom_config` option allows you to provide additional custom configurations for the model. This is a map where you -can define various settings that might be required by the specific model you're using. - -### custom_response_parse - -The `custom_response_parse` option allows you to specify how to parse the model's response. You can use JsonPath to -extract the specific data you need from the response. For example, by using `$.choices[*].message.content`, you can -extract the `content` field values from the following JSON. For more details on using JsonPath, please refer to -the [JsonPath Getting Started guide](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started). - -```json -{ - "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" -} -``` - -### custom_request_headers - -The `custom_request_headers` option allows you to define custom headers that should be included in the request sent to -the model's API. This is useful if the API requires additional headers beyond the standard ones, such as authorization -tokens, content types, etc. - -### custom_request_body +### openai.api_path -The `custom_request_body` option supports placeholders: - -- `${model}`: Placeholder for the model name. -- `${input}`: Placeholder to determine input value and define request body request type based on the type of body - value. Example: `"${input}"` -> "input" -- `${prompt}`:Placeholder for LLM model prompts. +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] @@ -181,82 +120,3 @@ sink { } ``` -### Customize the LLM model - -```hocon -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 = CUSTOM - 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" - custom_config={ - custom_response_parse = "$.choices[*].message.content" - custom_request_headers = { - Content-Type = "application/json" - Authorization = "Bearer xxxxxxxx" - } - custom_request_body ={ - model = "${model}" - messages = [ - { - role = "system" - content = "${prompt}" - }, - { - role = "user" - content = "${input}" - }] - } - } - 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 - } - ] - } - ] - } - } -} -``` - diff --git a/docs/sidebars.js b/docs/sidebars.js index 9f717d28aa2..6061df55084 100644 --- a/docs/sidebars.js +++ b/docs/sidebars.js @@ -84,7 +84,7 @@ const sidebars = { }, { "type": "category", - "label": "Concepts", + "label": "Concept", "items": [ "concept/config", "concept/connector-v2-features", @@ -106,7 +106,7 @@ const sidebars = { "link": { "type": "generated-index", "title": "Source(V2) of SeaTunnel", - "description": "List all source(v2) supported by Apache SeaTunnel for now.", + "description": "List all source(v2) supported Apache SeaTunnel for now.", "slug": "/connector-v2/source", "keywords": ["source"], "image": "/img/favicon.ico" @@ -124,7 +124,7 @@ const sidebars = { "link": { "type": "generated-index", "title": "Sink(V2) of SeaTunnel", - "description": "List all sink(v2) supported by Apache SeaTunnel for now.", + "description": "List all sink(v2) supported Apache SeaTunnel for now.", "slug": "/connector-v2/sink", "keywords": ["sink"], "image": "/img/favicon.ico" @@ -136,24 +136,6 @@ const sidebars = { } ] }, - { - "type": "category", - "label": "Formats", - "link": { - "type": "generated-index", - "title": "Formats", - "description": "List some special formats (not all) supported by Apache SeaTunnel for now.", - "slug": "/connector-v2/formats", - "keywords": ["formats"], - "image": "/img/favicon.ico" - }, - "items": [ - { - "type": "autogenerated", - "dirName": "connector-v2/formats" - } - ] - }, "connector-v2/source-common-options", "connector-v2/sink-common-options", "connector-v2/Error-Quick-Reference-Manual", diff --git a/docs/zh/concept/sql-config.md b/docs/zh/concept/sql-config.md index e080e161feb..7defa0010b2 100644 --- a/docs/zh/concept/sql-config.md +++ b/docs/zh/concept/sql-config.md @@ -120,10 +120,7 @@ CREATE TABLE sink_table WITH ( INSERT INTO sink_table SELECT id, name, age, email FROM source_table; ``` -* `SELECT FROM` 部分为源端映射表的表名,`SELECT` 部分的语法参考:[SQL-transform](../transform-v2/sql.md) `query` 配置项。如果select的字段是关键字([参考](https://github.com/JSQLParser/JSqlParser/blob/master/src/main/jjtree/net/sf/jsqlparser/parser/JSqlParserCC.jjt)),你应该像这样使用\`filedName\` -```sql -INSERT INTO sink_table SELECT id, name, age, email,`output` FROM source_table; -``` +* `SELECT FROM` 部分为源端映射表的表名,`SELECT` 部分的语法参考:[SQL-transform](../transform-v2/sql.md) `query` 配置项 * `INSERT INTO` 部分为目标端映射表的表名 * 注意:该语法**不支持**在 `INSERT` 中指定字段,如:`INSERT INTO sink_table (id, name, age, email) SELECT id, name, age, email FROM source_table;` diff --git a/docs/zh/connector-v2/sink/Clickhouse.md b/docs/zh/connector-v2/sink/Clickhouse.md index 61a359f5c0b..30002c607cf 100644 --- a/docs/zh/connector-v2/sink/Clickhouse.md +++ b/docs/zh/connector-v2/sink/Clickhouse.md @@ -23,9 +23,9 @@ 为了使用 Clickhouse 连接器,需要以下依赖项。它们可以通过 install-plugin.sh 或从 Maven 中央存储库下载。 -| 数据源 | 支持的版本 | 依赖 | -|------------|-----------|------------------------------------------------------------------------------------| -| Clickhouse | universal | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-clickhouse) | +| 数据源 | 支持的版本 | 依赖 | +|------------|-----------|------------------------------------------------------------------------------------------------------------| +| Clickhouse | universal | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | ## 数据类型映射 diff --git a/docs/zh/connector-v2/sink/Hbase.md b/docs/zh/connector-v2/sink/Hbase.md index f028a8c93ee..edc9e48510e 100644 --- a/docs/zh/connector-v2/sink/Hbase.md +++ b/docs/zh/connector-v2/sink/Hbase.md @@ -119,78 +119,6 @@ Hbase { ``` -### 写入多表 - -```hocon -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - tables_configs = [ - { - schema = { - table = "hbase_sink_1" - fields { - name = STRING - c_string = STRING - c_double = DOUBLE - c_bigint = BIGINT - c_float = FLOAT - c_int = INT - c_smallint = SMALLINT - c_boolean = BOOLEAN - time = BIGINT - } - } - rows = [ - { - kind = INSERT - fields = ["label_1", "sink_1", 4.3, 200, 2.5, 2, 5, true, 1627529632356] - } - ] - }, - { - schema = { - table = "hbase_sink_2" - fields { - name = STRING - c_string = STRING - c_double = DOUBLE - c_bigint = BIGINT - c_float = FLOAT - c_int = INT - c_smallint = SMALLINT - c_boolean = BOOLEAN - time = BIGINT - } - } - rows = [ - { - kind = INSERT - fields = ["label_2", "sink_2", 4.3, 200, 2.5, 2, 5, true, 1627529632357] - } - ] - } - ] - } -} - -sink { - Hbase { - zookeeper_quorum = "hadoop001:2181,hadoop002:2181,hadoop003:2181" - table = "${table_name}" - rowkey_column = ["name"] - family_name { - all_columns = info - } - } -} -``` - ## 写入指定列族 ```hocon diff --git a/docs/zh/connector-v2/sink/Http.md b/docs/zh/connector-v2/sink/Http.md index 12ce90614ff..38d02d9be62 100644 --- a/docs/zh/connector-v2/sink/Http.md +++ b/docs/zh/connector-v2/sink/Http.md @@ -25,9 +25,9 @@ 想使用 Http 连接器,需要安装以下必要的依赖。可以通过运行 install-plugin.sh 脚本或者从 Maven 中央仓库下载这些依赖 -| 数据源 | 支持版本 | 依赖 | -|------|------|------------------------------------------------------------------------------| -| Http | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-http) | +| 数据源 | 支持版本 | 依赖 | +|------|------|------------------------------------------------------------------------------------------------------| +| Http | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-http) | ## 接收器选项 diff --git a/docs/zh/connector-v2/sink/Jdbc.md b/docs/zh/connector-v2/sink/Jdbc.md index e1ab422952e..b05ecbc501c 100644 --- a/docs/zh/connector-v2/sink/Jdbc.md +++ b/docs/zh/connector-v2/sink/Jdbc.md @@ -225,7 +225,7 @@ Sink插件常用参数,请参考 [Sink常用选项](../sink-common-options.md) | SQL Server | com.microsoft.sqlserver.jdbc.SQLServerDriver | jdbc:sqlserver://localhost:1433 | com.microsoft.sqlserver.jdbc.SQLServerXADataSource | https://mvnrepository.com/artifact/com.microsoft.sqlserver/mssql-jdbc | | Oracle | oracle.jdbc.OracleDriver | jdbc:oracle:thin:@localhost:1521/xepdb1 | oracle.jdbc.xa.OracleXADataSource | https://mvnrepository.com/artifact/com.oracle.database.jdbc/ojdbc8 | | sqlite | org.sqlite.JDBC | jdbc:sqlite:test.db | / | https://mvnrepository.com/artifact/org.xerial/sqlite-jdbc | -| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar | +| GBase8a | com.gbase.jdbc.Driver | jdbc:gbase://e2e_gbase8aDb:5258/test | / | https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar | | StarRocks | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | / | https://mvnrepository.com/artifact/mysql/mysql-connector-java | | db2 | com.ibm.db2.jcc.DB2Driver | jdbc:db2://localhost:50000/testdb | com.ibm.db2.jcc.DB2XADataSource | https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc/db2jcc4 | | saphana | com.sap.db.jdbc.Driver | jdbc:sap://localhost:39015 | / | https://mvnrepository.com/artifact/com.sap.cloud.db.jdbc/ngdbc | diff --git a/docs/zh/connector-v2/sink/Kafka.md b/docs/zh/connector-v2/sink/Kafka.md index a7273c3b56c..45117a962d6 100644 --- a/docs/zh/connector-v2/sink/Kafka.md +++ b/docs/zh/connector-v2/sink/Kafka.md @@ -24,27 +24,25 @@ 为了使用 Kafka 连接器,需要以下依赖项 可以通过 install-plugin.sh 或从 Maven 中央存储库下载 -| 数据源 | 支持版本 | Maven | -|-------|------|-------------------------------------------------------------------------------| -| Kafka | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | +| 数据源 | 支持版本 | Maven | +|-------|------|-------------------------------------------------------------------------------------------------------| +| Kafka | 通用 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | ## 接收器选项 -| 名称 | 类型 | 是否需要 | 默认值 | 描述 | -|----------------------|--------|------|------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| topic | String | 是 | - | 当表用作接收器时,topic 名称是要写入数据的 topic | -| bootstrap.servers | String | 是 | - | Kafka brokers 使用逗号分隔 | -| kafka.config | Map | 否 | - | 除了上述 Kafka Producer 客户端必须指定的参数外,用户还可以为 Producer 客户端指定多个非强制参数,涵盖 [Kafka官方文档中指定的所有生产者参数](https://kafka.apache.org/documentation.html#producerconfigs) | -| semantics | String | 否 | NON | 可以选择的语义是 EXACTLY_ONCE/AT_LEAST_ONCE/NON,默认 NON。 | -| partition_key_fields | Array | 否 | - | 配置字段用作 kafka 消息的key | -| partition | Int | 否 | - | 可以指定分区,所有消息都会发送到此分区 | -| assign_partitions | Array | 否 | - | 可以根据消息的内容决定发送哪个分区,该参数的作用是分发信息 | -| transaction_prefix | String | 否 | - | 如果语义指定为EXACTLY_ONCE,生产者将把所有消息写入一个 Kafka 事务中,kafka 通过不同的 transactionId 来区分不同的事务。该参数是kafka transactionId的前缀,确保不同的作业使用不同的前缀 | -| format | String | 否 | json | 数据格式。默认格式是json。可选文本格式,canal-json、debezium-json 、 avro 和 protobuf。如果使用 json 或文本格式。默认字段分隔符是`,`。如果自定义分隔符,请添加`field_delimiter`选项。如果使用canal格式,请参考[canal-json](../formats/canal-json.md)。如果使用debezium格式,请参阅 [debezium-json](../formats/debezium-json.md) 了解详细信息 | -| field_delimiter | String | 否 | , | 自定义数据格式的字段分隔符 | -| common-options | | 否 | - | Sink插件常用参数,请参考 [Sink常用选项 ](../sink-common-options.md) 了解详情 | -|protobuf_message_name|String|否|-| format配置为protobuf时生效,取Message名称 | -|protobuf_schema|String|否|-| format配置为protobuf时生效取Schema名称 | +| 名称 | 类型 | 是否需要 | 默认值 | 描述 | +|----------------------|--------|------|------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | 是 | - | 当表用作接收器时,topic 名称是要写入数据的 topic | +| bootstrap.servers | String | 是 | - | Kafka brokers 使用逗号分隔 | +| kafka.config | Map | 否 | - | 除了上述 Kafka Producer 客户端必须指定的参数外,用户还可以为 Producer 客户端指定多个非强制参数,涵盖 [Kafka官方文档中指定的所有生产者参数](https://kafka.apache.org/documentation.html#producerconfigs) | +| semantics | String | 否 | NON | 可以选择的语义是 EXACTLY_ONCE/AT_LEAST_ONCE/NON,默认 NON。 | +| partition_key_fields | Array | 否 | - | 配置字段用作 kafka 消息的key | +| partition | Int | 否 | - | 可以指定分区,所有消息都会发送到此分区 | +| assign_partitions | Array | 否 | - | 可以根据消息的内容决定发送哪个分区,该参数的作用是分发信息 | +| transaction_prefix | String | 否 | - | 如果语义指定为EXACTLY_ONCE,生产者将把所有消息写入一个 Kafka 事务中,kafka 通过不同的 transactionId 来区分不同的事务。该参数是kafka transactionId的前缀,确保不同的作业使用不同的前缀 | +| format | String | 否 | json | 数据格式。默认格式是json。可选文本格式,canal-json、debezium-json 和 avro。如果使用 json 或文本格式。默认字段分隔符是`,`。如果自定义分隔符,请添加`field_delimiter`选项。如果使用canal格式,请参考[canal-json](../formats/canal-json.md)。如果使用debezium格式,请参阅 [debezium-json](../formats/debezium-json.md) 了解详细信息 | +| field_delimiter | String | 否 | , | 自定义数据格式的字段分隔符 | +| common-options | | 否 | - | Sink插件常用参数,请参考 [Sink常用选项 ](../sink-common-options.md) 了解详情 | ## 参数解释 @@ -196,56 +194,3 @@ sink { } ``` -### Protobuf配置 - -`format` 设置为 `protobuf`,配置`protobuf`数据结构,`protobuf_message_name`和`protobuf_schema`参数 - -使用样例: - -```hocon -sink { - kafka { - topic = "test_protobuf_topic_fake_source" - bootstrap.servers = "kafkaCluster:9092" - format = protobuf - kafka.request.timeout.ms = 60000 - kafka.config = { - acks = "all" - request.timeout.ms = 60000 - buffer.memory = 33554432 - } - protobuf_message_name = Person - protobuf_schema = """ - syntax = "proto3"; - - package org.apache.seatunnel.format.protobuf; - - option java_outer_classname = "ProtobufE2E"; - - message Person { - int32 c_int32 = 1; - int64 c_int64 = 2; - float c_float = 3; - double c_double = 4; - bool c_bool = 5; - string c_string = 6; - bytes c_bytes = 7; - - message Address { - string street = 1; - string city = 2; - string state = 3; - string zip = 4; - } - - Address address = 8; - - map attributes = 9; - - repeated string phone_numbers = 10; - } - """ - } -} -``` - diff --git a/docs/zh/connector-v2/sink/Paimon.md b/docs/zh/connector-v2/sink/Paimon.md index 41fa395eb7e..11e299eb0d8 100644 --- a/docs/zh/connector-v2/sink/Paimon.md +++ b/docs/zh/connector-v2/sink/Paimon.md @@ -30,22 +30,21 @@ libfb303-xxx.jar ## 连接器选项 -| 名称 | 类型 | 是否必须 | 默认值 | 描述 | -|-----------------------------|-------|----------|------------------------------|------------------------------------------------------------------------------------------------------| -| warehouse | 字符串 | 是 | - | Paimon warehouse路径 | -| catalog_type | 字符串 | 否 | filesystem | Paimon的catalog类型,目前支持filesystem和hive | -| catalog_uri | 字符串 | 否 | - | Paimon catalog的uri,仅当catalog_type为hive时需要配置 | -| database | 字符串 | 是 | - | 数据库名称 | -| table | 字符串 | 是 | - | 表名 | -| hdfs_site_path | 字符串 | 否 | - | hdfs-site.xml文件路径 | -| schema_save_mode | 枚举 | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | Schema保存模式 | -| data_save_mode | 枚举 | 否 | APPEND_DATA | 数据保存模式 | -| paimon.table.primary-keys | 字符串 | 否 | - | 主键字段列表,联合主键使用逗号分隔(注意:分区字段需要包含在主键字段中) | -| paimon.table.partition-keys | 字符串 | 否 | - | 分区字段列表,多字段使用逗号分隔 | -| paimon.table.write-props | Map | 否 | - | Paimon表初始化指定的属性, [参考](https://paimon.apache.org/docs/master/maintenance/configurations/#coreoptions) | -| paimon.hadoop.conf | Map | 否 | - | Hadoop配置文件属性信息 | -| paimon.hadoop.conf-path | 字符串 | 否 | - | Hadoop配置文件目录,用于加载'core-site.xml', 'hdfs-site.xml', 'hive-site.xml'文件配置 | - +| 名称 | 类型 | 是否必须 | 默认值 | 描述 | +|-----------------------------|-----|------|------------------------------|---------------------------------------------------------------------------------------------------|---| +| warehouse | 字符串 | 是 | - | Paimon warehouse路径 | +| catalog_type | 字符串 | 否 | filesystem | Paimon的catalog类型,目前支持filesystem和hive | +| catalog_uri | 字符串 | 否 | - | Paimon catalog的uri,仅当catalog_type为hive时需要配置 | | +| database | 字符串 | 是 | - | 数据库名称 | +| table | 字符串 | 是 | - | 表名 | +| hdfs_site_path | 字符串 | 否 | - | hdfs-site.xml文件路径 | +| schema_save_mode | 枚举 | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | Schema保存模式 | +| data_save_mode | 枚举 | 否 | APPEND_DATA | 数据保存模式 | +| paimon.table.primary-keys | 字符串 | 否 | - | 主键字段列表,联合主键使用逗号分隔(注意:分区字段需要包含在主键字段中) | +| paimon.table.partition-keys | 字符串 | 否 | - | 分区字段列表,多字段使用逗号分隔 | +| paimon.table.write-props | Map | 否 | - | Paimon表初始化指定的属性, [参考](https://paimon.apache.org/docs/0.8/maintenance/configurations/#coreoptions) | +| paimon.hadoop.conf | Map | 否 | - | Hadoop配置文件属性信息 | +| paimon.hadoop.conf-path | 字符串 | 否 | - | Hadoop配置文件目录,用于加载'core-site.xml', 'hdfs-site.xml', 'hive-site.xml'文件配置 | ## 示例 diff --git a/docs/zh/connector-v2/sink/Qdrant.md b/docs/zh/connector-v2/sink/Qdrant.md deleted file mode 100644 index 7394eb85414..00000000000 --- a/docs/zh/connector-v2/sink/Qdrant.md +++ /dev/null @@ -1,68 +0,0 @@ -# Qdrant - -> Qdrant 数据连接器 - -[Qdrant](https://qdrant.tech/) 是一个高性能的向量搜索引擎和向量数据库。 - -该连接器可用于将数据写入 Qdrant 集合。 - -## 数据类型映射 - -| SeaTunnel 数据类型 | Qdrant 数据类型 | -|---------------------|---------------| -| TINYINT | INTEGER | -| SMALLINT | INTEGER | -| INT | INTEGER | -| BIGINT | INTEGER | -| FLOAT | DOUBLE | -| DOUBLE | DOUBLE | -| BOOLEAN | BOOL | -| STRING | STRING | -| ARRAY | LIST | -| FLOAT_VECTOR | DENSE_VECTOR | -| BINARY_VECTOR | DENSE_VECTOR | -| FLOAT16_VECTOR | DENSE_VECTOR | -| BFLOAT16_VECTOR | DENSE_VECTOR | -| SPARSE_FLOAT_VECTOR | SPARSE_VECTOR | - -主键列的值将用作 Qdrant 中的点 ID。如果没有主键,则将使用随机 UUID。 - -## 选项 - -| 名称 | 类型 | 必填 | 默认值 | -|-----------------|--------|----|-----------| -| collection_name | string | 是 | - | -| batch_size | int | 否 | 64 | -| host | string | 否 | localhost | -| port | int | 否 | 6334 | -| api_key | string | 否 | - | -| use_tls | bool | 否 | false | -| common-options | | 否 | - | - -### collection_name [string] - -要从中读取数据的 Qdrant 集合的名称。 - -### batch_size [int] - -每个 upsert 请求到 Qdrant 的批量大小。 - -### host [string] - -Qdrant 实例的主机名。默认为 "localhost"。 - -### port [int] - -Qdrant 实例的 gRPC 端口。 - -### api_key [string] - -用于身份验证的 API 密钥(如果设置)。 - -### use_tls [bool] - -是否使用 TLS(SSL)连接。如果使用 Qdrant 云(https),则需要。 - -### 通用选项 - -接收插件的通用参数,请参考[源通用选项](../sink-common-options.md)了解详情。 diff --git a/docs/zh/connector-v2/sink/Typesense.md b/docs/zh/connector-v2/sink/Typesense.md deleted file mode 100644 index 99017f32cb5..00000000000 --- a/docs/zh/connector-v2/sink/Typesense.md +++ /dev/null @@ -1,95 +0,0 @@ -# Typesense - -## 描述 - -输出数据到 `Typesense` - -## 主要特性 - -- [ ] [精确一次](../../concept/connector-v2-features.md) -- [x] [cdc](../../concept/connector-v2-features.md) - -## 选项 - -| 名称 | 类型 | 是否必须 | 默认值 | -|------------------|--------|------|------------------------------| -| hosts | array | 是 | - | -| collection | string | 是 | - | -| schema_save_mode | string | 是 | CREATE_SCHEMA_WHEN_NOT_EXIST | -| data_save_mode | string | 是 | APPEND_DATA | -| primary_keys | array | 否 | | -| key_delimiter | string | 否 | `_` | -| api_key | string | 否 | | -| max_retry_count | int | 否 | 3 | -| max_batch_size | int | 否 | 10 | -| common-options | | 否 | - | - -### hosts [array] - -Typesense的访问地址,格式为 `host:port`,例如:["typesense-01:8108"] - -### collection [string] - -要写入的集合名,例如:“seatunnel” - -### primary_keys [array] - -主键字段用于生成文档 `id`。 - -### key_delimiter [string] - -设定复合键的分隔符(默认为 `_`)。 - -### api_key [config] - -typesense 安全认证的 api_key。 - -### max_retry_count [int] - -批次批量请求最大尝试大小 - -### max_batch_size [int] - -批次批量文档最大大小 - -### common options - -Sink插件常用参数,请参考 [Sink常用选项](../sink-common-options.md) 了解详情 - -### schema_save_mode - -在启动同步任务之前,针对目标侧已有的表结构选择不同的处理方案
-选项介绍:
-`RECREATE_SCHEMA` :当表不存在时会创建,当表已存在时会删除并重建
-`CREATE_SCHEMA_WHEN_NOT_EXIST` :当表不存在时会创建,当表已存在时则跳过创建
-`ERROR_WHEN_SCHEMA_NOT_EXIST` :当表不存在时将抛出错误
- -### data_save_mode - -在启动同步任务之前,针对目标侧已存在的数据选择不同的处理方案
-选项介绍:
-`DROP_DATA`: 保留数据库结构,删除数据
-`APPEND_DATA`:保留数据库结构,保留数据
-`ERROR_WHEN_DATA_EXISTS`:当有数据时抛出错误
- -## 示例 - -简单示例 - -```bash -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["localhost:8108"] - collection = "typesense_to_typesense_sink_with_query" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","id"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "APPEND_DATA" - } -} -``` - diff --git a/docs/zh/connector-v2/source/Elasticsearch.md b/docs/zh/connector-v2/source/Elasticsearch.md deleted file mode 100644 index 7a27f2b9371..00000000000 --- a/docs/zh/connector-v2/source/Elasticsearch.md +++ /dev/null @@ -1,247 +0,0 @@ -# Elasticsearch - -> Elasticsearch source 连接器 - -## 简介 - -支持读取 Elasticsearch2.x 版本和 8.x 版本之间的数据 - -## Key features - -- [x] [批处理](../../concept/connector-v2-features.md) -- [ ] [流处理](../../concept/connector-v2-features.md) -- [ ] [精准一次](../../concept/connector-v2-features.md) -- [x] [column projection](../../concept/connector-v2-features.md) -- [ ] [并行度](../../concept/connector-v2-features.md) -- [ ] [支持用户自定义的分片](../../concept/connector-v2-features.md) - -## 配置参数选项 - -| 参数名称 | 类型 | 是否必须 | 默认值或者描述 | -| ----------------------- | ------- | -------- | ------------------------------------------------------- | -| hosts | 数组 | | - | -| username | string | no | - | -| password | string | no | - | -| index | string | No | 单索引同步配置,如果index_list没有配置,则必须配置index | -| index_list | array | no | 用来定义多索引同步任务 | -| source | array | no | - | -| query | json | no | {"match_all": {}} | -| scroll_time | string | no | 1m | -| scroll_size | int | no | 100 | -| tls_verify_certificate | boolean | no | true | -| tls_verify_hostnames | boolean | no | true | -| array_column | map | no | | -| tls_keystore_path | string | no | - | -| tls_keystore_password | string | no | - | -| tls_truststore_path | string | no | - | -| tls_truststore_password | string | no | - | -| common-options | | no | - | - -### hosts [array] - -Elasticsearch 集群的 HTTP 地址,格式为 `host:port`,允许指定多个主机。例如:`["host1:9200", "host2:9200"]`。 - -### username [string] - -用户名 - -### password [string] - -密码 - -### index [string] - -Elasticsearch 索引名称,支持 * 模糊匹配。比如存在索引index1,index2,可以指定index*同时读取两个索引的数据。 - -### source [array] - -索引的字段 - -你可以通过指定字段 `_id` 来获取文档 ID。如果将 `_id` 写入到其他索引,由于 Elasticsearch 的限制,你需要为 `_id` 指定一个别名。 - -如果你没有配置 `source`,它将自动从索引的映射中获取。 - -### array_column [array] - -由于 Elasticsearch 中没有数组索引,因此需要指定数组类型。 - -假设tags和phones是数组类型: - -```hocon -array_column = {tags = "array",phones = "array"} -``` - -### query [json] - -ElasticsSearch的原生查询语句,用于控制读取哪些数据写入到其他数据源。 - -### scroll_time [String] - -`Seatunnel`底层会使用滚动查询来查询数据,所以需要使用这个参数控制搜索上下文的时间长度。 - -### scroll_size [int] - -滚动查询的最大文档数量。 - -### index_list [array] - -`index_list` 用于定义多索引同步任务。它是一个数组,包含单表同步所需的参数,如 `query`、`source/schema`、`scroll_size` 和 `scroll_time`。建议不要将 `index_list` 和 `query` 配置在同一层级。有关更多详细信息,请参考后面的多表同步示例。 - -### tls_verify_certificate [boolean] - -启用 HTTPS 端点的证书验证 - -### tls_verify_hostname [boolean] - -启用 HTTPS 端点的主机名验证 - -### tls_keystore_path [string] - -PEM 或 JKS 密钥库的路径。该文件必须对运行 SeaTunnel 的操作系统用户可读。 - -### tls_keystore_password [string] - -指定密钥库的密钥密码。 - -### tls_truststore_path [string] - -PEM 或 JKS 信任库的路径。该文件必须对运行 SeaTunnel 的操作系统用户可读。 - -### tls_truststore_password [string] - -指定信任库的密钥密码。 - -### common options - -Source 插件常用参数,具体请参考 [Source 常用选项](../source-common-options.md) - -## 使用案例 - -案例一 - -> 案例一会从满足seatunnel-*匹配的索引中按照query读取数据,查询只会返回文档`id`,`name`,`age`,`tags`,`phones` 三个字段。在这个例子中,使用了source字段配置应该读取哪些字段,使用`array_column`指定了`tags`,`phones`应该被当做数组处理。 - -```hocon -Elasticsearch { - hosts = ["localhost:9200"] - index = "seatunnel-*" - array_column = {tags = "array",phones = "array"} - source = ["_id","name","age","tags","phones"] - query = {"range":{"firstPacket":{"gte":1669225429990,"lte":1669225429990}}} -} -``` - -案例二:多索引同步 - -> 此示例演示了如何从 `read_index1` 和 `read_index2` 中读取不同的数据数据,并将其分别写入 `read_index1_copy`,`read_index12_copy` 索引。 -> 在 `read_index1` 中,我使用 `source` 来指定要读取的字段,并使用`array_column`指明哪些字段是数组字段。 - -```hocon -source { - Elasticsearch { - hosts = ["https://elasticsearch:9200"] - username = "elastic" - password = "elasticsearch" - tls_verify_certificate = false - tls_verify_hostname = false - index_list = [ - { - index = "read_index1" - query = {"range": {"c_int": {"gte": 10, "lte": 20}}} - source = [ - c_map, - c_array, - c_string, - c_boolean, - c_tinyint, - c_smallint, - c_bigint, - c_float, - c_double, - c_decimal, - c_bytes, - c_int, - c_date, - c_timestamp - ] - array_column = { - c_array = "array" - } - } - { - index = "read_index2" - query = {"match_all": {}} - source = [ - c_int2, - c_date2, - c_null - ] - - } - - ] - - } -} - -transform { -} - -sink { - Elasticsearch { - hosts = ["https://elasticsearch:9200"] - username = "elastic" - password = "elasticsearch" - tls_verify_certificate = false - tls_verify_hostname = false - - index = "multi_source_write_test_index" - index_type = "st" - "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" - "data_save_mode"="APPEND_DATA" - } -} -``` - -案例三:SSL(禁用证书验证) - -```hocon -source { - Elasticsearch { - hosts = ["https://localhost:9200"] - username = "elastic" - password = "elasticsearch" - - tls_verify_certificate = false - } -} -``` - -案例四:SSL(禁用主机名验证) - -```hocon -source { - Elasticsearch { - hosts = ["https://localhost:9200"] - username = "elastic" - password = "elasticsearch" - - tls_verify_hostname = false - } -} -``` - -案例五:SSL(启用证书验证) - -```hocon -source { - Elasticsearch { - hosts = ["https://localhost:9200"] - username = "elastic" - password = "elasticsearch" - - tls_keystore_path = "${your elasticsearch home}/config/certs/http.p12" - tls_keystore_password = "${your password}" - } -} -``` \ No newline at end of file diff --git a/docs/zh/connector-v2/source/Kafka.md b/docs/zh/connector-v2/source/Kafka.md deleted file mode 100644 index 8f65e92e924..00000000000 --- a/docs/zh/connector-v2/source/Kafka.md +++ /dev/null @@ -1,288 +0,0 @@ -# Kafka - -> Kafka 源连接器 - -## 支持以下引擎 - -> Spark
-> Flink
-> Seatunnel Zeta
- -## 主要功能 - -- [x] [批处理](../../concept/connector-v2-features.md) -- [x] [流处理](../../concept/connector-v2-features.md) -- [x] [精确一次](../../concept/connector-v2-features.md) -- [ ] [列投影](../../concept/connector-v2-features.md) -- [x] [并行度](../../concept/connector-v2-features.md) -- [ ] [支持用户定义拆分](../../concept/connector-v2-features.md) - -## 描述 - -用于 Apache Kafka 的源连接器。 - -## 支持的数据源信息 - -使用 Kafka 连接器需要以下依赖项。 -可以通过 install-plugin.sh 下载或从 Maven 中央仓库获取。 - -| 数据源 | 支持的版本 | Maven 下载链接 | -|-------|-------|-------------------------------------------------------------------------------| -| Kafka | 通用版本 | [下载](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-kafka) | - -## 源选项 - -| 名称 | 类型 | 是否必填 | 默认值 | 描述 | -|-------------------------------------|-------------------------------------|------|--------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| topic | String | 是 | - | 使用表作为数据源时要读取数据的主题名称。它也支持通过逗号分隔的多个主题列表,例如 'topic-1,topic-2'。 | -| table_list | Map | 否 | - | 主题列表配置,你可以同时配置一个 `table_list` 和一个 `topic`。 | -| bootstrap.servers | String | 是 | - | 逗号分隔的 Kafka brokers 列表。 | -| pattern | Boolean | 否 | false | 如果 `pattern` 设置为 `true`,则会使用指定的正则表达式匹配并订阅主题。 | -| consumer.group | String | 否 | SeaTunnel-Consumer-Group | `Kafka 消费者组 ID`,用于区分不同的消费者组。 | -| commit_on_checkpoint | Boolean | 否 | true | 如果为 true,消费者的偏移量将会定期在后台提交。 | -| kafka.config | Map | 否 | - | 除了上述必要参数外,用户还可以指定多个非强制的消费者客户端参数,覆盖 [Kafka 官方文档](https://kafka.apache.org/documentation.html#consumerconfigs) 中指定的所有消费者参数。 | -| schema | Config | 否 | - | 数据结构,包括字段名称和字段类型。 | -| format | String | 否 | json | 数据格式。默认格式为 json。可选格式包括 text, canal_json, debezium_json, ogg_json, maxwell_json, avro 和 protobuf。默认字段分隔符为 ", "。如果自定义分隔符,添加 "field_delimiter" 选项。如果使用 canal 格式,请参考 [canal-json](../formats/canal-json.md) 了解详细信息。如果使用 debezium 格式,请参考 [debezium-json](../formats/debezium-json.md)。一些Format的详细信息请参考 [formats](../formats) | -| format_error_handle_way | String | 否 | fail | 数据格式错误的处理方式。默认值为 fail,可选值为 fail 和 skip。当选择 fail 时,数据格式错误将阻塞并抛出异常。当选择 skip 时,数据格式错误将跳过此行数据。 | -| field_delimiter | String | 否 | , | 自定义数据格式的字段分隔符。 | -| start_mode | StartMode[earliest],[group_offsets] | 否 | group_offsets | 消费者的初始消费模式。 | -| start_mode.offsets | Config | 否 | - | 用于 specific_offsets 消费模式的偏移量。 | -| start_mode.timestamp | Long | 否 | - | 用于 "timestamp" 消费模式的时间。 | -| partition-discovery.interval-millis | Long | 否 | -1 | 动态发现主题和分区的间隔时间。 | -| common-options | | 否 | - | 源插件的常见参数,详情请参考 [Source Common Options](../source-common-options.md)。 | -| protobuf_message_name | String | 否 | - | 当格式设置为 protobuf 时有效,指定消息名称。 | -| protobuf_schema | String | 否 | - | 当格式设置为 protobuf 时有效,指定 Schema 定义。 | - -## 任务示例 - -### 简单示例 - -> 此示例读取 Kafka 的 topic_1、topic_2 和 topic_3 的数据并将其打印到客户端。如果尚未安装和部署 SeaTunnel,请按照 [安装指南](../../start-v2/locally/deployment.md) 进行安装和部署。然后,按照 [快速开始](../../start-v2/locally/quick-start-seatunnel-engine.md) 运行此任务。 - -```hocon -# 定义运行环境 -env { - parallelism = 2 - job.mode = "BATCH" -} -source { - Kafka { - schema = { - fields { - name = "string" - age = "int" - } - } - format = text - field_delimiter = "#" - topic = "topic_1,topic_2,topic_3" - bootstrap.servers = "localhost:9092" - kafka.config = { - client.id = client_1 - max.poll.records = 500 - auto.offset.reset = "earliest" - enable.auto.commit = "false" - } - } -} -sink { - Console {} -} -``` - -### 正则表达式主题 - -```hocon -source { - Kafka { - topic = ".*seatunnel*." - pattern = "true" - bootstrap.servers = "localhost:9092" - consumer.group = "seatunnel_group" - } -} -``` - -### AWS MSK SASL/SCRAM - -将以下 `${username}` 和 `${password}` 替换为 AWS MSK 中的配置值。 - -```hocon -source { - Kafka { - topic = "seatunnel" - bootstrap.servers = "xx.amazonaws.com.cn:9096,xxx.amazonaws.com.cn:9096,xxxx.amazonaws.com.cn:9096" - consumer.group = "seatunnel_group" - kafka.config = { - security.protocol=SASL_SSL - sasl.mechanism=SCRAM-SHA-512 - sasl.jaas.config="org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";" - } - } -} -``` - -### AWS MSK IAM - -从 [此处](https://github.com/aws/aws-msk-iam-auth/releases) 下载 `aws-msk-iam-auth-1.1.5.jar` 并将其放在 `$SEATUNNEL_HOME/plugin/kafka/lib` 目录下。 - -确保 IAM 策略中包含 `"kafka-cluster:Connect"` 权限,如下所示: - -```hocon -"Effect": "Allow", -"Action": [ - "kafka-cluster:Connect", - "kafka-cluster:AlterCluster", - "kafka-cluster:DescribeCluster" -], -``` - -源配置示例: - -```hocon -source { - Kafka { - topic = "seatunnel" - bootstrap.servers = "xx.amazonaws.com.cn:9098,xxx.amazonaws.com.cn:9098,xxxx.amazonaws.com.cn:9098" - consumer.group = "seatunnel_group" - kafka.config = { - security.protocol=SASL_SSL - sasl.mechanism=AWS_MSK_IAM - sasl.jaas.config="software.amazon.msk.auth.iam.IAMLoginModule required;" - sasl.client.callback.handler.class="software.amazon.msk.auth.iam.IAMClientCallbackHandler" - } - } -} -``` - -### Kerberos 认证示例 - -源配置示例: - -```hocon -source { - Kafka { - topic = "seatunnel" - bootstrap.servers = "127.0.0.1:9092" - consumer.group = "seatunnel_group" - kafka.config = { - security.protocol=SASL_PLAINTEXT - sasl.kerberos.service.name=kafka - sasl.mechanism=GSSAPI - java.security.krb5.conf="/etc/krb5.conf" - sasl.jaas.config="com.sun.security.auth.module.Krb5LoginModule required \n useKeyTab=true \n storeKey=true \n keyTab=\"/path/to/xxx.keytab\" \n principal=\"user@xxx.com\";" - } - } -} -``` - -### 多 Kafka 源示例 - -> 根据不同的 Kafka 主题和格式解析数据,并基于 ID 执行 upsert 操作。 - -```hocon -env { - execution.parallelism = 1 - job.mode = "BATCH" -} - -source { - Kafka { - - - bootstrap.servers = "kafka_e2e:9092" - table_list = [ - { - topic = "^test-ogg-sou.*" - pattern = "true" - consumer.group = "ogg_multi_group" - start_mode = earliest - schema = { - fields { - id = "int" - name = "string" - description = "string" - weight = "string" - } - }, - format = ogg_json - }, - { - topic = "test-cdc_mds" - start_mode = earliest - schema = { - fields { - id = "int" - name = "string" - description = "string" - weight = "string" - } - }, - format = canal_json - } - ] - } -} - -sink { - Jdbc { - driver = org.postgresql.Driver - url = "jdbc:postgresql://postgresql:5432/test?loggerLevel=OFF" - user = test - password = test - generate_sink_sql = true - database = test - table = public.sink - primary_keys = ["id"] - } -} -``` - -### Protobuf配置 - -`format` 设置为 `protobuf`,配置`protobuf`数据结构,`protobuf_message_name`和`protobuf_schema`参数 - -使用样例: - -```hocon -source { - Kafka { - topic = "test_protobuf_topic_fake_source" - format = protobuf - protobuf_message_name = Person - protobuf_schema = """ - syntax = "proto3"; - - package org.apache.seatunnel.format.protobuf; - - option java_outer_classname = "ProtobufE2E"; - - message Person { - int32 c_int32 = 1; - int64 c_int64 = 2; - float c_float = 3; - double c_double = 4; - bool c_bool = 5; - string c_string = 6; - bytes c_bytes = 7; - - message Address { - string street = 1; - string city = 2; - string state = 3; - string zip = 4; - } - - Address address = 8; - - map attributes = 9; - - repeated string phone_numbers = 10; - } - """ - bootstrap.servers = "kafkaCluster:9092" - start_mode = "earliest" - result_table_name = "kafka_table" - } -} -``` \ No newline at end of file diff --git a/docs/zh/connector-v2/source/Opengauss-CDC.md b/docs/zh/connector-v2/source/Opengauss-CDC.md deleted file mode 100644 index 83da40b363e..00000000000 --- a/docs/zh/connector-v2/source/Opengauss-CDC.md +++ /dev/null @@ -1,169 +0,0 @@ -# Opengauss CDC - -> Opengauss CDC源连接器 - -## 支持这些引擎 - -> SeaTunnel Zeta
-> Flink
- -## 主要功能 - -- [ ] [批处理](../../concept/connector-v2-features.md) -- [x] [流处理](../../concept/connector-v2-features.md) -- [x] [精确一次](../../concept/connector-v2-features.md) -- [ ] [列投影](../../concept/connector-v2-features.md) -- [x] [并行度](../../concept/connector-v2-features.md) -- [x] [支持用户定义的拆分](../../concept/connector-v2-features.md) - -## 描述 - -Opengauss CDC连接器允许从Opengauss数据库读取快照数据和增量数据。这个文档描述如何设置Opengauss CDC连接器以在Opengauss database中运行SQL查询。 - -## 使用步骤 - -> 这里是启用Opengauss CDC的步骤: - -1. 确保wal_level被设置为logical, 你可以直接使用SQL命令来修改这个配置: - -```sql -ALTER SYSTEM SET wal_level TO 'logical'; -SELECT pg_reload_conf(); -``` - -2. 改变指定表的REPLICA策略为FULL - -```sql -ALTER TABLE your_table_name REPLICA IDENTITY FULL; -``` - -如果你有很多表,你可以使用下面SQL的结果集来改变所有表的REPLICA策略 - -```sql -select 'ALTER TABLE ' || schemaname || '.' || tablename || ' REPLICA IDENTITY FULL;' from pg_tables where schemaname = 'YourTableSchema' -``` - -## 数据类型映射 - -| Opengauss Data type | SeaTunnel Data type | -|-----------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------| -| BOOL
| BOOLEAN | -| BYTEA
| BYTES | -| INT2
SMALLSERIAL
INT4
SERIAL
| INT | -| INT8
BIGSERIAL
| BIGINT | -| FLOAT4
| FLOAT | -| FLOAT8
| DOUBLE | -| NUMERIC(Get the designated column's specified column size>0) | DECIMAL(Get the designated column's specified column size,Gets the number of digits in the specified column to the right of the decimal point) | -| NUMERIC(Get the designated column's specified column size<0) | DECIMAL(38, 18) | -| BPCHAR
CHARACTER
VARCHAR
TEXT
GEOMETRY
GEOGRAPHY
JSON
JSONB | STRING | -| TIMESTAMP
| TIMESTAMP | -| TIME
| TIME | -| DATE
| DATE | -| OTHER DATA TYPES | NOT SUPPORTED YET | - -## 源端可选项 - -| Name | Type | Required | Default | Description | -|------------------------------------------------|------|----------|----------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| base-url | 字符串 | 是 | - | JDBC连接的URL. 参考: `jdbc:postgresql://localhost:5432/postgres_cdc?loggerLevel=OFF`. | -| username | 字符串 | 是 | - | 连接数据库的用户名 | -| password | 字符串 | 是 | - | 连接数据库的密码 | -| database-names | 列表 | 否 | - | 监控的数据库名称 | -| table-names | 列表 | 是 | - | 监控的数据表名称. 表名需要包含数据库名称, 例如: `database_name.table_name` | -| table-names-config | 列表 | 否 | - | 表配置的列表集合. 例如: [{"table": "db1.schema1.table1","primaryKeys":["key1"]}] | -| startup.mode | 枚举 | 否 | INITIAL | Opengauss CDC消费者的可选启动模式, 有效的枚举是`initial`, `earliest`, `latest` and `specific`.
`initial`: 启动时同步历史数据,然后同步增量数据
`earliest`: 从可能的最早偏移量启动
`latest`: 从最近的偏移量启动
`specific`: 从用户指定的偏移量启动 | -| snapshot.split.size | 整型 | 否 | 8096 | 表快照的分割大小(行数),在读取表的快照时,捕获的表被分割成多个split | -| snapshot.fetch.size | 整型 | 否 | 1024 | 读取表快照时,每次轮询的最大读取大小 | -| slot.name | 字符串 | 否 | - | Opengauss逻辑解码插槽的名称,该插槽是为特定数据库/模式的特定插件的流式更改而创建的。服务器使用此插槽将事件流传输到正在配置的连接器。默认值为seatunnel | -| decoding.plugin.name | 字符串 | 否 | pgoutput | 安装在服务器上的Postgres逻辑解码插件的名称,支持的值是decoderbufs、wal2json、wal2json_rds、wal2json_streaming、wal2json_rds_streaming和pgoutput | -| server-time-zone | 字符串 | 否 | UTC | 数据库服务器中的会话时区。如果没有设置,则使用ZoneId.systemDefault()来确定服务器的时区 | -| connect.timeout.ms | 时间间隔 | 否 | 30000 | 在尝试连接数据库服务器之后,连接器在超时之前应该等待的最大时间 | -| connect.max-retries | 整型 | 否 | 3 | 连接器在建立数据库服务器连接时应该重试的最大次数 | -| connection.pool.size | 整型 | 否 | 20 | jdbc连接池的大小 | -| chunk-key.even-distribution.factor.upper-bound | 双浮点型 | 否 | 100 | chunk的key分布因子的上界。该因子用于确定表数据是否均匀分布。如果分布因子被计算为小于或等于这个上界(即(MAX(id) - MIN(id) + 1) /行数),表的所有chunk将被优化以达到均匀分布。否则,如果分布因子更大,则认为表分布不均匀,如果估计的分片数量超过`sample-sharding.threshold`指定的值,则将使用基于采样的分片策略。默认值为100.0。 | -| chunk-key.even-distribution.factor.lower-bound | 双浮点型 | 否 | 0.05 | chunk的key分布因子的下界。该因子用于确定表数据是否均匀分布。如果分布因子的计算结果大于或等于这个下界(即(MAX(id) - MIN(id) + 1) /行数),那么表的所有块将被优化以达到均匀分布。否则,如果分布因子较小,则认为表分布不均匀,如果估计的分片数量超过`sample-sharding.threshold`指定的值,则使用基于采样的分片策略。缺省值为0.05。 | -| sample-sharding.threshold | 整型 | 否 | 1000 | 此配置指定了用于触发采样分片策略的估计分片数的阈值。当分布因子超出了由`chunk-key.even-distribution.factor.upper-bound `和`chunk-key.even-distribution.factor.lower-bound`,并且估计的分片计数(以近似的行数/块大小计算)超过此阈值,则将使用样本分片策略。这有助于更有效地处理大型数据集。默认值为1000个分片。 | -| inverse-sampling.rate | 整型 | 否 | 1000 | 采样分片策略中使用的采样率的倒数。例如,如果该值设置为1000,则意味着在采样过程中应用了1/1000的采样率。该选项提供了控制采样粒度的灵活性,从而影响最终的分片数量。当处理非常大的数据集时,它特别有用,其中首选较低的采样率。缺省值为1000。 | -| exactly_once | 布尔 | 否 | false | 启用exactly once语义 | -| format | 枚举 | 否 | DEFAULT | Opengauss CDC可选的输出格式, 有效的枚举是`DEFAULT`, `COMPATIBLE_DEBEZIUM_JSON`. | -| debezium | 配置 | 否 | - | 将 [Debezium的属性](https://github.com/debezium/debezium/blob/v1.9.8.Final/documentation/modules/ROOT/pages/connectors/postgresql.adoc#connector-configuration-properties) 传递到Debezium嵌入式引擎,该引擎用于捕获来自Opengauss服务的数据更改 | -| common-options | | 否 | - | 源码插件通用参数, 请参考[Source Common Options](../source-common-options.md)获取详情 | - -## 任务示例 - -### 简单 - -> 支持多表读 - -``` - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1","opengauss_cdc.inventory.opengauss_cdc_table_2"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" - decoding.plugin.name = "pgoutput" - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = "opengauss_cdc" - schema = "inventory" - tablePrefix = "sink_" - primary_keys = ["id"] - } -} - -``` - -### 支持自定义主键 - -``` -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - exactly_once = true - table-names-config = [ - { - table = "opengauss_cdc.inventory.full_types_no_primary_key" - primaryKeys = ["id"] - } - ] - } -} -``` - diff --git a/docs/zh/connector-v2/source/Qdrant.md b/docs/zh/connector-v2/source/Qdrant.md deleted file mode 100644 index 140ff36a395..00000000000 --- a/docs/zh/connector-v2/source/Qdrant.md +++ /dev/null @@ -1,79 +0,0 @@ -# Qdrant - -> Qdrant 数据源连接器 - -[Qdrant](https://qdrant.tech/) 是一个高性能的向量搜索引擎和向量数据库。 - -该连接器可用于从 Qdrant 集合中读取数据。 - -## 选项 - -| 名称 | 类型 | 必填 | 默认值 | -|-----------------|--------|----|-----------| -| collection_name | string | 是 | - | -| schema | config | 是 | - | -| host | string | 否 | localhost | -| port | int | 否 | 6334 | -| api_key | string | 否 | - | -| use_tls | bool | 否 | false | -| common-options | | 否 | - | - -### collection_name [string] - -要从中读取数据的 Qdrant 集合的名称。 - -### schema [config] - -要将数据读取到的表的模式。 - -例如: - -```hocon -schema = { - fields { - age = int - address = string - some_vector = float_vector - } -} -``` - -Qdrant 中的每个条目称为一个点。 - -`float_vector` 类型的列从每个点的向量中读取,其他列从与该点关联的 JSON 有效负载中读取。 - -如果列被标记为主键,Qdrant 点的 ID 将写入其中。它可以是 `"string"` 或 `"int"` 类型。因为 Qdrant 仅[允许](https://qdrant.tech/documentation/concepts/points/#point-ids)使用正整数和 UUID 作为点 ID。 - -如果集合是用单个默认/未命名向量创建的,请使用 `default_vector` 作为向量名称。 - -```hocon -schema = { - fields { - age = int - address = string - default_vector = float_vector - } -} -``` - -Qdrant 中点的 ID 将写入标记为主键的列中。它可以是 `int` 或 `string` 类型。 - -### host [string] - -Qdrant 实例的主机名。默认为 "localhost"。 - -### port [int] - -Qdrant 实例的 gRPC 端口。 - -### api_key [string] - -用于身份验证的 API 密钥(如果设置)。 - -### use_tls [bool] - -是否使用 TLS(SSL)连接。如果使用 Qdrant 云(https),则需要。 - -### 通用选项 - -源插件的通用参数,请参考[源通用选项](../source-common-options.md)了解详情。**** diff --git a/docs/zh/connector-v2/source/Sls.md b/docs/zh/connector-v2/source/Sls.md index 59b00df0850..d0e10257258 100644 --- a/docs/zh/connector-v2/source/Sls.md +++ b/docs/zh/connector-v2/source/Sls.md @@ -26,9 +26,9 @@ 为了使用Sls连接器,需要以下依赖关系。 它们可以通过install-plugin.sh或Maven中央存储库下载。 -| 数据源 | 支持的版本 | Maven | -|-----|-----------|-----------------------------------------------------------------------------------| -| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/connector-sls) | +| 数据源 | 支持的版本 | Maven | +|-----|-----------|-----------------------------------------------------------------------------------------------------------| +| Sls | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-sls) | ## Source Options diff --git a/docs/zh/connector-v2/source/Typesense.md b/docs/zh/connector-v2/source/Typesense.md deleted file mode 100644 index 35f04e23a27..00000000000 --- a/docs/zh/connector-v2/source/Typesense.md +++ /dev/null @@ -1,79 +0,0 @@ -# Typesense - -> Typesense 源连接器 - -## 描述 - -从 Typesense 读取数据。 - -## 主要功能 - -- [x] [批处理](../../concept/connector-v2-features.md) -- [ ] [流处理](../../concept/connector-v2-features.md) -- [ ] [精确一次](../../concept/connector-v2-features.md) -- [x] [Schema](../../concept/connector-v2-features.md) -- [x] [并行度](../../concept/connector-v2-features.md) -- [ ] [支持用户定义的拆分](../../concept/connector-v2-features.md) - -## 选项 - -| 名称 | 类型 | 必填 | 默认值 | -|------------|--------|----|-----| -| hosts | array | 是 | - | -| collection | string | 是 | - | -| schema | config | 是 | - | -| api_key | string | 否 | - | -| query | string | 否 | - | -| batch_size | int | 否 | 100 | - -### hosts [array] - -Typesense的访问地址,格式为 `host:port`,例如:["typesense-01:8108"] - -### collection [string] - -要写入的集合名,例如:“seatunnel” - -### schema [config] - -typesense 需要读取的列。有关更多信息,请参阅:[guide](../../concept/schema-feature.md#how-to-declare-type-supported)。 - -### api_key [config] - -typesense 安全认证的 api_key。 - -### batch_size - -读取数据时,每批次查询数量 - -### 常用选项 - -Source 插件常用参数,具体请参考 [Source 常用选项](../source-common-options.md) - -## 示例 - -```bash -source { - Typesense { - hosts = ["localhost:8108"] - collection = "companies" - api_key = "xyz" - query = "q=*&filter_by=num_employees:>9000" - schema = { - fields { - company_name_list = array - company_name = string - num_employees = long - country = string - id = string - c_row = { - c_int = int - c_string = string - c_array_int = array - } - } - } - } -} -``` - diff --git a/docs/zh/faq.md b/docs/zh/faq.md index 4fc24e6a3ad..3be6ce38e56 100644 --- a/docs/zh/faq.md +++ b/docs/zh/faq.md @@ -204,6 +204,23 @@ spark { } ``` +## 如何为 YARN 上的 SeaTunnel 指定不同的 JDK 版本? + +例如要设置JDK版本为JDK8,有两种情况: + +- YARN集群已部署JDK8,但默认JDK不是JDK8。 在 SeaTunnel 配置文件中添加两个配置: + + ``` + env { + ... + spark.executorEnv.JAVA_HOME="/your/java_8_home/directory" + spark.yarn.appMasterEnv.JAVA_HOME="/your/java_8_home/directory" + ... + } + ``` +- YARN集群未部署JDK8。 此时,启动附带JDK8的SeaTunnel。 详细操作参见: + https://www.cnblogs.com/jasondan/p/spark-specific-jdk-version.html + ## Spark local[*]模式运行SeaTunnel时总是出现OOM怎么办? 如果以本地模式运行,则需要修改`start-seatunnel.sh`启动脚本。 在 `spark-submit` 之后添加参数 `--driver-memory 4g` 。 一般情况下,生产环境中不使用本地模式。 因此,On YARN时一般不需要设置该参数。 有关详细信息,请参阅:[应用程序属性](https://spark.apache.org/docs/latest/configuration.html#application-properties)。 @@ -318,6 +335,10 @@ spark-submit --verbose ... ``` +## 如何使用SeaTunnel跨HDFS集群同步数据? + +只需正确配置 hdfs-site.xml 即可。 参考:https://www.cnblogs.com/suanec/p/7828139.html。 + ## 我想学习SeaTunnel的源代码。 我应该从哪里开始? SeaTunnel 拥有完全抽象、结构化的代码实现,很多人都选择 SeaTunnel 作为学习 Spark 的方式。 您可以从主程序入口了解源代码:SeaTunnel.java diff --git a/docs/zh/seatunnel-engine/checkpoint-storage.md b/docs/zh/seatunnel-engine/checkpoint-storage.md index 86165d5d3be..7dd26ca11f0 100644 --- a/docs/zh/seatunnel-engine/checkpoint-storage.md +++ b/docs/zh/seatunnel-engine/checkpoint-storage.md @@ -65,6 +65,7 @@ seatunnel: fs.oss.accessKeyId: your-access-key fs.oss.accessKeySecret: your-secret-key fs.oss.endpoint: endpoint address + fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider ``` 有关Hadoop Credential Provider API的更多信息,请参见: [Credential Provider API](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/CredentialProviderAPI.html). diff --git a/docs/zh/seatunnel-engine/rest-api.md b/docs/zh/seatunnel-engine/rest-api.md index 6d5a15d3f48..69199cccc21 100644 --- a/docs/zh/seatunnel-engine/rest-api.md +++ b/docs/zh/seatunnel-engine/rest-api.md @@ -68,7 +68,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回所有作业及其当前状态的概览 +### 返回所有作业及其当前状态的概览。
GET /hazelcast/rest/maps/running-jobs (返回所有作业及其当前状态的概览。) @@ -107,7 +107,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回作业的详细信息 +### 返回作业的详细信息。
GET /hazelcast/rest/maps/job-info/:jobId (返回作业的详细信息。) @@ -233,7 +233,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回所有已完成的作业信息 +### 返回所有已完成的作业信息。
GET /hazelcast/rest/maps/finished-jobs/:state (返回所有已完成的作业信息。) @@ -265,7 +265,7 @@ network: ------------------------------------------------------------------------------------------ -### 返回系统监控信息 +### 返回系统监控信息。
GET /hazelcast/rest/maps/system-monitoring-information (返回系统监控信息。) @@ -330,7 +330,7 @@ network: ------------------------------------------------------------------------------------------ -### 提交作业 +### 提交作业。
POST /hazelcast/rest/maps/submit-job (如果作业提交成功,返回jobId和jobName。) @@ -388,110 +388,7 @@ network: ------------------------------------------------------------------------------------------ - -### 批量提交作业 - -
-POST /hazelcast/rest/maps/submit-jobs (如果作业提交成功,返回jobId和jobName。) - -#### 参数(在请求体中params字段中添加) - -> | 参数名称 | 是否必传 | 参数类型 | 参数描述 | -> |----------------------|----------|--------|-----------------------------------| -> | jobId | optional | string | job id | -> | jobName | optional | string | job name | -> | isStartWithSavePoint | optional | string | if job is started with save point | - - - -#### 请求体 - -```json -[ - { - "params":{ - "jobId":"123456", - "jobName":"SeaTunnel-01" - }, - "env": { - "job.mode": "batch" - }, - "source": [ - { - "plugin_name": "FakeSource", - "result_table_name": "fake", - "row.num": 1000, - "schema": { - "fields": { - "name": "string", - "age": "int", - "card": "int" - } - } - } - ], - "transform": [ - ], - "sink": [ - { - "plugin_name": "Console", - "source_table_name": ["fake"] - } - ] - }, - { - "params":{ - "jobId":"1234567", - "jobName":"SeaTunnel-02" - }, - "env": { - "job.mode": "batch" - }, - "source": [ - { - "plugin_name": "FakeSource", - "result_table_name": "fake", - "row.num": 1000, - "schema": { - "fields": { - "name": "string", - "age": "int", - "card": "int" - } - } - } - ], - "transform": [ - ], - "sink": [ - { - "plugin_name": "Console", - "source_table_name": ["fake"] - } - ] - } -] -``` - -#### 响应 - -```json -[ - { - "jobId": "123456", - "jobName": "SeaTunnel-01" - },{ - "jobId": "1234567", - "jobName": "SeaTunnel-02" - } -] -``` - -
- ------------------------------------------------------------------------------------------- - -### 停止作业 +### 停止作业。
POST /hazelcast/rest/maps/stop-job (如果作业成功停止,返回jobId。) @@ -515,47 +412,9 @@ network:
- ------------------------------------------------------------------------------------------- - -### 批量停止作业 - -
-POST /hazelcast/rest/maps/stop-jobs (如果作业成功停止,返回jobId。) - -#### 请求体 - -```json -[ - { - "jobId": 881432421482889220, - "isStopWithSavePoint": false - }, - { - "jobId": 881432456517910529, - "isStopWithSavePoint": false - } -] -``` - -#### 响应 - -```json -[ - { - "jobId": 881432421482889220 - }, - { - "jobId": 881432456517910529 - } -] -``` - -
- ------------------------------------------------------------------------------------------ -### 加密配置 +### 加密配置。
POST /hazelcast/rest/maps/encrypt-config (如果配置加密成功,则返回加密后的配置。) diff --git a/docs/zh/start-v2/locally/deployment.md b/docs/zh/start-v2/locally/deployment.md index e2ae8dadd03..a21612570e5 100644 --- a/docs/zh/start-v2/locally/deployment.md +++ b/docs/zh/start-v2/locally/deployment.md @@ -5,17 +5,15 @@ sidebar_position: 1 import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -# 部署 +# 本地部署 -## 准备工作 +## 步骤 1: 准备工作 在开始本地运行前,您需要确保您已经安装了SeaTunnel所需要的以下软件: * 安装[Java](https://www.java.com/en/download/) (Java 8 或 11, 其他高于Java 8的版本理论上也可以工作) 以及设置 `JAVA_HOME`。 -## 下载 SeaTunnel 发行包 - -### 下载二进制包 +## 步骤 2: 下载 SeaTunnel 进入[SeaTunnel下载页面](https://seatunnel.apache.org/download)下载最新版本的二进制安装包`seatunnel--bin.tar.gz` @@ -27,7 +25,7 @@ wget "https://archive.apache.org/dist/seatunnel/${version}/apache-seatunnel-${ve tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" ``` -### 下载连接器插件 +## 步骤 3: 下载连接器插件 从2.2.0-beta版本开始,二进制包不再默认提供连接器依赖,因此在第一次使用时,您需要执行以下命令来安装连接器:(当然,您也可以从 [Apache Maven Repository](https://repo.maven.apache.org/maven2/org/apache/seatunnel/) 手动下载连接器,然后将其移动至`connectors/`目录下,如果是2.3.5之前则需要放入`connectors/seatunnel`目录下)。 @@ -58,33 +56,4 @@ connector-console ::: -## 从源码构建SeaTunnel - -### 下载源码 - -从源码构建SeaTunnel。下载源码的方式与下载二进制包的方式相同。 -您可以从[下载页面](https://seatunnel.apache.org/download/)下载源码,或者从[GitHub仓库](https://github.com/apache/seatunnel/releases)克隆源码。 - -### 构建源码 - -```shell -cd seatunnel -sh ./mvnw clean package -DskipTests -Dskip.spotless=true -# 获取构建好的二进制包 -cp seatunnel-dist/target/apache-seatunnel-2.3.8-bin.tar.gz /The-Path-You-Want-To-Copy - -cd /The-Path-You-Want-To-Copy -tar -xzvf "apache-seatunnel-${version}-bin.tar.gz" -``` - -当从源码构建时,所有的连接器插件和一些必要的依赖(例如:mysql驱动)都包含在二进制包中。您可以直接使用连接器插件,而无需单独安装它们。 - -# 启动SeaTunnel - -现在您已经下载了SeaTunnel二进制包和连接器插件。接下来,您可以选择不同的引擎选项来运行同步任务。 - -如果您使用Flink来运行同步任务,则无需部署SeaTunnel引擎服务集群。您可以参考[Flink 引擎快速开始](quick-start-flink.md)来运行您的同步任务。 - -如果您使用Spark来运行同步任务,则无需部署SeaTunnel引擎服务集群。您可以参考[Spark 引擎快速开始](quick-start-spark.md)来运行您的同步任务。 - -如果您使用内置的SeaTunnel引擎(Zeta)来运行任务,则需要先部署SeaTunnel引擎服务。请参考[SeaTunnel 引擎快速开始](quick-start-seatunnel-engine.md)。 +现在,您已经完成了SeaTunnel部署。您可以按照[快速开始](quick-start-seatunnel-engine.md)来配置并运行数据同步作业了。 diff --git a/docs/zh/start-v2/locally/quick-start-flink.md b/docs/zh/start-v2/locally/quick-start-flink.md index 87de2808c4a..f52d8507824 100644 --- a/docs/zh/start-v2/locally/quick-start-flink.md +++ b/docs/zh/start-v2/locally/quick-start-flink.md @@ -2,7 +2,7 @@ sidebar_position: 3 --- -# Flink 引擎快速开始 +# Flink Engine快速开始 ## 步骤 1: 部署SeaTunnel及连接器 diff --git a/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md b/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md index bf2515ddb6e..16f02d6c385 100644 --- a/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md +++ b/docs/zh/start-v2/locally/quick-start-seatunnel-engine.md @@ -2,7 +2,7 @@ sidebar_position: 2 --- -# SeaTunnel 引擎快速开始 +# SeaTunnel Engine快速开始 ## 步骤 1: 部署SeaTunnel及连接器 diff --git a/docs/zh/start-v2/locally/quick-start-spark.md b/docs/zh/start-v2/locally/quick-start-spark.md index 59fbe47049a..e00a7fa68ac 100644 --- a/docs/zh/start-v2/locally/quick-start-spark.md +++ b/docs/zh/start-v2/locally/quick-start-spark.md @@ -2,7 +2,7 @@ sidebar_position: 4 --- -# Spark 引擎快速开始 +# Spark引擎快速开始 ## 步骤 1: 部署SeaTunnel及连接器 diff --git a/docs/zh/transform-v2/embedding.md b/docs/zh/transform-v2/embedding.md deleted file mode 100644 index fbee0ac33e9..00000000000 --- a/docs/zh/transform-v2/embedding.md +++ /dev/null @@ -1,382 +0,0 @@ -# Embedding - -> Embedding Transform Plugin - -## 描述 - -`Embedding` 转换插件利用 embedding 模型将文本数据转换为向量化表示。此转换可以应用于各种字段。该插件支持多种模型提供商,并且可以与不同的API集成。 - -## 配置选项 - -| 名称 | 类型 | 是否必填 | 默认值 | 描述 | -|--------------------------------|--------|------|-----|------------------------------------------------------------------| -| model_provider | enum | 是 | - | embedding模型的提供商。可选项包括 `QIANFAN`、`OPENAI` 等。 | -| api_key | string | 是 | - | 用于验证embedding服务的API密钥。 | -| secret_key | string | 是 | - | 用于额外验证的密钥。一些提供商可能需要此密钥进行安全的API请求。 | -| single_vectorized_input_number | int | 否 | 1 | 单次请求向量化的输入数量。默认值为1。 | -| vectorization_fields | map | 是 | - | 输入字段和相应的输出向量字段之间的映射。 | -| model | string | 是 | - | 要使用的具体embedding模型。例如,如果提供商为OPENAI,可以指定 `text-embedding-3-small`。 | -| api_path | string | 否 | - | embedding服务的API。通常由模型提供商提供。 | -| oauth_path | string | 否 | - | oauth 服务的 API 。 | -| custom_config | map | 否 | | 模型的自定义配置。 | -| custom_response_parse | string | 否 | | 使用 JsonPath 解析模型响应的方式。示例:`$.choices[*].message.content`。 | -| custom_request_headers | map | 否 | | 发送到模型的请求的自定义头信息。 | -| custom_request_body | map | 否 | | 请求体的自定义配置。支持占位符如 `${model}`、`${input}`。 | - -### embedding_model_provider - -用于生成 embedding 的模型提供商。常见选项包括 `DOUBAO`、`QIANFAN`、`OPENAI` 等,同时可选择 `CUSTOM` 实现自定义 embedding -模型的请求以及获取。 - -### api_key - -用于验证 embedding 服务请求的API密钥。通常由模型提供商在你注册他们的服务时提供。 - -### secret_key - -用于额外验证的密钥。一些提供商可能要求此密钥以确保API请求的安全性。 - -### single_vectorized_input_number - -指定单次请求向量化的输入数量。默认值为1。根据处理能力和模型提供商的API限制进行调整。 - -### vectorization_fields - -输入字段和相应的输出向量字段之间的映射。这使得插件可以理解要向量化的文本字段以及如何存储生成的向量。 - -```hocon -vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography -} -``` - -### model - -要使用的具体 embedding 模型。这取决于`embedding_model_provider`。例如,如果使用 OPENAI ,可以指定 `text-embedding-3-small`。 - -### api_path - -用于向 embedding 服务发送请求的API。根据提供商和所用模型的不同可能有所变化。通常由模型提供商提供。 - -### oauth_path - -用于向oauth服务发送请求的API,获取对应的认证信息。根据提供商和所用模型的不同可能有所变化。通常由模型提供商提供。 - -### custom_config - -`custom_config` 选项允许您为模型提供额外的自定义配置。这是一个映射,您可以在其中定义特定模型可能需要的各种设置。 - -### custom_response_parse - -`custom_response_parse` 选项允许您指定如何解析模型的响应。您可以使用 JsonPath -从响应中提取所需的特定数据。例如,使用 `$.data[*].embedding` 提取如下json中的 `embedding` 字段 -值,获取 `List` 嵌套 `List` 的结果。JsonPath -的使用请参考 [JsonPath 快速入门](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started) - -```json -{ - "object": "list", - "data": [ - { - "object": "embedding", - "index": 0, - "embedding": [ - -0.006929283495992422, - -0.005336422007530928, - -0.00004547132266452536, - -0.024047505110502243 - ] - } - ], - "model": "text-embedding-3-small", - "usage": { - "prompt_tokens": 5, - "total_tokens": 5 - } -} -``` - -### custom_request_headers - -`custom_request_headers` 选项允许您定义应包含在发送到模型 API 的请求中的自定义头信息。如果 API -需要标准头信息之外的额外头信息,例如授权令牌、内容类型等,这个选项会非常有用。 - -### custom_request_body - -`custom_request_body` 选项支持占位符: - -- `${model}`:用于模型名称的占位符。 -- `${input}`:用于确定输入值的占位符,同时根据 body value 的类型定义请求体请求类型。例如:`["${input}"]` -> ["input"] ( - list)。 - -### common options - -转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情 - -## 示例配置 - -```hocon -env { - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - schema = { - fields { - book_id = "int" - book_name = "string" - book_intro = "string" - author_biography = "string" - } - } - rows = [ - {fields = [1, "To Kill a Mockingbird", - "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", - "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." - ], kind = INSERT} - {fields = [2, "1984", - "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", - "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." - ], kind = INSERT} - {fields = [3, "Pride and Prejudice", - "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", - "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." - ], kind = INSERT} - {fields = [4, "The Great GatsbyThe Great Gatsby", - "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", - "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." - ], kind = INSERT} - {fields = [5, "Moby-Dick", - "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", - "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." - ], kind = INSERT} - ] - result_table_name = "fake" - } -} - -transform { - Embedding { - source_table_name = "fake" - embedding_model_provider = QIANFAN - model = bge_large_en - api_key = xxxxxxxxxx - secret_key = xxxxxxxxxx - api_path = xxxxxxxxxx - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - result_table_name = "embedding_output" - } -} - -sink { - Assert { - source_table_name = "embedding_output" - - - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } -} -``` - -### Customize the embedding model - -```hocon - -env { - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - schema = { - fields { - book_id = "int" - book_name = "string" - book_intro = "string" - author_biography = "string" - } - } - rows = [ - {fields = [1, "To Kill a Mockingbird", - "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", - "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." - ], kind = INSERT} - {fields = [2, "1984", - "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", - "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." - ], kind = INSERT} - {fields = [3, "Pride and Prejudice", - "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", - "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." - ], kind = INSERT} - {fields = [4, "The Great GatsbyThe Great Gatsby", - "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", - "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." - ], kind = INSERT} - {fields = [5, "Moby-Dick", - "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", - "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." - ], kind = INSERT} - ] - result_table_name = "fake" - } -} - -transform { - Embedding { - source_table_name = "fake" - model_provider = CUSTOM - model = text-embedding-3-small - api_key = xxxxxxxx - api_path = "http://mockserver:1080/v1/doubao/embedding" - single_vectorized_input_number = 2 - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - custom_config={ - custom_response_parse = "$.data[*].embedding" - custom_request_headers = { - "Content-Type"= "application/json" - "Authorization"= "Bearer xxxxxxx - } - custom_request_body ={ - modelx = "${model}" - inputx = ["${input}"] - } - } - result_table_name = "embedding_output_1" - } -} - -sink { - Assert { - source_table_name = "embedding_output_1" - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } -} - -``` \ No newline at end of file diff --git a/docs/zh/transform-v2/llm.md b/docs/zh/transform-v2/llm.md index 5efcf47125d..acd3245b8eb 100644 --- a/docs/zh/transform-v2/llm.md +++ b/docs/zh/transform-v2/llm.md @@ -8,23 +8,19 @@ ## 属性 -| 名称 | 类型 | 是否必须 | 默认值 | -|------------------------|--------|------|--------| -| model_provider | enum | yes | | -| output_data_type | enum | no | String | -| prompt | string | yes | | -| model | string | yes | | -| api_key | string | yes | | -| api_path | string | no | | -| custom_config | map | no | | -| custom_response_parse | string | no | | -| custom_request_headers | map | no | | -| custom_request_body | map | no | | +| 名称 | 类型 | 是否必须 | 默认值 | +|------------------|--------|------|--------------------------------------------| +| model_provider | enum | yes | | +| output_data_type | enum | no | String | +| 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、DOUBAO、CUSTOM +OPENAI ### output_data_type @@ -38,7 +34,7 @@ STRING,INT,BIGINT,DOUBLE,BOOLEAN. 从源读取的数据是这样的表格: -| name | age | +| name | age | |---------------|-----| | Jia Fan | 20 | | Hailin Wang | 20 | @@ -53,7 +49,7 @@ Determine whether someone is Chinese or American by their name 这将返回: -| name | age | llm_output | +| name | age | llm_output | |---------------|-----|------------| | Jia Fan | 20 | Chinese | | Hailin Wang | 20 | Chinese | @@ -63,68 +59,16 @@ Determine whether someone is Chinese or American by their name ### model 要使用的模型。不同的模型提供者有不同的模型。例如,OpenAI 模型可以是 `gpt-4o-mini`。 -如果使用 OpenAI 模型,请参考 https://platform.openai.com/docs/models/model-endpoint-compatibility -文档的`/v1/chat/completions` 端点。 +如果使用 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 密钥。 -### api_path +### openai.api_path -用于模型提供者的 API 路径。在大多数情况下,您不需要更改此配置。如果使用 API 代理的服务,您可能需要将其配置为代理的 API 地址。 - -### custom_config - -`custom_config` 选项允许您为模型提供额外的自定义配置。这是一个 Map,您可以在其中定义特定模型可能需要的各种设置。 - -### custom_response_parse - -`custom_response_parse` 选项允许您指定如何解析模型的响应。您可以使用 JsonPath -从响应中提取所需的特定数据。例如,使用 `$.choices[*].message.content` 提取如下json中的 `content` 字段 -值。JsonPath 的使用请参考 [JsonPath 快速入门](https://github.com/json-path/JsonPath?tab=readme-ov-file#getting-started) - -```json -{ - "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" -} -``` - -### custom_request_headers - -`custom_request_headers` 选项允许您定义应包含在发送到模型 API 的请求中的自定义头信息。如果 API -需要标准头信息之外的额外头信息,例如授权令牌、内容类型等,这个选项会非常有用。 - -### custom_request_body - -`custom_request_body` 选项支持占位符: - -- `${model}`:用于模型名称的占位符。 -- `${input}`:用于确定输入值的占位符,同时根据 body value 的类型定义请求体请求类型。例如:`"${input}"` -> "input"。 -- `${prompt}`:用于 LLM 模型提示的占位符。 +用于 OpenAI 模型提供者的 API 路径。在大多数情况下,您不需要更改此配置。如果使用 API 代理的服务,您可能需要将其配置为代理的 API 地址。 ### common options [string] @@ -174,83 +118,3 @@ sink { } ``` -### Customize the LLM model - -```hocon -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 = CUSTOM - 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" - custom_config={ - custom_response_parse = "$.choices[*].message.content" - custom_request_headers = { - Content-Type = "application/json" - Authorization = "Bearer xxxxxxxx" - } - custom_request_body ={ - model = "${model}" - messages = [ - { - role = "system" - content = "${prompt}" - }, - { - role = "user" - content = "${input}" - }] - } - } - 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 - } - ] - } - ] - } - } -} -``` - - diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 630d56654d4..a74b9e1223e 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -85,7 +85,6 @@ seatunnel.sink.InfluxDB = connector-influxdb seatunnel.source.GoogleSheets = connector-google-sheets seatunnel.sink.GoogleFirestore = connector-google-firestore seatunnel.sink.Tablestore = connector-tablestore -seatunnel.source.Tablestore = connector-tablestore seatunnel.source.Lemlist = connector-http-lemlist seatunnel.source.Klaviyo = connector-http-klaviyo seatunnel.sink.Slack = connector-slack @@ -131,12 +130,8 @@ seatunnel.sink.ObsFile = connector-file-obs seatunnel.source.Milvus = connector-milvus seatunnel.sink.Milvus = connector-milvus seatunnel.sink.ActiveMQ = connector-activemq -seatunnel.source.Qdrant = connector-qdrant -seatunnel.sink.Qdrant = connector-qdrant seatunnel.source.Sls = connector-sls -seatunnel.source.Typesense = connector-typesense -seatunnel.sink.Typesense = connector-typesense -seatunnel.source.Opengauss-CDC = connector-cdc-opengauss + seatunnel.transform.Sql = seatunnel-transforms-v2 seatunnel.transform.FieldMapper = seatunnel-transforms-v2 seatunnel.transform.Filter = seatunnel-transforms-v2 @@ -147,5 +142,3 @@ seatunnel.transform.Split = seatunnel-transforms-v2 seatunnel.transform.Copy = seatunnel-transforms-v2 seatunnel.transform.DynamicCompile = seatunnel-transforms-v2 seatunnel.transform.LLM = seatunnel-transforms-v2 -seatunnel.transform.Embedding = seatunnel-transforms-v2 - diff --git a/pom.xml b/pom.xml index 043f8fc0d86..3815a5c393d 100644 --- a/pom.xml +++ b/pom.xml @@ -789,8 +789,6 @@ --no-cache -t ${docker.hub}/${docker.repo}:${docker.tag} - -t - ${docker.hub}/${docker.repo}:latest ${project.basedir} --build-arg VERSION=${project.version} @@ -844,8 +842,8 @@ linux/amd64,linux/arm64 --no-cache --push - --all-tags - ${docker.hub}/${docker.repo} + -t + ${docker.hub}/${docker.repo}:${docker.tag} ${project.basedir} --build-arg VERSION=${revision} diff --git a/release-note.md b/release-note.md index 4ed0d51fed9..32067c22dfa 100644 --- a/release-note.md +++ b/release-note.md @@ -58,7 +58,6 @@ - [Connector-v2] [Mongodb] Support to convert to double from numeric type that mongodb saved it as numeric internally (#6997) - [Connector-v2] [Redis] Using scan replace keys operation command,support batchWrite in single mode(#7030,#7085) - [Connector-V2] [Clickhouse] Add a new optional configuration `clickhouse.config` to the source connector of ClickHouse (#7143) -- [Connector-V2] [ElasticsSource] Source support multiSource (#6730) ### Zeta(ST-Engine) @@ -88,7 +87,6 @@ - [zeta] dynamically replace the value of the variable at runtime (#4950) - [Zeta] Add from_unixtime function (#5462) - [zeta] Fix CDC task restore throw NPE (#5507) -- [Zeta] Fix a checkpoint storage document with OSS (#7507) ### E2E 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 3f7f7fa9c6a..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 @@ -64,7 +64,6 @@ public String getPluginName() { public SinkWriter createWriter( SinkWriter.Context context) throws IOException { Map> writers = new HashMap<>(); - Map sinkWritersContext = new HashMap<>(); for (int i = 0; i < replicaNum; i++) { for (String tableIdentifier : sinks.keySet()) { SeaTunnelSink sink = sinks.get(tableIdentifier); @@ -72,18 +71,15 @@ public SinkWriter createWri writers.put( SinkIdentifier.of(tableIdentifier, index), sink.createWriter(new SinkContextProxy(index, context))); - sinkWritersContext.put(SinkIdentifier.of(tableIdentifier, index), context); } } - return new MultiTableSinkWriter(writers, replicaNum, sinkWritersContext); + return new MultiTableSinkWriter(writers, replicaNum); } @Override public SinkWriter restoreWriter( SinkWriter.Context context, List states) throws IOException { Map> writers = new HashMap<>(); - Map sinkWritersContext = new HashMap<>(); - for (int i = 0; i < replicaNum; i++) { for (String tableIdentifier : sinks.keySet()) { SeaTunnelSink sink = sinks.get(tableIdentifier); @@ -106,10 +102,9 @@ public SinkWriter restoreWr sinkIdentifier, sink.restoreWriter(new SinkContextProxy(index, context), state)); } - sinkWritersContext.put(SinkIdentifier.of(tableIdentifier, index), context); } } - return new MultiTableSinkWriter(writers, replicaNum, sinkWritersContext); + return new MultiTableSinkWriter(writers, replicaNum); } @Override diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java index 649417351ae..3c73435fafb 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/multitablesink/MultiTableSinkWriter.java @@ -20,10 +20,8 @@ import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; -import org.apache.seatunnel.api.sink.event.WriterCloseEvent; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.tracing.MDCTracer; import lombok.extern.slf4j.Slf4j; @@ -47,7 +45,6 @@ public class MultiTableSinkWriter implements SinkWriter { private final Map> sinkWriters; - private final Map sinkWritersContext; private final Map> sinkPrimaryKeys = new HashMap<>(); private final List>> sinkWritersWithIndex; private final List runnable = new ArrayList<>(); @@ -58,28 +55,21 @@ public class MultiTableSinkWriter private volatile boolean submitted = false; public MultiTableSinkWriter( - Map> sinkWriters, - int queueSize, - Map sinkWritersContext) { + Map> sinkWriters, int queueSize) { this.sinkWriters = sinkWriters; - this.sinkWritersContext = sinkWritersContext; AtomicInteger cnt = new AtomicInteger(0); executorService = - MDCTracer.tracing( - Executors.newFixedThreadPool( - // we use it in `MultiTableWriterRunnable` and `prepare commit - // task`, so it - // should be double. - queueSize * 2, - runnable -> { - Thread thread = new Thread(runnable); - thread.setDaemon(true); - thread.setName( - "st-multi-table-sink-writer" - + "-" - + cnt.incrementAndGet()); - return thread; - })); + Executors.newFixedThreadPool( + // we use it in `MultiTableWriterRunnable` and `prepare commit task`, so it + // should be double. + queueSize * 2, + runnable -> { + Thread thread = new Thread(runnable); + thread.setDaemon(true); + thread.setName( + "st-multi-table-sink-writer" + "-" + cnt.incrementAndGet()); + return thread; + }); sinkWritersWithIndex = new ArrayList<>(); for (int i = 0; i < queueSize; i++) { BlockingQueue queue = new LinkedBlockingQueue<>(1024); @@ -94,7 +84,6 @@ public MultiTableSinkWriter( entry.getKey().getTableIdentifier(), entry.getValue()); sinkIdentifierMap.put(entry.getKey(), entry.getValue()); }); - sinkWritersWithIndex.add(sinkIdentifierMap); blockingQueues.add(queue); MultiTableWriterRunnable r = new MultiTableWriterRunnable(tableIdWriterMap, queue); @@ -278,34 +267,26 @@ public void abortPrepare() { @Override public void close() throws IOException { - // The variables used in lambda expressions should be final or valid final, so they are - // modified to arrays - final Throwable[] firstE = {null}; + Throwable firstE = null; try { checkQueueRemain(); } catch (Exception e) { - firstE[0] = e; + firstE = e; } executorService.shutdownNow(); for (int i = 0; i < sinkWritersWithIndex.size(); i++) { synchronized (runnable.get(i)) { - Map> sinkIdentifierSinkWriterMap = - sinkWritersWithIndex.get(i); - sinkIdentifierSinkWriterMap.forEach( - (identifier, sinkWriter) -> { - try { - sinkWriter.close(); - sinkWritersContext - .get(identifier) - .getEventListener() - .onEvent(new WriterCloseEvent()); - } catch (Throwable e) { - if (firstE[0] == null) { - firstE[0] = e; - } - log.error("close error", e); - } - }); + for (SinkWriter sinkWriter : + sinkWritersWithIndex.get(i).values()) { + try { + sinkWriter.close(); + } catch (Throwable e) { + if (firstE == null) { + firstE = e; + } + log.error("close error", e); + } + } } } try { @@ -315,8 +296,8 @@ public void close() throws IOException { } catch (Throwable e) { log.error("close resourceManager error", e); } - if (firstE[0] != null) { - throw new RuntimeException(firstE[0]); + if (firstE != null) { + throw new RuntimeException(firstE); } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java index ab85455b34e..e043c0ecd72 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/schema/ReadonlyConfigParser.java @@ -95,8 +95,7 @@ public List parse(ReadonlyConfig schemaConfig) { String value = entry.getValue(); SeaTunnelDataType dataType = SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType(key, value); - PhysicalColumn column = - PhysicalColumn.of(key, dataType, null, null, true, null, null); + PhysicalColumn column = PhysicalColumn.of(key, dataType, 0, true, null, null); columns.add(column); } return columns; diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java index 10a5b33a935..801f037b186 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java @@ -295,9 +295,6 @@ private int getBytesForValue(Object v) { size += getBytesForValue(entry.getKey()) + getBytesForValue(entry.getValue()); } return size; - case "HeapByteBuffer": - case "ByteBuffer": - return ((ByteBuffer) v).capacity(); case "SeaTunnelRow": int rowSize = 0; SeaTunnelRow row = (SeaTunnelRow) v; diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java deleted file mode 100644 index f3cae160dab..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCCallable.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -import java.util.concurrent.Callable; - -/** - * Callable that sets MDC context before calling the delegate and clears it afterwards. - * - * @param - */ -public class MDCCallable implements Callable { - private final MDCContext context; - private final Callable delegate; - - public MDCCallable(Callable delegate) { - this(MDCContext.current(), delegate); - } - - public MDCCallable(MDCContext context, Callable delegate) { - this.context = context; - this.delegate = delegate; - } - - @Override - public V call() throws Exception { - try { - context.put(); - - return delegate.call(); - } finally { - context.clear(); - } - } -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java deleted file mode 100644 index 4a439374590..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCContext.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -import org.slf4j.MDC; - -import lombok.Builder; -import lombok.EqualsAndHashCode; -import lombok.extern.slf4j.Slf4j; - -import java.io.Serializable; - -/** - * MDC context for tracing. - * - *

reference: https://www.slf4j.org/manual.html#mdc - */ -@Slf4j -@Builder -@EqualsAndHashCode -public class MDCContext implements Serializable { - private static final MDCContext EMPTY = MDCContext.builder().build(); - private static final String EMPTY_TO_STRING = "NA"; - - public static final String JOB_ID = "ST-JID"; - public static final String PIPELINE_ID = "ST-PID"; - public static final String TASK_ID = "ST-TID"; - - private final Long jobId; - private final Long pipelineId; - private final Long taskId; - - public static MDCContext of(long jobId) { - return MDCContext.builder().jobId(jobId).build(); - } - - public static MDCContext of(long jobId, long pipelineId) { - return MDCContext.builder().jobId(jobId).pipelineId(pipelineId).build(); - } - - public static MDCContext of(long jobId, long pipelineId, long taskId) { - return MDCContext.builder().jobId(jobId).pipelineId(pipelineId).taskId(taskId).build(); - } - - public static MDCContext current() { - return MDCContext.builder() - .jobId(MDC.get(JOB_ID) != null ? Long.parseLong(MDC.get(JOB_ID)) : null) - .pipelineId( - MDC.get(PIPELINE_ID) != null ? Long.parseLong(MDC.get(PIPELINE_ID)) : null) - .taskId(MDC.get(TASK_ID) != null ? Long.parseLong(MDC.get(TASK_ID)) : null) - .build(); - } - - public static MDCContext valueOf(String s) { - if (EMPTY_TO_STRING.equals(s)) { - return EMPTY; - } - - String[] arr = s.split("/"); - Long jobId = Long.parseLong(arr[0]); - Long pipelineId = Long.parseLong(arr[1]); - Long taskId = Long.parseLong(arr[2]); - if (pipelineId == 0 || taskId == 0) { - return MDCContext.of(jobId); - } - return MDCContext.of(jobId, pipelineId, taskId); - } - - @Override - public String toString() { - if (jobId != null) { - return String.format( - "%d/%d/%d", - jobId, pipelineId == null ? 0 : pipelineId, taskId == null ? 0 : taskId); - } else { - return EMPTY_TO_STRING; - } - } - - public void put() { - try { - if (jobId != null) { - MDC.put(JOB_ID, String.valueOf(jobId)); - } - if (pipelineId != null) { - MDC.put(PIPELINE_ID, String.valueOf(pipelineId)); - } - if (taskId != null) { - MDC.put(TASK_ID, String.valueOf(taskId)); - } - } catch (Throwable e) { - log.error("Failed to put MDC context", e); - throw e; - } - } - - public void clear() { - try { - MDC.remove(JOB_ID); - MDC.remove(PIPELINE_ID); - MDC.remove(TASK_ID); - } catch (Throwable e) { - log.error("Failed to clear MDC context", e); - throw e; - } - } -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.java deleted file mode 100644 index 1651f7d6c40..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutor.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -import java.util.concurrent.Executor; - -/** Executor that sets MDC context before calling the delegate and clears it afterwards. */ -public class MDCExecutor implements Executor { - private final MDCContext context; - private final Executor delegate; - - public MDCExecutor(MDCContext context, Executor delegate) { - this.context = context; - this.delegate = delegate; - } - - @Override - public void execute(Runnable command) { - delegate.execute(new MDCRunnable(context, command)); - } -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java deleted file mode 100644 index 7ef93f41dfc..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCExecutorService.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; - -/** ExecutorService that sets MDC context before calling the delegate and clears it afterwards. */ -public class MDCExecutorService extends MDCExecutor implements ExecutorService { - private final MDCContext context; - private final ExecutorService delegate; - - public MDCExecutorService(MDCContext context, ExecutorService delegate) { - super(context, delegate); - this.context = context; - this.delegate = delegate; - } - - @Override - public void shutdown() { - delegate.shutdown(); - } - - @Override - public List shutdownNow() { - return delegate.shutdownNow(); - } - - @Override - public boolean isShutdown() { - return delegate.isShutdown(); - } - - @Override - public boolean isTerminated() { - return delegate.isTerminated(); - } - - @Override - public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { - return delegate.awaitTermination(timeout, unit); - } - - @Override - public Future submit(Callable task) { - return delegate.submit(new MDCCallable<>(context, task)); - } - - @Override - public Future submit(Runnable task, T result) { - return delegate.submit(new MDCRunnable(context, task), result); - } - - @Override - public Future submit(Runnable task) { - return delegate.submit(new MDCRunnable(context, task)); - } - - @Override - public List> invokeAll(Collection> tasks) - throws InterruptedException { - return delegate.invokeAll( - tasks.stream() - .map(task -> new MDCCallable<>(context, task)) - .collect(Collectors.toList())); - } - - @Override - public List> invokeAll( - Collection> tasks, long timeout, TimeUnit unit) - throws InterruptedException { - return delegate.invokeAll( - tasks.stream() - .map(task -> new MDCCallable<>(context, task)) - .collect(Collectors.toList()), - timeout, - unit); - } - - @Override - public T invokeAny(Collection> tasks) - throws InterruptedException, ExecutionException { - return delegate.invokeAny( - tasks.stream() - .map(task -> new MDCCallable<>(context, task)) - .collect(Collectors.toList())); - } - - @Override - public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - return delegate.invokeAny( - tasks.stream() - .map(task -> new MDCCallable<>(context, task)) - .collect(Collectors.toList()), - timeout, - unit); - } -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java deleted file mode 100644 index e6d310de10e..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCRunnable.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -/** Runnable that sets MDC context before calling the delegate and clears it afterwards. */ -public class MDCRunnable implements Runnable { - private final MDCContext context; - private final Runnable delegate; - - public MDCRunnable(Runnable delegate) { - this(MDCContext.current(), delegate); - } - - public MDCRunnable(MDCContext context, Runnable delegate) { - this.context = context; - this.delegate = delegate; - } - - @Override - public void run() { - try { - context.put(); - - delegate.run(); - } finally { - context.clear(); - } - } -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java deleted file mode 100644 index 804e953aced..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCScheduledExecutorService.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -import java.util.concurrent.Callable; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; - -/** - * ScheduledExecutorService that sets MDC context before calling the delegate and clears it - * afterwards. - */ -public class MDCScheduledExecutorService extends MDCExecutorService - implements ScheduledExecutorService { - private final MDCContext context; - private final ScheduledExecutorService delegate; - - public MDCScheduledExecutorService(MDCContext context, ScheduledExecutorService delegate) { - super(context, delegate); - this.context = context; - this.delegate = delegate; - } - - @Override - public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - return delegate.schedule(new MDCRunnable(context, command), delay, unit); - } - - @Override - public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { - return delegate.schedule(new MDCCallable<>(context, callable), delay, unit); - } - - @Override - public ScheduledFuture scheduleAtFixedRate( - Runnable command, long initialDelay, long period, TimeUnit unit) { - return delegate.scheduleAtFixedRate( - new MDCRunnable(context, command), initialDelay, period, unit); - } - - @Override - public ScheduledFuture scheduleWithFixedDelay( - Runnable command, long initialDelay, long delay, TimeUnit unit) { - return delegate.scheduleWithFixedDelay( - new MDCRunnable(context, command), initialDelay, delay, unit); - } -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java deleted file mode 100644 index 1ce2120cc3c..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/tracing/MDCTracer.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -import java.util.concurrent.Callable; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; - -/** - * Tracer for MDC context. - * - *

It wraps the given {@link Runnable}, {@link Callable}, {@link Executor}, {@link - * ExecutorService}, {@link ScheduledExecutorService} to trace the MDC context. - * - *

It is useful to trace the MDC context in the asynchronous execution. For example, when you - * submit a task to the {@link ExecutorService}, the MDC context is not propagated to the worker - * thread. - * - *

It is recommended to use the {@link MDCTracer} to wrap the task to trace the MDC context. - * - *

{@code
- * MDCContext mdcContext = MDCContext.of(1);
- * ExecutorService executorService = Executors.newFixedThreadPool(10);
- * executorService.submit(MDCTracer.tracing(mdcContext, () -> {
- *    // Your task
- *    logger.info("Task is running");
- *    return null;
- *    }));
- *
- * }
- */ -public class MDCTracer { - - public static MDCRunnable tracing(Runnable delegate) { - return tracing(MDCContext.current(), delegate); - } - - public static MDCRunnable tracing(Long jobId, Runnable delegate) { - return tracing(MDCContext.of(jobId), delegate); - } - - public static MDCRunnable tracing(MDCContext context, Runnable delegate) { - if (delegate instanceof MDCRunnable) { - throw new IllegalArgumentException("Already an MDCRunnable"); - } - return new MDCRunnable(context, delegate); - } - - public static MDCCallable tracing(Callable delegate) { - return tracing(MDCContext.current(), delegate); - } - - public static MDCCallable tracing(Long jobId, Callable delegate) { - return tracing(MDCContext.of(jobId), delegate); - } - - public static MDCCallable tracing(MDCContext context, Callable delegate) { - if (delegate instanceof MDCCallable) { - throw new IllegalArgumentException("Already an MDCCallable"); - } - return new MDCCallable<>(context, delegate); - } - - public static MDCExecutor tracing(Executor delegate) { - return tracing(MDCContext.current(), delegate); - } - - public static MDCExecutor tracing(Long jobId, Executor delegate) { - return tracing(MDCContext.of(jobId), delegate); - } - - public static MDCExecutor tracing(MDCContext context, Executor delegate) { - if (delegate instanceof MDCExecutor) { - throw new IllegalArgumentException("Already an MDCExecutor"); - } - return new MDCExecutor(context, delegate); - } - - public static MDCExecutorService tracing(ExecutorService delegate) { - return tracing(MDCContext.current(), delegate); - } - - public static MDCExecutorService tracing(Long jobId, ExecutorService delegate) { - return tracing(MDCContext.of(jobId), delegate); - } - - public static MDCExecutorService tracing(MDCContext context, ExecutorService delegate) { - if (delegate instanceof MDCExecutor) { - throw new IllegalArgumentException("Already an MDCExecutor"); - } - return new MDCExecutorService(context, delegate); - } - - public static MDCScheduledExecutorService tracing(ScheduledExecutorService delegate) { - return tracing(MDCContext.current(), delegate); - } - - public static MDCScheduledExecutorService tracing( - Long jobId, ScheduledExecutorService delegate) { - return tracing(MDCContext.of(jobId), delegate); - } - - public static MDCScheduledExecutorService tracing( - MDCContext context, ScheduledExecutorService delegate) { - if (delegate instanceof MDCExecutor) { - throw new IllegalArgumentException("Already an MDCExecutor"); - } - return new MDCScheduledExecutorService(context, delegate); - } -} diff --git a/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java b/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java deleted file mode 100644 index 88ac95885f6..00000000000 --- a/seatunnel-api/src/test/java/org/apache/seatunnel/api/tracing/MDCTracerTest.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.tracing; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.slf4j.MDC; - -import java.util.concurrent.Callable; -import java.util.concurrent.Executors; - -public class MDCTracerTest { - - @Test - public void testMDCTracedRunnable() { - MDCContext mdcContext = MDCContext.of(1, 2, 3); - Runnable tracedRunnable = - MDCTracer.tracing( - mdcContext, - new Runnable() { - @Override - public void run() { - Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); - Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); - } - }); - - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - - tracedRunnable.run(); - - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - } - - @Test - public void testMDCTracedCallable() throws Exception { - MDCContext mdcContext = MDCContext.of(1, 2, 3); - - Callable tracedCallable = - MDCTracer.tracing( - mdcContext, - new Callable() { - @Override - public Void call() throws Exception { - Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); - Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); - return null; - } - }); - - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - - tracedCallable.call(); - - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - } - - @Test - public void testMDCTracedExecutorService() throws Exception { - MDCContext mdcContext = MDCContext.of(1, 2, 3); - - MDCExecutorService tracedExecutorService = - MDCTracer.tracing(mdcContext, Executors.newSingleThreadExecutor()); - - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - tracedExecutorService - .submit( - new Runnable() { - @Override - public void run() { - Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); - Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); - } - }) - .get(); - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - tracedExecutorService - .submit( - new Callable() { - @Override - public Void call() throws Exception { - Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); - Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); - return null; - } - }) - .get(); - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - } - - @Test - public void testMDCContext() throws Exception { - MDCContext.current(); - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - - MDCContext mdcContext = MDCContext.of(1, 2, 3); - mdcContext.put(); - Assertions.assertEquals("1", MDC.get(MDCContext.JOB_ID)); - Assertions.assertEquals("2", MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertEquals("3", MDC.get(MDCContext.TASK_ID)); - - MDCContext currentMDCCOntext = MDCContext.current(); - Assertions.assertEquals(mdcContext, currentMDCCOntext); - - mdcContext.clear(); - Assertions.assertNull(MDC.get(MDCContext.JOB_ID)); - Assertions.assertNull(MDC.get(MDCContext.PIPELINE_ID)); - Assertions.assertNull(MDC.get(MDCContext.TASK_ID)); - } -} 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 0148e651884..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 @@ -266,8 +266,8 @@ public static SeaTunnelRuntimeException writeRowErrorWithFiledsCountNotMatch( String connector, int sourceFieldsNum, int sinkFieldsNum) { Map params = new HashMap<>(); params.put("connector", connector); - params.put("sourceFieldsNum", String.valueOf(sourceFieldsNum)); - params.put("sinkFieldsNum", String.valueOf(sinkFieldsNum)); + params.put("sourceFiledName", String.valueOf(sourceFieldsNum)); + params.put("sourceFiledType", String.valueOf(sinkFieldsNum)); return new SeaTunnelRuntimeException( WRITE_SEATUNNEL_ROW_ERROR_WITH_FILEDS_NOT_MATCH, params); } diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java deleted file mode 100644 index d5ddd48106b..00000000000 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/TemporaryClassLoaderContext.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.common.utils; - -public final class TemporaryClassLoaderContext implements AutoCloseable { - - /** - * Sets the context class loader to the given ClassLoader and returns a resource that sets it - * back to the current context ClassLoader when the resource is closed. - * - *
{@code
-     * try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(classloader)) {
-     *     // code that needs the context class loader
-     * }
-     * }
- */ - public static TemporaryClassLoaderContext of(ClassLoader cl) { - final Thread t = Thread.currentThread(); - final ClassLoader original = t.getContextClassLoader(); - - t.setContextClassLoader(cl); - - return new TemporaryClassLoaderContext(t, original); - } - - private final Thread thread; - - private final ClassLoader originalContextClassLoader; - - private TemporaryClassLoaderContext(Thread thread, ClassLoader originalContextClassLoader) { - this.thread = thread; - this.originalContextClassLoader = originalContextClassLoader; - } - - @Override - public void close() { - thread.setContextClassLoader(originalContextClassLoader); - } -} diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java index f6908b989d2..d60fb46c8c9 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/excecutor/AssertExecutor.java @@ -36,7 +36,6 @@ import java.io.IOException; import java.math.BigDecimal; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -308,13 +307,6 @@ private Boolean checkType(Object value, SeaTunnelDataType fieldType) { return checkDecimalType(value, fieldType); } - if (fieldType.getSqlType() == SqlType.FLOAT_VECTOR - || fieldType.getSqlType() == SqlType.FLOAT16_VECTOR - || fieldType.getSqlType() == SqlType.BFLOAT16_VECTOR - || fieldType.getSqlType() == SqlType.BINARY_VECTOR) { - return value instanceof ByteBuffer; - } - return value.getClass().equals(fieldType.getTypeClass()); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml deleted file mode 100644 index 098c60370d4..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/pom.xml +++ /dev/null @@ -1,91 +0,0 @@ - - - - 4.0.0 - - org.apache.seatunnel - connector-cdc - ${revision} - - - connector-cdc-opengauss - SeaTunnel : Connectors V2 : CDC : Opengauss - - - 5.1.0 - - - - - org.opengauss - opengauss-jdbc - ${opengauss.version} - - - - org.apache.seatunnel - connector-cdc-postgres - ${project.version} - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - - shade - - package - - false - true - false - false - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - org.postgresql - ${seatunnel.shade.package}.org.postgresql - - - - - - - - - - diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java deleted file mode 100644 index 57c393acfaf..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java +++ /dev/null @@ -1,815 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.debezium.connector.postgresql.connection; - -import org.apache.kafka.connect.errors.ConnectException; - -import org.postgresql.core.BaseConnection; -import org.postgresql.jdbc.PgConnection; -import org.postgresql.jdbc.TimestampUtils; -import org.postgresql.replication.LogSequenceNumber; -import org.postgresql.util.PGmoney; -import org.postgresql.util.PSQLState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.debezium.DebeziumException; -import io.debezium.annotation.VisibleForTesting; -import io.debezium.config.Configuration; -import io.debezium.connector.postgresql.PgOid; -import io.debezium.connector.postgresql.PostgresConnectorConfig; -import io.debezium.connector.postgresql.PostgresSchema; -import io.debezium.connector.postgresql.PostgresType; -import io.debezium.connector.postgresql.PostgresValueConverter; -import io.debezium.connector.postgresql.TypeRegistry; -import io.debezium.connector.postgresql.spi.SlotState; -import io.debezium.data.SpecialValueDecimal; -import io.debezium.jdbc.JdbcConfiguration; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Column; -import io.debezium.relational.ColumnEditor; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; -import io.debezium.schema.DatabaseSchema; -import io.debezium.util.Clock; -import io.debezium.util.Metronome; - -import java.nio.charset.Charset; -import java.sql.ResultSet; -import java.sql.ResultSetMetaData; -import java.sql.SQLException; -import java.sql.Statement; -import java.time.Duration; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Copied from Debezium 1.9.8.Final. {@link JdbcConnection} connection extension used for connecting - * to Postgres instances. - * - *

Line 616 : skip validateServerVersion because the version based pg of opengauss is below 9.4 - */ -public class PostgresConnection extends JdbcConnection { - - public static final String CONNECTION_STREAMING = "Debezium Streaming"; - public static final String CONNECTION_SLOT_INFO = "Debezium Slot Info"; - public static final String CONNECTION_DROP_SLOT = "Debezium Drop Slot"; - public static final String CONNECTION_VALIDATE_CONNECTION = "Debezium Validate Connection"; - public static final String CONNECTION_HEARTBEAT = "Debezium Heartbeat"; - public static final String CONNECTION_GENERAL = "Debezium General"; - - private static Logger LOGGER = LoggerFactory.getLogger(PostgresConnection.class); - - private static final String URL_PATTERN = - "jdbc:postgresql://${" - + JdbcConfiguration.HOSTNAME - + "}:${" - + JdbcConfiguration.PORT - + "}/${" - + JdbcConfiguration.DATABASE - + "}"; - protected static final ConnectionFactory FACTORY = - JdbcConnection.patternBasedFactory( - URL_PATTERN, - org.postgresql.Driver.class.getName(), - PostgresConnection.class.getClassLoader(), - JdbcConfiguration.PORT.withDefault( - PostgresConnectorConfig.PORT.defaultValueAsString())); - - /** - * Obtaining a replication slot may fail if there's a pending transaction. We're retrying to get - * a slot for 30 min. - */ - private static final int MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT = 900; - - private static final Duration PAUSE_BETWEEN_REPLICATION_SLOT_RETRIEVAL_ATTEMPTS = - Duration.ofSeconds(2); - - private final TypeRegistry typeRegistry; - private final PostgresDefaultValueConverter defaultValueConverter; - - /** - * Creates a Postgres connection using the supplied configuration. If necessary this connection - * is able to resolve data type mappings. Such a connection requires a {@link - * PostgresValueConverter}, and will provide its own {@link TypeRegistry}. Usually only one such - * connection per connector is needed. - * - * @param config {@link Configuration} instance, may not be null. - * @param valueConverterBuilder supplies a configured {@link PostgresValueConverter} for a given - * {@link TypeRegistry} - * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools - */ - public PostgresConnection( - JdbcConfiguration config, - PostgresValueConverterBuilder valueConverterBuilder, - String connectionUsage) { - super( - addDefaultSettings(config, connectionUsage), - FACTORY, - PostgresConnection::validateServerVersion, - null, - "\"", - "\""); - - if (Objects.isNull(valueConverterBuilder)) { - this.typeRegistry = null; - this.defaultValueConverter = null; - } else { - this.typeRegistry = new TypeRegistry(this); - - final PostgresValueConverter valueConverter = - valueConverterBuilder.build(this.typeRegistry); - this.defaultValueConverter = - new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils()); - } - } - - /** - * Create a Postgres connection using the supplied configuration and {@link TypeRegistry} - * - * @param config {@link Configuration} instance, may not be null. - * @param typeRegistry an existing/already-primed {@link TypeRegistry} instance - * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools - */ - public PostgresConnection( - PostgresConnectorConfig config, TypeRegistry typeRegistry, String connectionUsage) { - super( - addDefaultSettings(config.getJdbcConfig(), connectionUsage), - FACTORY, - PostgresConnection::validateServerVersion, - null, - "\"", - "\""); - if (Objects.isNull(typeRegistry)) { - this.typeRegistry = null; - this.defaultValueConverter = null; - } else { - this.typeRegistry = typeRegistry; - final PostgresValueConverter valueConverter = - PostgresValueConverter.of(config, this.getDatabaseCharset(), typeRegistry); - this.defaultValueConverter = - new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils()); - } - } - - /** - * Creates a Postgres connection using the supplied configuration. The connector is the regular - * one without datatype resolution capabilities. - * - * @param config {@link Configuration} instance, may not be null. - * @param connectionUsage a symbolic name of the connection to be tracked in monitoring tools - */ - public PostgresConnection(JdbcConfiguration config, String connectionUsage) { - this(config, null, connectionUsage); - } - - static JdbcConfiguration addDefaultSettings( - JdbcConfiguration configuration, String connectionUsage) { - // we require Postgres 9.4 as the minimum server version since that's where logical - // replication was first introduced - return JdbcConfiguration.adapt( - configuration - .edit() - .with("assumeMinServerVersion", "9.4") - .with("ApplicationName", connectionUsage) - .build()); - } - - /** - * Returns a JDBC connection string for the current configuration. - * - * @return a {@code String} where the variables in {@code urlPattern} are replaced with values - * from the configuration - */ - public String connectionString() { - return connectionString(URL_PATTERN); - } - - /** - * Prints out information about the REPLICA IDENTITY status of a table. This in turn determines - * how much information is available for UPDATE and DELETE operations for logical replication. - * - * @param tableId the identifier of the table - * @return the replica identity information; never null - * @throws SQLException if there is a problem obtaining the replica identity information for the - * given table - */ - public ServerInfo.ReplicaIdentity readReplicaIdentityInfo(TableId tableId) throws SQLException { - String statement = - "SELECT relreplident FROM pg_catalog.pg_class c " - + "LEFT JOIN pg_catalog.pg_namespace n ON c.relnamespace=n.oid " - + "WHERE n.nspname=? and c.relname=?"; - String schema = - tableId.schema() != null && tableId.schema().length() > 0 - ? tableId.schema() - : "public"; - StringBuilder replIdentity = new StringBuilder(); - prepareQuery( - statement, - stmt -> { - stmt.setString(1, schema); - stmt.setString(2, tableId.table()); - }, - rs -> { - if (rs.next()) { - replIdentity.append(rs.getString(1)); - } else { - LOGGER.warn( - "Cannot determine REPLICA IDENTITY information for table '{}'", - tableId); - } - }); - return ServerInfo.ReplicaIdentity.parseFromDB(replIdentity.toString()); - } - - /** - * Returns the current state of the replication slot - * - * @param slotName the name of the slot - * @param pluginName the name of the plugin used for the desired slot - * @return the {@link SlotState} or null, if no slot state is found - * @throws SQLException - */ - public SlotState getReplicationSlotState(String slotName, String pluginName) - throws SQLException { - ServerInfo.ReplicationSlot slot; - try { - slot = readReplicationSlotInfo(slotName, pluginName); - if (slot.equals(ServerInfo.ReplicationSlot.INVALID)) { - return null; - } else { - return slot.asSlotState(); - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ConnectException( - "Interrupted while waiting for valid replication slot info", e); - } - } - - /** - * Fetches the state of a replication stage given a slot name and plugin name - * - * @param slotName the name of the slot - * @param pluginName the name of the plugin used for the desired slot - * @return the {@link ServerInfo.ReplicationSlot} object or a {@link - * ServerInfo.ReplicationSlot#INVALID} if the slot is not valid - * @throws SQLException is thrown by the underlying JDBC - */ - private ServerInfo.ReplicationSlot fetchReplicationSlotInfo(String slotName, String pluginName) - throws SQLException { - final String database = database(); - final ServerInfo.ReplicationSlot slot = - queryForSlot( - slotName, - database, - pluginName, - rs -> { - if (rs.next()) { - boolean active = rs.getBoolean("active"); - final Lsn confirmedFlushedLsn = - parseConfirmedFlushLsn(slotName, pluginName, database, rs); - if (confirmedFlushedLsn == null) { - return null; - } - Lsn restartLsn = - parseRestartLsn(slotName, pluginName, database, rs); - if (restartLsn == null) { - return null; - } - final Long xmin = rs.getLong("catalog_xmin"); - return new ServerInfo.ReplicationSlot( - active, confirmedFlushedLsn, restartLsn, xmin); - } else { - LOGGER.debug( - "No replication slot '{}' is present for plugin '{}' and database '{}'", - slotName, - pluginName, - database); - return ServerInfo.ReplicationSlot.INVALID; - } - }); - return slot; - } - - /** - * Fetches a replication slot, repeating the query until either the slot is created or until the - * max number of attempts has been reached - * - *

To fetch the slot without the retries, use the {@link - * PostgresConnection#fetchReplicationSlotInfo} call - * - * @param slotName the slot name - * @param pluginName the name of the plugin - * @return the {@link ServerInfo.ReplicationSlot} object or a {@link - * ServerInfo.ReplicationSlot#INVALID} if the slot is not valid - * @throws SQLException is thrown by the underyling jdbc driver - * @throws InterruptedException is thrown if we don't return an answer within the set number of - * retries - */ - @VisibleForTesting - ServerInfo.ReplicationSlot readReplicationSlotInfo(String slotName, String pluginName) - throws SQLException, InterruptedException { - final String database = database(); - final Metronome metronome = - Metronome.parker(PAUSE_BETWEEN_REPLICATION_SLOT_RETRIEVAL_ATTEMPTS, Clock.SYSTEM); - - for (int attempt = 1; attempt <= MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT; attempt++) { - final ServerInfo.ReplicationSlot slot = fetchReplicationSlotInfo(slotName, pluginName); - if (slot != null) { - LOGGER.info("Obtained valid replication slot {}", slot); - return slot; - } - LOGGER.warn( - "Cannot obtain valid replication slot '{}' for plugin '{}' and database '{}' [during attempt {} out of {}, concurrent tx probably blocks taking snapshot.", - slotName, - pluginName, - database, - attempt, - MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT); - metronome.pause(); - } - - throw new ConnectException( - "Unable to obtain valid replication slot. " - + "Make sure there are no long-running transactions running in parallel as they may hinder the allocation of the replication slot when starting this connector"); - } - - protected ServerInfo.ReplicationSlot queryForSlot( - String slotName, - String database, - String pluginName, - ResultSetMapper map) - throws SQLException { - return prepareQueryAndMap( - "select * from pg_replication_slots where slot_name = ? and database = ? and plugin = ?", - statement -> { - statement.setString(1, slotName); - statement.setString(2, database); - statement.setString(3, pluginName); - }, - map); - } - - /** - * Obtains the LSN to resume streaming from. On PG 9.5 there is no confirmed_flushed_lsn yet, so - * restart_lsn will be read instead. This may result in more records to be re-read after a - * restart. - */ - private Lsn parseConfirmedFlushLsn( - String slotName, String pluginName, String database, ResultSet rs) { - Lsn confirmedFlushedLsn = null; - - try { - confirmedFlushedLsn = - tryParseLsn(slotName, pluginName, database, rs, "confirmed_flush_lsn"); - } catch (SQLException e) { - LOGGER.info("unable to find confirmed_flushed_lsn, falling back to restart_lsn"); - try { - confirmedFlushedLsn = - tryParseLsn(slotName, pluginName, database, rs, "restart_lsn"); - } catch (SQLException e2) { - throw new ConnectException( - "Neither confirmed_flush_lsn nor restart_lsn could be found"); - } - } - - return confirmedFlushedLsn; - } - - private Lsn parseRestartLsn(String slotName, String pluginName, String database, ResultSet rs) { - Lsn restartLsn = null; - try { - restartLsn = tryParseLsn(slotName, pluginName, database, rs, "restart_lsn"); - } catch (SQLException e) { - throw new ConnectException("restart_lsn could be found"); - } - - return restartLsn; - } - - private Lsn tryParseLsn( - String slotName, String pluginName, String database, ResultSet rs, String column) - throws ConnectException, SQLException { - Lsn lsn = null; - - String lsnStr = rs.getString(column); - if (lsnStr == null) { - return null; - } - try { - lsn = Lsn.valueOf(lsnStr); - } catch (Exception e) { - throw new ConnectException( - "Value " - + column - + " in the pg_replication_slots table for slot = '" - + slotName - + "', plugin = '" - + pluginName - + "', database = '" - + database - + "' is not valid. This is an abnormal situation and the database status should be checked."); - } - if (!lsn.isValid()) { - throw new ConnectException("Invalid LSN returned from database"); - } - return lsn; - } - - /** - * Drops a replication slot that was created on the DB - * - * @param slotName the name of the replication slot, may not be null - * @return {@code true} if the slot was dropped, {@code false} otherwise - */ - public boolean dropReplicationSlot(String slotName) { - final int ATTEMPTS = 3; - for (int i = 0; i < ATTEMPTS; i++) { - try { - execute("select pg_drop_replication_slot('" + slotName + "')"); - return true; - } catch (SQLException e) { - // slot is active - if (PSQLState.OBJECT_IN_USE.getState().equals(e.getSQLState())) { - if (i < ATTEMPTS - 1) { - LOGGER.debug( - "Cannot drop replication slot '{}' because it's still in use", - slotName); - } else { - LOGGER.warn( - "Cannot drop replication slot '{}' because it's still in use", - slotName); - return false; - } - } else if (PSQLState.UNDEFINED_OBJECT.getState().equals(e.getSQLState())) { - LOGGER.debug("Replication slot {} has already been dropped", slotName); - return false; - } else { - LOGGER.error("Unexpected error while attempting to drop replication slot", e); - return false; - } - } - try { - Metronome.parker(Duration.ofSeconds(1), Clock.system()).pause(); - } catch (InterruptedException e) { - } - } - return false; - } - - /** - * Drops the debezium publication that was created. - * - * @param publicationName the publication name, may not be null - * @return {@code true} if the publication was dropped, {@code false} otherwise - */ - public boolean dropPublication(String publicationName) { - try { - LOGGER.debug("Dropping publication '{}'", publicationName); - execute("DROP PUBLICATION " + publicationName); - return true; - } catch (SQLException e) { - if (PSQLState.UNDEFINED_OBJECT.getState().equals(e.getSQLState())) { - LOGGER.debug("Publication {} has already been dropped", publicationName); - } else { - LOGGER.error("Unexpected error while attempting to drop publication", e); - } - return false; - } - } - - @Override - public synchronized void close() { - try { - super.close(); - } catch (SQLException e) { - LOGGER.error("Unexpected error while closing Postgres connection", e); - } - } - - /** - * Returns the PG id of the current active transaction - * - * @return a PG transaction identifier, or null if no tx is active - * @throws SQLException if anything fails. - */ - public Long currentTransactionId() throws SQLException { - AtomicLong txId = new AtomicLong(0); - query( - "select (case pg_is_in_recovery() when 't' then 0 else txid_current() end) AS pg_current_txid", - rs -> { - if (rs.next()) { - txId.compareAndSet(0, rs.getLong(1)); - } - }); - long value = txId.get(); - return value > 0 ? value : null; - } - - /** - * Returns the current position in the server tx log. - * - * @return a long value, never negative - * @throws SQLException if anything unexpected fails. - */ - public long currentXLogLocation() throws SQLException { - AtomicLong result = new AtomicLong(0); - int majorVersion = connection().getMetaData().getDatabaseMajorVersion(); - query( - majorVersion >= 10 - ? "select (case pg_is_in_recovery() when 't' then pg_last_wal_receive_lsn() else pg_current_wal_lsn() end) AS pg_current_wal_lsn" - : "select * from pg_current_xlog_location()", - rs -> { - if (!rs.next()) { - throw new IllegalStateException( - "there should always be a valid xlog position"); - } - result.compareAndSet(0, LogSequenceNumber.valueOf(rs.getString(1)).asLong()); - }); - return result.get(); - } - - /** - * Returns information about the PG server to which this instance is connected. - * - * @return a {@link ServerInfo} instance, never {@code null} - * @throws SQLException if anything fails - */ - public ServerInfo serverInfo() throws SQLException { - ServerInfo serverInfo = new ServerInfo(); - query( - "SELECT version(), current_user, current_database()", - rs -> { - if (rs.next()) { - serverInfo - .withServer(rs.getString(1)) - .withUsername(rs.getString(2)) - .withDatabase(rs.getString(3)); - } - }); - String username = serverInfo.username(); - if (username != null) { - query( - "SELECT oid, rolname, rolsuper, rolinherit, rolcreaterole, rolcreatedb, rolcanlogin, rolreplication FROM pg_roles " - + "WHERE pg_has_role('" - + username - + "', oid, 'member')", - rs -> { - while (rs.next()) { - String roleInfo = - "superuser: " - + rs.getBoolean(3) - + ", replication: " - + rs.getBoolean(8) - + ", inherit: " - + rs.getBoolean(4) - + ", create role: " - + rs.getBoolean(5) - + ", create db: " - + rs.getBoolean(6) - + ", can log in: " - + rs.getBoolean(7); - String roleName = rs.getString(2); - serverInfo.addRole(roleName, roleInfo); - } - }); - } - return serverInfo; - } - - public Charset getDatabaseCharset() { - try { - return Charset.forName(((BaseConnection) connection()).getEncoding().name()); - } catch (SQLException e) { - throw new DebeziumException("Couldn't obtain encoding for database " + database(), e); - } - } - - public TimestampUtils getTimestampUtils() { - try { - return ((PgConnection) this.connection()).getTimestampUtils(); - } catch (SQLException e) { - throw new DebeziumException( - "Couldn't get timestamp utils from underlying connection", e); - } - } - - private static void validateServerVersion(Statement statement) throws SQLException {} - - @Override - public String quotedColumnIdString(String columnName) { - if (columnName.contains("\"")) { - columnName = columnName.replaceAll("\"", "\"\""); - } - - return super.quotedColumnIdString(columnName); - } - - @Override - protected int resolveNativeType(String typeName) { - return getTypeRegistry().get(typeName).getRootType().getOid(); - } - - @Override - protected int resolveJdbcType(int metadataJdbcType, int nativeType) { - // Special care needs to be taken for columns that use user-defined domain type data types - // where resolution of the column's JDBC type needs to be that of the root type instead of - // the actual column to properly influence schema building and value conversion. - return getTypeRegistry().get(nativeType).getRootType().getJdbcId(); - } - - @Override - protected Optional readTableColumn( - ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnFilter) - throws SQLException { - return doReadTableColumn(columnMetadata, tableId, columnFilter); - } - - public Optional readColumnForDecoder( - ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnNameFilter) - throws SQLException { - return doReadTableColumn(columnMetadata, tableId, columnNameFilter) - .map(ColumnEditor::create); - } - - private Optional doReadTableColumn( - ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnFilter) - throws SQLException { - final String columnName = columnMetadata.getString(4); - if (columnFilter == null - || columnFilter.matches( - tableId.catalog(), tableId.schema(), tableId.table(), columnName)) { - final ColumnEditor column = Column.editor().name(columnName); - column.type(columnMetadata.getString(6)); - - // first source the length/scale from the column metadata provided by the driver - // this may be overridden below if the column type is a user-defined domain type - column.length(columnMetadata.getInt(7)); - if (columnMetadata.getObject(9) != null) { - column.scale(columnMetadata.getInt(9)); - } - - column.optional(isNullable(columnMetadata.getInt(11))); - column.position(columnMetadata.getInt(17)); - column.autoIncremented("YES".equalsIgnoreCase(columnMetadata.getString(23))); - - String autogenerated = null; - try { - autogenerated = columnMetadata.getString(24); - } catch (SQLException e) { - // ignore, some drivers don't have this index - e.g. Postgres - } - column.generated("YES".equalsIgnoreCase(autogenerated)); - - // Lookup the column type from the TypeRegistry - // For all types, we need to set the Native and Jdbc types by using the root-type - final PostgresType nativeType = getTypeRegistry().get(column.typeName()); - column.nativeType(nativeType.getRootType().getOid()); - column.jdbcType(nativeType.getRootType().getJdbcId()); - - // For domain types, the postgres driver is unable to traverse a nested unbounded - // hierarchy of types and report the right length/scale of a given type. We use - // the TypeRegistry to accomplish this since it is capable of traversing the type - // hierarchy upward to resolve length/scale regardless of hierarchy depth. - if (TypeRegistry.DOMAIN_TYPE == nativeType.getJdbcId()) { - column.length(nativeType.getDefaultLength()); - column.scale(nativeType.getDefaultScale()); - } - - final String defaultValueExpression = columnMetadata.getString(13); - if (defaultValueExpression != null - && getDefaultValueConverter().supportConversion(column.typeName())) { - column.defaultValueExpression(defaultValueExpression); - } - - return Optional.of(column); - } - - return Optional.empty(); - } - - public PostgresDefaultValueConverter getDefaultValueConverter() { - Objects.requireNonNull( - defaultValueConverter, "Connection does not provide default value converter"); - return defaultValueConverter; - } - - public TypeRegistry getTypeRegistry() { - Objects.requireNonNull(typeRegistry, "Connection does not provide type registry"); - return typeRegistry; - } - - @Override - public > Object getColumnValue( - ResultSet rs, int columnIndex, Column column, Table table, T schema) - throws SQLException { - try { - final ResultSetMetaData metaData = rs.getMetaData(); - final String columnTypeName = metaData.getColumnTypeName(columnIndex); - final PostgresType type = - ((PostgresSchema) schema).getTypeRegistry().get(columnTypeName); - - LOGGER.trace("Type of incoming data is: {}", type.getOid()); - LOGGER.trace("ColumnTypeName is: {}", columnTypeName); - LOGGER.trace("Type is: {}", type); - - if (type.isArrayType()) { - return rs.getArray(columnIndex); - } - - switch (type.getOid()) { - case PgOid.MONEY: - // TODO author=Horia Chiorean date=14/11/2016 description=workaround for - // https://github.com/pgjdbc/pgjdbc/issues/100 - final String sMoney = rs.getString(columnIndex); - if (sMoney == null) { - return sMoney; - } - if (sMoney.startsWith("-")) { - // PGmoney expects negative values to be provided in the format of - // "($XXXXX.YY)" - final String negativeMoney = "(" + sMoney.substring(1) + ")"; - return new PGmoney(negativeMoney).val; - } - return new PGmoney(sMoney).val; - case PgOid.BIT: - return rs.getString(columnIndex); - case PgOid.NUMERIC: - final String s = rs.getString(columnIndex); - if (s == null) { - return s; - } - - Optional value = PostgresValueConverter.toSpecialValue(s); - return value.isPresent() - ? value.get() - : new SpecialValueDecimal(rs.getBigDecimal(columnIndex)); - case PgOid.TIME: - // To handle time 24:00:00 supported by TIME columns, read the column as a - // string. - case PgOid.TIMETZ: - // In order to guarantee that we resolve TIMETZ columns with proper microsecond - // precision, - // read the column as a string instead and then re-parse inside the converter. - return rs.getString(columnIndex); - default: - Object x = rs.getObject(columnIndex); - if (x != null) { - LOGGER.trace( - "rs getobject returns class: {}; rs getObject value is: {}", - x.getClass(), - x); - } - return x; - } - } catch (SQLException e) { - // not a known type - return super.getColumnValue(rs, columnIndex, column, table, schema); - } - } - - @Override - protected String[] supportedTableTypes() { - return new String[] {"VIEW", "MATERIALIZED VIEW", "TABLE", "PARTITIONED TABLE"}; - } - - @Override - protected boolean isTableType(String tableType) { - return "TABLE".equals(tableType) || "PARTITIONED TABLE".equals(tableType); - } - - /** - * Retrieves all {@code TableId}s in a given database catalog, including partitioned tables. - * - * @param catalogName the catalog/database name - * @return set of all table ids for existing table objects - * @throws SQLException if a database exception occurred - */ - public Set getAllTableIds(String catalogName) throws SQLException { - return readTableNames(catalogName, null, null, new String[] {"TABLE", "PARTITIONED TABLE"}); - } - - @FunctionalInterface - public interface PostgresValueConverterBuilder { - PostgresValueConverter build(TypeRegistry registry); - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java deleted file mode 100644 index c69a63c7136..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java +++ /dev/null @@ -1,928 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.debezium.connector.postgresql.connection; - -import org.apache.kafka.connect.errors.ConnectException; - -import org.postgresql.core.BaseConnection; -import org.postgresql.core.ServerVersion; -import org.postgresql.replication.PGReplicationStream; -import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder; -import org.postgresql.util.PSQLException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.debezium.DebeziumException; -import io.debezium.connector.postgresql.PostgresConnectorConfig; -import io.debezium.connector.postgresql.PostgresSchema; -import io.debezium.connector.postgresql.TypeRegistry; -import io.debezium.connector.postgresql.spi.SlotCreationResult; -import io.debezium.jdbc.JdbcConfiguration; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.jdbc.JdbcConnectionException; -import io.debezium.relational.RelationalTableFilters; -import io.debezium.relational.TableId; -import io.debezium.util.Clock; -import io.debezium.util.Metronome; - -import java.nio.ByteBuffer; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.SQLWarning; -import java.sql.Statement; -import java.time.Duration; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.BiFunction; -import java.util.function.Function; -import java.util.stream.Collectors; - -import static java.lang.Math.toIntExact; - -/** - * Copied from Debezium 1.9.8.Final. Implementation of a {@link ReplicationConnection} for - * Postgresql. Note that replication connections in PG cannot execute regular statements but only a - * limited number of replication-related commands. - * - *

Line 179 : Modify the method named initPublication which we use the regular - i.e. not a - * replication - connection to avoid the I/O error - * - *

Line 440: Modify the method named createReplicationSlot which add logical that create the slot - * if it doesn't exist - */ -public class PostgresReplicationConnection extends JdbcConnection implements ReplicationConnection { - - private static Logger LOGGER = LoggerFactory.getLogger(PostgresReplicationConnection.class); - - private final String slotName; - private final String publicationName; - private final RelationalTableFilters tableFilter; - private final PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode; - private final PostgresConnectorConfig.LogicalDecoder plugin; - private final boolean dropSlotOnClose; - private final PostgresConnectorConfig connectorConfig; - private final Duration statusUpdateInterval; - private final MessageDecoder messageDecoder; - private final PostgresConnection jdbcConnection; - private final TypeRegistry typeRegistry; - private final Properties streamParams; - - private Lsn defaultStartingPos; - private SlotCreationResult slotCreationInfo; - private boolean hasInitedSlot; - - /** - * Creates a new replication connection with the given params. - * - * @param config the JDBC configuration for the connection; may not be null - * @param slotName the name of the DB slot for logical replication; may not be null - * @param publicationName the name of the DB publication for logical replication; may not be - * null - * @param tableFilter the tables to watch of the DB publication for logical replication; may not - * be null - * @param publicationAutocreateMode the mode for publication autocreation; may not be null - * @param plugin decoder matching the server side plug-in used for streaming changes; may not be - * null - * @param dropSlotOnClose whether the replication slot should be dropped once the connection is - * closed - * @param statusUpdateInterval the interval at which the replication connection should - * periodically send status - * @param doSnapshot whether the connector is doing snapshot - * @param jdbcConnection general PostgreSQL JDBC connection - * @param typeRegistry registry with PostgreSQL types - * @param streamParams additional parameters to pass to the replication stream - * @param schema the schema; must not be null - *

updates to the server - */ - private PostgresReplicationConnection( - PostgresConnectorConfig config, - String slotName, - String publicationName, - RelationalTableFilters tableFilter, - PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode, - PostgresConnectorConfig.LogicalDecoder plugin, - boolean dropSlotOnClose, - boolean doSnapshot, - Duration statusUpdateInterval, - PostgresConnection jdbcConnection, - TypeRegistry typeRegistry, - Properties streamParams, - PostgresSchema schema) { - super( - addDefaultSettings(config.getJdbcConfig()), - PostgresConnection.FACTORY, - null, - null, - "\"", - "\""); - - this.connectorConfig = config; - this.slotName = slotName; - this.publicationName = publicationName; - this.tableFilter = tableFilter; - this.publicationAutocreateMode = publicationAutocreateMode; - this.plugin = plugin; - this.dropSlotOnClose = dropSlotOnClose; - this.statusUpdateInterval = statusUpdateInterval; - this.messageDecoder = - plugin.messageDecoder(new MessageDecoderContext(config, schema), jdbcConnection); - this.jdbcConnection = jdbcConnection; - this.typeRegistry = typeRegistry; - this.streamParams = streamParams; - this.slotCreationInfo = null; - this.hasInitedSlot = false; - } - - private static JdbcConfiguration addDefaultSettings(JdbcConfiguration configuration) { - // first copy the parent's default settings... - // then set some additional replication specific settings - return JdbcConfiguration.adapt( - PostgresConnection.addDefaultSettings( - configuration, PostgresConnection.CONNECTION_STREAMING) - .edit() - .with("replication", "database") - .with( - "preferQueryMode", - "simple") // replication protocol only supports simple query mode - .build()); - } - - private ServerInfo.ReplicationSlot getSlotInfo() throws SQLException, InterruptedException { - try (PostgresConnection connection = - new PostgresConnection( - connectorConfig.getJdbcConfig(), PostgresConnection.CONNECTION_SLOT_INFO)) { - return connection.readReplicationSlotInfo(slotName, plugin.getPostgresPluginName()); - } - } - - protected void initPublication() { - String tableFilterString = null; - if (PostgresConnectorConfig.LogicalDecoder.PGOUTPUT.equals(plugin)) { - LOGGER.info("Initializing PgOutput logical decoder publication"); - try { - PostgresConnection conn = jdbcConnection; - // Unless the autocommit is disabled the SELECT publication query will stay running - conn.setAutoCommit(false); - - String selectPublication = - String.format( - "SELECT COUNT(1) FROM pg_publication WHERE pubname = '%s'", - publicationName); - conn.query( - selectPublication, - rs -> { - if (rs.next()) { - Long count = rs.getLong(1); - // Close eagerly as the transaction might stay running - if (count == 0L) { - LOGGER.info( - "Creating new publication '{}' for plugin '{}'", - publicationName, - plugin); - switch (publicationAutocreateMode) { - case DISABLED: - throw new ConnectException( - "Publication autocreation is disabled, please create one and restart the connector."); - case ALL_TABLES: - String createPublicationStmt = - String.format( - "CREATE PUBLICATION %s FOR ALL TABLES;", - publicationName); - LOGGER.info( - "Creating Publication with statement '{}'", - createPublicationStmt); - // Publication doesn't exist, create it. - conn.executeWithoutCommitting(createPublicationStmt); - break; - case FILTERED: - createOrUpdatePublicationModeFilterted( - tableFilterString, conn, false); - break; - } - } else { - switch (publicationAutocreateMode) { - case FILTERED: - createOrUpdatePublicationModeFilterted( - tableFilterString, conn, true); - break; - default: - LOGGER.trace( - "A logical publication named '{}' for plugin '{}' and database '{}' is already active on the server " - + "and will be used by the plugin", - publicationName, - plugin, - database()); - } - } - } - }); - conn.commit(); - conn.setAutoCommit(true); - } catch (SQLException e) { - throw new JdbcConnectionException(e); - } - } - } - - private void createOrUpdatePublicationModeFilterted( - String tableFilterString, PostgresConnection conn, boolean isUpdate) { - String createOrUpdatePublicationStmt; - try { - Set tablesToCapture = determineCapturedTables(); - tableFilterString = - tablesToCapture.stream() - .map(TableId::toDoubleQuotedString) - .collect(Collectors.joining(", ")); - if (tableFilterString.isEmpty()) { - throw new DebeziumException( - String.format( - "No table filters found for filtered publication %s", - publicationName)); - } - createOrUpdatePublicationStmt = - isUpdate - ? String.format( - "ALTER PUBLICATION %s SET TABLE %s;", - publicationName, tableFilterString) - : String.format( - "CREATE PUBLICATION %s FOR TABLE %s;", - publicationName, tableFilterString); - LOGGER.info( - isUpdate - ? "Updating Publication with statement '{}'" - : "Creating Publication with statement '{}'", - createOrUpdatePublicationStmt); - conn.execute(createOrUpdatePublicationStmt); - } catch (Exception e) { - throw new ConnectException( - String.format( - "Unable to %s filtered publication %s for %s", - isUpdate ? "update" : "create", publicationName, tableFilterString), - e); - } - } - - private Set determineCapturedTables() throws Exception { - Set allTableIds = jdbcConnection.getAllTableIds(connectorConfig.databaseName()); - - Set capturedTables = new HashSet<>(); - - for (TableId tableId : allTableIds) { - if (tableFilter.dataCollectionFilter().isIncluded(tableId)) { - LOGGER.trace("Adding table {} to the list of captured tables", tableId); - capturedTables.add(tableId); - } else { - LOGGER.trace( - "Ignoring table {} as it's not included in the filter configuration", - tableId); - } - } - - return capturedTables.stream() - .sorted() - .collect(Collectors.toCollection(LinkedHashSet::new)); - } - - protected void initReplicationSlot() throws SQLException, InterruptedException { - ServerInfo.ReplicationSlot slotInfo = getSlotInfo(); - - boolean shouldCreateSlot = ServerInfo.ReplicationSlot.INVALID == slotInfo; - try { - // there's no info for this plugin and slot so create a new slot - if (shouldCreateSlot) { - this.createReplicationSlot(); - } - - // replication connection does not support parsing of SQL statements so we need to - // create - // the connection without executing on connect statements - see JDBC opt - // preferQueryMode=simple - pgConnection(); - final String identifySystemStatement = "IDENTIFY_SYSTEM"; - LOGGER.debug( - "running '{}' to validate replication connection", identifySystemStatement); - final Lsn xlogStart = - queryAndMap( - identifySystemStatement, - rs -> { - if (!rs.next()) { - throw new IllegalStateException( - "The DB connection is not a valid replication connection"); - } - String xlogpos = rs.getString("xlogpos"); - LOGGER.debug("received latest xlogpos '{}'", xlogpos); - return Lsn.valueOf(xlogpos); - }); - - if (slotCreationInfo != null) { - this.defaultStartingPos = slotCreationInfo.startLsn(); - } else if (shouldCreateSlot || !slotInfo.hasValidFlushedLsn()) { - // this is a new slot or we weren't able to read a valid flush LSN pos, so we always - // start from the xlog pos that was reported - this.defaultStartingPos = xlogStart; - } else { - Lsn latestFlushedLsn = slotInfo.latestFlushedLsn(); - this.defaultStartingPos = - latestFlushedLsn.compareTo(xlogStart) < 0 ? latestFlushedLsn : xlogStart; - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("found previous flushed LSN '{}'", latestFlushedLsn); - } - } - hasInitedSlot = true; - } catch (SQLException e) { - throw new JdbcConnectionException(e); - } - } - - // Temporary replication slots is a new feature of PostgreSQL 10 - private boolean useTemporarySlot() throws SQLException { - // Temporary replication slots cannot be used due to connection restart - // when finding WAL position - // return dropSlotOnClose && pgConnection().haveMinimumServerVersion(ServerVersion.v10); - return false; - } - - /** - * creating a replication connection and starting to stream involves a few steps: 1. we create - * the connection and ensure that a. the slot exists b. the slot isn't currently being used 2. - * we query to get our potential start position in the slot (lsn) 3. we try and start streaming, - * depending on our options (such as in wal2json) this may fail, which can result in the - * connection being killed and we need to start the process over if we are using a temporary - * slot 4. actually start the streamer - * - *

This method takes care of all of these and this method queries for a default starting - * position If you know where you are starting from you should call {@link #startStreaming(Lsn, - * WalPositionLocator)}, this method delegates to that method - * - * @return - * @throws SQLException - * @throws InterruptedException - */ - @Override - public ReplicationStream startStreaming(WalPositionLocator walPosition) - throws SQLException, InterruptedException { - return startStreaming(null, walPosition); - } - - @Override - public ReplicationStream startStreaming(Lsn offset, WalPositionLocator walPosition) - throws SQLException, InterruptedException { - initConnection(); - - connect(); - if (offset == null || !offset.isValid()) { - offset = defaultStartingPos; - } - Lsn lsn = offset; - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("starting streaming from LSN '{}'", lsn); - } - - final int maxRetries = connectorConfig.maxRetries(); - final Duration delay = connectorConfig.retryDelay(); - int tryCount = 0; - while (true) { - try { - return createReplicationStream(lsn, walPosition); - } catch (Exception e) { - String message = "Failed to start replication stream at " + lsn; - if (++tryCount > maxRetries) { - if (e.getMessage().matches(".*replication slot .* is active.*")) { - message += - "; when setting up multiple connectors for the same database host, please make sure to use a distinct replication slot name for each."; - } - throw new DebeziumException(message, e); - } else { - LOGGER.warn( - message + ", waiting for {} ms and retrying, attempt number {} over {}", - delay, - tryCount, - maxRetries); - final Metronome metronome = Metronome.sleeper(delay, Clock.SYSTEM); - metronome.pause(); - } - } - } - } - - @Override - public void initConnection() throws SQLException, InterruptedException { - // See https://www.postgresql.org/docs/current/logical-replication-quick-setup.html - // For pgoutput specifically, the publication must be created before the slot. - initPublication(); - if (!hasInitedSlot) { - initReplicationSlot(); - } - } - - @Override - public Optional createReplicationSlot() throws SQLException { - // note that some of these options are only supported in Postgres 9.4+, additionally - // the options are not yet exported by the jdbc api wrapper, therefore, we just do - // this ourselves but eventually this should be moved back to the jdbc API - // see https://github.com/pgjdbc/pgjdbc/issues/1305 - ServerInfo.ReplicationSlot slotInfo; - try { - slotInfo = getSlotInfo(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - - boolean shouldCreateSlot = ServerInfo.ReplicationSlot.INVALID == slotInfo; - - if (shouldCreateSlot) { - LOGGER.debug("Creating new replication slot '{}' for plugin '{}'", slotName, plugin); - String tempPart = ""; - // Exported snapshots are supported in Postgres 9.4+ - boolean canExportSnapshot = pgConnection().haveMinimumServerVersion(ServerVersion.v9_4); - if ((dropSlotOnClose) && !canExportSnapshot) { - LOGGER.warn( - "A slot marked as temporary or with an exported snapshot was created, " - + "but not on a supported version of Postgres, ignoring!"); - } - if (useTemporarySlot()) { - tempPart = "TEMPORARY"; - } - - // See https://www.postgresql.org/docs/current/logical-replication-quick-setup.html - // For pgoutput specifically, the publication must be created prior to the slot. - initPublication(); - - try (Statement stmt = pgConnection().createStatement()) { - String createCommand = - String.format( - "CREATE_REPLICATION_SLOT \"%s\" %s LOGICAL %s", - slotName, tempPart, plugin.getPostgresPluginName()); - LOGGER.info("Creating replication slot with command {}", createCommand); - stmt.execute(createCommand); - // when we are in Postgres 9.4+, we can parse the slot creation info, - // otherwise, it returns nothing - if (canExportSnapshot) { - this.slotCreationInfo = parseSlotCreation(stmt.getResultSet()); - } - } - } - return Optional.ofNullable(slotCreationInfo); - } - - protected BaseConnection pgConnection() throws SQLException { - return (BaseConnection) connection(false); - } - - private SlotCreationResult parseSlotCreation(ResultSet rs) { - try { - if (rs.next()) { - String slotName = rs.getString("slot_name"); - String startPoint = rs.getString("consistent_point"); - String snapName = rs.getString("snapshot_name"); - String pluginName = rs.getString("output_plugin"); - - return new SlotCreationResult(slotName, startPoint, snapName, pluginName); - } else { - throw new ConnectException("No replication slot found"); - } - } catch (SQLException ex) { - throw new ConnectException("Unable to parse create_replication_slot response", ex); - } - } - - private ReplicationStream createReplicationStream( - final Lsn startLsn, WalPositionLocator walPosition) - throws SQLException, InterruptedException { - PGReplicationStream s; - - try { - try { - s = - startPgReplicationStream( - startLsn, - plugin.forceRds() - ? messageDecoder::optionsWithoutMetadata - : messageDecoder::optionsWithMetadata); - messageDecoder.setContainsMetadata(plugin.forceRds() ? false : true); - } catch (PSQLException e) { - LOGGER.debug( - "Could not register for streaming, retrying without optional options", e); - - // re-init the slot after a failed start of slot, as this - // may have closed the slot - if (useTemporarySlot()) { - initReplicationSlot(); - } - - s = - startPgReplicationStream( - startLsn, - plugin.forceRds() - ? messageDecoder::optionsWithoutMetadata - : messageDecoder::optionsWithMetadata); - messageDecoder.setContainsMetadata(plugin.forceRds() ? false : true); - } - } catch (PSQLException e) { - if (e.getMessage().matches("(?s)ERROR: option .* is unknown.*")) { - // It is possible we are connecting to an old wal2json plug-in - LOGGER.warn( - "Could not register for streaming with metadata in messages, falling back to messages without metadata"); - - // re-init the slot after a failed start of slot, as this - // may have closed the slot - if (useTemporarySlot()) { - initReplicationSlot(); - } - - s = startPgReplicationStream(startLsn, messageDecoder::optionsWithoutMetadata); - messageDecoder.setContainsMetadata(false); - } else if (e.getMessage() - .matches("(?s)ERROR: requested WAL segment .* has already been removed.*")) { - LOGGER.error("Cannot rewind to last processed WAL position", e); - throw new ConnectException( - "The offset to start reading from has been removed from the database write-ahead log. Create a new snapshot and consider setting of PostgreSQL parameter wal_keep_segments = 0."); - } else { - throw e; - } - } - - final PGReplicationStream stream = s; - - return new ReplicationStream() { - - private static final int CHECK_WARNINGS_AFTER_COUNT = 100; - private int warningCheckCounter = CHECK_WARNINGS_AFTER_COUNT; - private ExecutorService keepAliveExecutor = null; - private AtomicBoolean keepAliveRunning; - private final Metronome metronome = - Metronome.sleeper(statusUpdateInterval, Clock.SYSTEM); - - // make sure this is volatile since multiple threads may be interested in this value - private volatile Lsn lastReceivedLsn; - - @Override - public void read(ReplicationMessageProcessor processor) - throws SQLException, InterruptedException { - processWarnings(false); - ByteBuffer read = stream.read(); - final Lsn lastReceiveLsn = Lsn.valueOf(stream.getLastReceiveLSN()); - LOGGER.trace( - "Streaming requested from LSN {}, received LSN {}", - startLsn, - lastReceiveLsn); - if (messageDecoder.shouldMessageBeSkipped( - read, lastReceiveLsn, startLsn, walPosition)) { - return; - } - deserializeMessages(read, processor); - } - - @Override - public boolean readPending(ReplicationMessageProcessor processor) - throws SQLException, InterruptedException { - processWarnings(false); - ByteBuffer read = stream.readPending(); - final Lsn lastReceiveLsn = Lsn.valueOf(stream.getLastReceiveLSN()); - LOGGER.trace( - "Streaming requested from LSN {}, received LSN {}", - startLsn, - lastReceiveLsn); - - if (read == null) { - return false; - } - - if (messageDecoder.shouldMessageBeSkipped( - read, lastReceiveLsn, startLsn, walPosition)) { - return true; - } - - deserializeMessages(read, processor); - - return true; - } - - private void deserializeMessages( - ByteBuffer buffer, ReplicationMessageProcessor processor) - throws SQLException, InterruptedException { - lastReceivedLsn = Lsn.valueOf(stream.getLastReceiveLSN()); - LOGGER.trace("Received message at LSN {}", lastReceivedLsn); - messageDecoder.processMessage(buffer, processor, typeRegistry); - } - - @Override - public void close() throws SQLException { - processWarnings(true); - stream.close(); - } - - @Override - public void flushLsn(Lsn lsn) throws SQLException { - doFlushLsn(lsn); - } - - private void doFlushLsn(Lsn lsn) throws SQLException { - stream.setFlushedLSN(lsn.asLogSequenceNumber()); - stream.setAppliedLSN(lsn.asLogSequenceNumber()); - - stream.forceUpdateStatus(); - } - - @Override - public Lsn lastReceivedLsn() { - return lastReceivedLsn; - } - - @Override - public void startKeepAlive(ExecutorService service) { - if (keepAliveExecutor == null) { - keepAliveExecutor = service; - keepAliveRunning = new AtomicBoolean(true); - keepAliveExecutor.submit( - () -> { - while (keepAliveRunning.get()) { - try { - LOGGER.trace( - "Forcing status update with replication stream"); - stream.forceUpdateStatus(); - metronome.pause(); - } catch (Exception exp) { - throw new RuntimeException( - "received unexpected exception will perform keep alive", - exp); - } - } - }); - } - } - - @Override - public void stopKeepAlive() { - if (keepAliveExecutor != null) { - keepAliveRunning.set(false); - keepAliveExecutor.shutdownNow(); - keepAliveExecutor = null; - } - } - - private void processWarnings(final boolean forced) throws SQLException { - if (--warningCheckCounter == 0 || forced) { - warningCheckCounter = CHECK_WARNINGS_AFTER_COUNT; - for (SQLWarning w = connection().getWarnings(); - w != null; - w = w.getNextWarning()) { - LOGGER.debug( - "Server-side message: '{}', state = {}, code = {}", - w.getMessage(), - w.getSQLState(), - w.getErrorCode()); - } - connection().clearWarnings(); - } - } - - @Override - public Lsn startLsn() { - return startLsn; - } - }; - } - - private PGReplicationStream startPgReplicationStream( - final Lsn lsn, - BiFunction< - ChainedLogicalStreamBuilder, - Function, - ChainedLogicalStreamBuilder> - configurator) - throws SQLException { - assert lsn != null; - ChainedLogicalStreamBuilder streamBuilder = - pgConnection() - .getReplicationAPI() - .replicationStream() - .logical() - .withSlotName("\"" + slotName + "\"") - .withStartPosition(lsn.asLogSequenceNumber()) - .withSlotOptions(streamParams); - streamBuilder = configurator.apply(streamBuilder, this::hasMinimumVersion); - - if (statusUpdateInterval != null && statusUpdateInterval.toMillis() > 0) { - streamBuilder.withStatusInterval( - toIntExact(statusUpdateInterval.toMillis()), TimeUnit.MILLISECONDS); - } - - PGReplicationStream stream = streamBuilder.start(); - - // TODO DBZ-508 get rid of this - // Needed by tests when connections are opened and closed in a fast sequence - try { - Thread.sleep(10); - } catch (Exception e) { - } - stream.forceUpdateStatus(); - return stream; - } - - private Boolean hasMinimumVersion(int version) { - try { - return pgConnection().haveMinimumServerVersion(version); - } catch (SQLException e) { - throw new DebeziumException(e); - } - } - - @Override - public synchronized void close() { - close(true); - } - - public synchronized void close(boolean dropSlot) { - try { - LOGGER.debug("Closing message decoder"); - messageDecoder.close(); - } catch (Throwable e) { - LOGGER.error("Unexpected error while closing message decoder", e); - } - - try { - LOGGER.debug("Closing replication connection"); - super.close(); - } catch (Throwable e) { - LOGGER.error("Unexpected error while closing Postgres connection", e); - } - if (dropSlotOnClose && dropSlot) { - // we're dropping the replication slot via a regular - i.e. not a replication - - // connection - try (PostgresConnection connection = - new PostgresConnection( - connectorConfig.getJdbcConfig(), - PostgresConnection.CONNECTION_DROP_SLOT)) { - connection.dropReplicationSlot(slotName); - connection.dropPublication(publicationName); - } catch (Throwable e) { - LOGGER.error("Unexpected error while dropping replication slot", e); - } - } - } - - @Override - public void reconnect() throws SQLException { - close(false); - // Don't re-execute initial commands on reconnection - connection(false); - } - - protected static class ReplicationConnectionBuilder implements Builder { - - private final PostgresConnectorConfig config; - private String slotName = DEFAULT_SLOT_NAME; - private String publicationName = DEFAULT_PUBLICATION_NAME; - private RelationalTableFilters tableFilter; - private PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode = - PostgresConnectorConfig.AutoCreateMode.ALL_TABLES; - private PostgresConnectorConfig.LogicalDecoder plugin = - PostgresConnectorConfig.LogicalDecoder.DECODERBUFS; - private boolean dropSlotOnClose = DEFAULT_DROP_SLOT_ON_CLOSE; - private Duration statusUpdateIntervalVal; - private boolean doSnapshot; - private TypeRegistry typeRegistry; - private PostgresSchema schema; - private Properties slotStreamParams = new Properties(); - private PostgresConnection jdbcConnection; - - protected ReplicationConnectionBuilder(PostgresConnectorConfig config) { - assert config != null; - this.config = config; - } - - @Override - public ReplicationConnectionBuilder withSlot(final String slotName) { - assert slotName != null; - this.slotName = slotName; - return this; - } - - @Override - public Builder withPublication(String publicationName) { - assert publicationName != null; - this.publicationName = publicationName; - return this; - } - - @Override - public Builder withTableFilter(RelationalTableFilters tableFilter) { - assert tableFilter != null; - this.tableFilter = tableFilter; - return this; - } - - @Override - public Builder withPublicationAutocreateMode( - PostgresConnectorConfig.AutoCreateMode publicationAutocreateMode) { - assert publicationName != null; - this.publicationAutocreateMode = publicationAutocreateMode; - return this; - } - - @Override - public ReplicationConnectionBuilder withPlugin( - final PostgresConnectorConfig.LogicalDecoder plugin) { - assert plugin != null; - this.plugin = plugin; - return this; - } - - @Override - public ReplicationConnectionBuilder dropSlotOnClose(final boolean dropSlotOnClose) { - this.dropSlotOnClose = dropSlotOnClose; - return this; - } - - @Override - public ReplicationConnectionBuilder streamParams(final String slotStreamParams) { - if (slotStreamParams != null && !slotStreamParams.isEmpty()) { - this.slotStreamParams = new Properties(); - String[] paramsWithValues = slotStreamParams.split(";"); - for (String paramsWithValue : paramsWithValues) { - String[] paramAndValue = paramsWithValue.split("="); - if (paramAndValue.length == 2) { - this.slotStreamParams.setProperty(paramAndValue[0], paramAndValue[1]); - } else { - LOGGER.warn( - "The following STREAM_PARAMS value is invalid: {}", - paramsWithValue); - } - } - } - return this; - } - - @Override - public ReplicationConnectionBuilder statusUpdateInterval( - final Duration statusUpdateInterval) { - this.statusUpdateIntervalVal = statusUpdateInterval; - return this; - } - - @Override - public Builder doSnapshot(boolean doSnapshot) { - this.doSnapshot = doSnapshot; - return this; - } - - @Override - public Builder jdbcMetadataConnection(PostgresConnection jdbcConnection) { - this.jdbcConnection = jdbcConnection; - return this; - } - - @Override - public ReplicationConnection build() { - assert plugin != null : "Decoding plugin name is not set"; - return new PostgresReplicationConnection( - config, - slotName, - publicationName, - tableFilter, - publicationAutocreateMode, - plugin, - dropSlotOnClose, - doSnapshot, - statusUpdateIntervalVal, - jdbcConnection, - typeRegistry, - slotStreamParams, - schema); - } - - @Override - public Builder withTypeRegistry(TypeRegistry typeRegistry) { - this.typeRegistry = typeRegistry; - return this; - } - - @Override - public Builder withSchema(PostgresSchema schema) { - this.schema = schema; - return this; - } - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java deleted file mode 100644 index e9f552db6c0..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-opengauss/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/opengauss/OpengaussIncrementalSourceFactory.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.cdc.opengauss; - -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.table.catalog.CatalogOptions; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; -import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.connector.TableSource; -import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableSourceFactory; -import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceTableConfig; -import org.apache.seatunnel.connectors.cdc.base.option.JdbcSourceOptions; -import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; -import org.apache.seatunnel.connectors.cdc.base.utils.CatalogTableUtils; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option.PostgresOptions; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresIncrementalSource; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresSourceOptions; -import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; - -import com.google.auto.service.AutoService; - -import java.io.Serializable; -import java.util.List; -import java.util.Optional; - -@AutoService(Factory.class) -public class OpengaussIncrementalSourceFactory implements TableSourceFactory { - private static final String IDENTIFIER = "Opengauss-CDC"; - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public OptionRule optionRule() { - return JdbcSourceOptions.getBaseRule() - .required( - JdbcSourceOptions.USERNAME, - JdbcSourceOptions.PASSWORD, - JdbcCatalogOptions.BASE_URL) - .exclusive(CatalogOptions.TABLE_NAMES, CatalogOptions.TABLE_PATTERN) - .optional( - JdbcSourceOptions.DATABASE_NAMES, - JdbcSourceOptions.SERVER_TIME_ZONE, - JdbcSourceOptions.CONNECT_TIMEOUT_MS, - JdbcSourceOptions.CONNECT_MAX_RETRIES, - JdbcSourceOptions.CONNECTION_POOL_SIZE, - PostgresOptions.DECODING_PLUGIN_NAME, - PostgresOptions.SLOT_NAME, - JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND, - JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND, - JdbcSourceOptions.SAMPLE_SHARDING_THRESHOLD, - JdbcSourceOptions.TABLE_NAMES_CONFIG) - .optional(PostgresSourceOptions.STARTUP_MODE, PostgresSourceOptions.STOP_MODE) - .conditional( - PostgresSourceOptions.STARTUP_MODE, - StartupMode.INITIAL, - JdbcSourceOptions.EXACTLY_ONCE) - .build(); - } - - @Override - public Class getSourceClass() { - return PostgresIncrementalSource.class; - } - - @Override - public - TableSource createSource(TableSourceFactoryContext context) { - return () -> { - List catalogTables = - CatalogTableUtil.getCatalogTables( - "Postgres", context.getOptions(), context.getClassLoader()); - Optional> tableConfigs = - context.getOptions().getOptional(JdbcSourceOptions.TABLE_NAMES_CONFIG); - if (tableConfigs.isPresent()) { - catalogTables = - CatalogTableUtils.mergeCatalogTableConfig( - catalogTables, tableConfigs.get(), s -> TablePath.of(s, true)); - } - SeaTunnelDataType dataType = - CatalogTableUtil.convertToMultipleRowType(catalogTables); - return (SeaTunnelSource) - new PostgresIncrementalSource<>(context.getOptions(), dataType, catalogTables); - }; - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/pom.xml b/seatunnel-connectors-v2/connector-cdc/pom.xml index a422f6406d3..44916d35caa 100644 --- a/seatunnel-connectors-v2/connector-cdc/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/pom.xml @@ -36,7 +36,6 @@ connector-cdc-mongodb connector-cdc-postgres connector-cdc-oracle - connector-cdc-opengauss diff --git a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java index d83e8b5c96b..4c9e6f47605 100644 --- a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java +++ b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.sink.event.WriterCloseEvent; 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; @@ -98,7 +99,9 @@ public void write(SeaTunnelRow element) { } @Override - public void close() {} + public void close() { + context.getEventListener().onEvent(new WriterCloseEvent()); + } private String fieldsInfo(SeaTunnelRowType seaTunnelRowType) { String[] fieldsInfo = new String[seaTunnelRowType.getTotalFields()]; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java index 32a86dd75d6..bbf594eb10b 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/catalog/ElasticSearchCatalog.java @@ -110,7 +110,8 @@ public String getDefaultDatabase() throws CatalogException { public boolean databaseExists(String databaseName) throws CatalogException { // check if the index exist try { - return esRestClient.checkIndexExist(databaseName); + List indexDocsCount = esRestClient.getIndexDocsCount(databaseName); + return true; } catch (Exception e) { log.error( String.format( diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java index b54541bf93b..f80f20f6736 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java @@ -58,7 +58,6 @@ import javax.net.ssl.SSLContext; -import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -77,7 +76,7 @@ import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType.OBJECT; @Slf4j -public class EsRestClient implements Closeable { +public class EsRestClient { private static final int CONNECTION_REQUEST_TIMEOUT = 10 * 1000; @@ -259,7 +258,6 @@ public ElasticsearchClusterInfo getClusterInfo() { } } - @Override public void close() { try { restClient.close(); @@ -372,30 +370,6 @@ private ScrollResult getDocsFromScrollResponse(ObjectNode responseJson) { return scrollResult; } - /** - * Instead of the getIndexDocsCount method to determine if the index exists, - * - *

- * - *

getIndexDocsCount throws an exception if the index does not exist - * - *

- * - * @param index index - * @return true or false - */ - public boolean checkIndexExist(String index) { - Request request = new Request("HEAD", "/" + index); - try { - Response response = restClient.performRequest(request); - int statusCode = response.getStatusLine().getStatusCode(); - return statusCode == 200; - } catch (Exception ex) { - throw new ElasticsearchConnectorException( - ElasticsearchConnectorErrorCode.CHECK_INDEX_FAILED, ex); - } - } - public List getIndexDocsCount(String index) { String endpoint = String.format("/_cat/indices/%s?h=index,docsCount&format=json", index); Request request = new Request("GET", endpoint); diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java index ffeb69d67f2..c63cd375952 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SourceConfig.java @@ -17,31 +17,15 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.config; -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; - import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; -import org.apache.seatunnel.api.table.catalog.CatalogTable; - -import lombok.Getter; -import lombok.Setter; -import java.io.Serializable; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -@Getter -@Setter -public class SourceConfig implements Serializable { - - public static final Option>> INDEX_LIST = - Options.key("index_list") - .type(new TypeReference>>() {}) - .noDefaultValue() - .withDescription("index_list for multiTable sync"); +public class SourceConfig { public static final Option INDEX = Options.key("index") @@ -77,30 +61,11 @@ public class SourceConfig implements Serializable { .withDescription( "Maximum number of hits to be returned with each Elasticsearch scroll request"); - public static final Option> QUERY = + public static final Option QUERY = Options.key("query") - .type(new TypeReference>() {}) + .objectType(Map.class) .defaultValue( Collections.singletonMap("match_all", new HashMap())) .withDescription( "Elasticsearch query language. You can control the range of data read"); - - private String index; - private List source; - private Map query; - private String scrollTime; - private int scrollSize; - - private CatalogTable catalogTable; - - public SourceConfig clone() { - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setIndex(index); - sourceConfig.setSource(new ArrayList<>(source)); - sourceConfig.setQuery(new HashMap<>(query)); - sourceConfig.setScrollTime(scrollTime); - sourceConfig.setScrollSize(scrollSize); - sourceConfig.setCatalogTable(catalogTable); - return sourceConfig; - } } diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java similarity index 74% rename from seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java index e29c7326102..6c0a5667da7 100644 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceState.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java @@ -15,18 +15,21 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.typesense.source; +package org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source; import lombok.AllArgsConstructor; -import lombok.Getter; +import lombok.Data; import java.io.Serializable; import java.util.List; import java.util.Map; +@Data @AllArgsConstructor -@Getter -public class TypesenseSourceState implements Serializable { - private boolean shouldEnumerate; - private Map> pendingSplit; +public class SourceIndexInfo implements Serializable { + private String index; + private List source; + private Map query; + private String scrollTime; + private int scrollSize; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java index 8ffbb7f4b9d..fe182868d4d 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ElasticsearchConnectorErrorCode.java @@ -29,11 +29,7 @@ public enum ElasticsearchConnectorErrorCode implements SeaTunnelErrorCode { DROP_INDEX_FAILED("ELASTICSEARCH-06", "Drop elasticsearch index failed"), CREATE_INDEX_FAILED("ELASTICSEARCH-07", "Create elasticsearch index failed"), ES_FIELD_TYPE_NOT_SUPPORT("ELASTICSEARCH-08", "Not support the elasticsearch field type"), - CLEAR_INDEX_DATA_FAILED("ELASTICSEARCH-09", "Clear elasticsearch index data failed"), - CHECK_INDEX_FAILED("ELASTICSEARCH-10", "Failed to check whether the index exists"), - SOURCE_CONFIG_ERROR_01( - "ELASTICSEARCH-11", - "'index' or 'index_list' must be configured, with at least one being required."); + CLEAR_INDEX_DATA_FAILED("ELASTICSEARCH-09", "Clear elasticsearch index data failed"); ; private final String code; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java index fd176f2f034..a409a4ae886 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/DefaultSeaTunnelRowDeserializer.java @@ -137,9 +137,7 @@ SeaTunnelRow convert(ElasticsearchRecord rowRecord) { fieldName, value, seaTunnelDataType, JsonUtils.toJsonString(rowRecord)), ex); } - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(seaTunnelFields); - seaTunnelRow.setTableId(rowRecord.getTableId()); - return seaTunnelRow; + return new SeaTunnelRow(seaTunnelFields); } Object convertValue(SeaTunnelDataType fieldType, String fieldValue) diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java index 57c9dcb084f..3e5eb10b582 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/source/ElasticsearchRecord.java @@ -30,6 +30,4 @@ public class ElasticsearchRecord { private Map doc; private List source; - - private String tableId; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java index a22ca179569..7b153f0be3a 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java @@ -40,8 +40,6 @@ import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsType; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ElasticsearchConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ElasticsearchConnectorException; import org.apache.commons.collections4.CollectionUtils; @@ -52,7 +50,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; @Slf4j public class ElasticsearchSource @@ -61,60 +58,30 @@ public class ElasticsearchSource SupportParallelism, SupportColumnProjection { - private final List sourceConfigList; - private final ReadonlyConfig connectionConfig; + private final ReadonlyConfig config; - public ElasticsearchSource(ReadonlyConfig config) { - this.connectionConfig = config; - boolean multiSource = config.getOptional(SourceConfig.INDEX_LIST).isPresent(); - boolean singleSource = config.getOptional(SourceConfig.INDEX).isPresent(); - if (multiSource && singleSource) { - log.warn( - "Elasticsearch Source config warn: when both 'index' and 'index_list' are present in the configuration, only the 'index_list' configuration will take effect"); - } - if (!multiSource && !singleSource) { - throw new ElasticsearchConnectorException( - ElasticsearchConnectorErrorCode.SOURCE_CONFIG_ERROR_01, - ElasticsearchConnectorErrorCode.SOURCE_CONFIG_ERROR_01.getDescription()); - } - if (multiSource) { - this.sourceConfigList = createMultiSource(config); - } else { - this.sourceConfigList = Collections.singletonList(parseOneIndexQueryConfig(config)); - } - } - - private List createMultiSource(ReadonlyConfig config) { - List> configMaps = config.get(SourceConfig.INDEX_LIST); - List configList = - configMaps.stream().map(ReadonlyConfig::fromMap).collect(Collectors.toList()); - List sourceConfigList = new ArrayList<>(configList.size()); - for (ReadonlyConfig readonlyConfig : configList) { - SourceConfig sourceConfig = parseOneIndexQueryConfig(readonlyConfig); - sourceConfigList.add(sourceConfig); - } - return sourceConfigList; - } - - private SourceConfig parseOneIndexQueryConfig(ReadonlyConfig readonlyConfig) { + private CatalogTable catalogTable; - Map query = readonlyConfig.get(SourceConfig.QUERY); - String index = readonlyConfig.get(SourceConfig.INDEX); + private List source; - CatalogTable catalogTable; - List source; - Map arrayColumn; + private Map arrayColumn; - if (readonlyConfig.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { + public ElasticsearchSource(ReadonlyConfig config) { + this.config = config; + if (config.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { // todo: We need to remove the schema in ES. log.warn( - "The schema config in ElasticSearch source/sink is deprecated, please use source config instead!"); - catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); + "The schema config in ElasticSearch sink is deprecated, please use source config instead!"); + catalogTable = CatalogTableUtil.buildWithConfig(config); source = Arrays.asList(catalogTable.getSeaTunnelRowType().getFieldNames()); } else { - source = readonlyConfig.get(SourceConfig.SOURCE); - arrayColumn = readonlyConfig.get(SourceConfig.ARRAY_COLUMN); - Map> esFieldType = getFieldTypeMapping(index, source); + source = config.get(SourceConfig.SOURCE); + arrayColumn = config.get(SourceConfig.ARRAY_COLUMN); + EsRestClient esRestClient = EsRestClient.createInstance(config); + Map> esFieldType = + esRestClient.getFieldTypeMapping(config.get(SourceConfig.INDEX), source); + esRestClient.close(); + if (CollectionUtils.isEmpty(source)) { source = new ArrayList<>(esFieldType.keySet()); } @@ -123,48 +90,26 @@ private SourceConfig parseOneIndexQueryConfig(ReadonlyConfig readonlyConfig) { for (int i = 0; i < source.size(); i++) { String key = source.get(i); - String sourceType = esFieldType.get(key).getDataType(); if (arrayColumn.containsKey(key)) { String value = arrayColumn.get(key); SeaTunnelDataType dataType = SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType(key, value); - builder.column( - PhysicalColumn.of( - key, dataType, 0L, true, null, null, sourceType, null)); + builder.column(PhysicalColumn.of(key, dataType, 0, true, null, null)); continue; } builder.column( - PhysicalColumn.of( - source.get(i), - fieldTypes[i], - 0L, - true, - null, - null, - sourceType, - null)); + PhysicalColumn.of(source.get(i), fieldTypes[i], 0, true, null, null)); } catalogTable = CatalogTable.of( - TableIdentifier.of("elasticsearch", null, index), + TableIdentifier.of( + "elasticsearch", null, config.get(SourceConfig.INDEX)), builder.build(), Collections.emptyMap(), Collections.emptyList(), ""); } - - String scrollTime = readonlyConfig.get(SourceConfig.SCROLL_TIME); - int scrollSize = readonlyConfig.get(SourceConfig.SCROLL_SIZE); - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setSource(source); - sourceConfig.setCatalogTable(catalogTable); - sourceConfig.setQuery(query); - sourceConfig.setScrollTime(scrollTime); - sourceConfig.setScrollSize(scrollSize); - sourceConfig.setIndex(index); - sourceConfig.setCatalogTable(catalogTable); - return sourceConfig; } @Override @@ -179,23 +124,21 @@ public Boundedness getBoundedness() { @Override public List getProducedCatalogTables() { - return sourceConfigList.stream() - .map(SourceConfig::getCatalogTable) - .collect(Collectors.toList()); + return Collections.singletonList(catalogTable); } @Override public SourceReader createReader( SourceReader.Context readerContext) { - return new ElasticsearchSourceReader(readerContext, connectionConfig); + return new ElasticsearchSourceReader( + readerContext, config, catalogTable.getSeaTunnelRowType()); } @Override public SourceSplitEnumerator createEnumerator( SourceSplitEnumerator.Context enumeratorContext) { - return new ElasticsearchSourceSplitEnumerator( - enumeratorContext, connectionConfig, sourceConfigList); + return new ElasticsearchSourceSplitEnumerator(enumeratorContext, config, source); } @Override @@ -204,7 +147,7 @@ public SourceReader createReader( SourceSplitEnumerator.Context enumeratorContext, ElasticsearchSourceState sourceState) { return new ElasticsearchSourceSplitEnumerator( - enumeratorContext, sourceState, connectionConfig, sourceConfigList); + enumeratorContext, sourceState, config, source); } @VisibleForTesting @@ -219,13 +162,4 @@ public static SeaTunnelDataType[] getSeaTunnelDataType( } return fieldTypes; } - - private Map> getFieldTypeMapping( - String index, List source) { - // EsRestClient#getFieldTypeMapping may throw runtime exception - // so here we use try-resources-finally to close the resource - try (EsRestClient esRestClient = EsRestClient.createInstance(connectionConfig)) { - return esRestClient.getFieldTypeMapping(index, source); - } - } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java index 8f41256e37c..6ff08b7d069 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceFactory.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSourceFactory; @@ -39,10 +40,10 @@ import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.EsClusterConnectionConfig.TLS_VERIFY_HOSTNAME; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.EsClusterConnectionConfig.USERNAME; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.INDEX; -import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.INDEX_LIST; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.QUERY; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.SCROLL_SIZE; import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.SCROLL_TIME; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig.SOURCE; @AutoService(Factory.class) public class ElasticsearchSourceFactory implements TableSourceFactory { @@ -54,10 +55,8 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { return OptionRule.builder() - .required(HOSTS) + .required(HOSTS, INDEX) .optional( - INDEX, - INDEX_LIST, USERNAME, PASSWORD, SCROLL_TIME, @@ -69,6 +68,7 @@ public OptionRule optionRule() { TLS_KEY_STORE_PASSWORD, TLS_TRUST_STORE_PATH, TLS_TRUST_STORE_PASSWORD) + .exclusive(SOURCE, TableSchemaOptions.SCHEMA) .build(); } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java index a58c2c622d8..7d2398816a1 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java @@ -23,8 +23,8 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.DefaultSeaTunnelRowDeserializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.ElasticsearchRecord; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.SeaTunnelRowDeserializer; @@ -44,23 +44,27 @@ public class ElasticsearchSourceReader SourceReader.Context context; - private final ReadonlyConfig connConfig; + private final ReadonlyConfig config; private EsRestClient esRestClient; + private final SeaTunnelRowDeserializer deserializer; + Deque splits = new LinkedList<>(); boolean noMoreSplit; private final long pollNextWaitTime = 1000L; - public ElasticsearchSourceReader(SourceReader.Context context, ReadonlyConfig connConfig) { + public ElasticsearchSourceReader( + SourceReader.Context context, ReadonlyConfig config, SeaTunnelRowType rowTypeInfo) { this.context = context; - this.connConfig = connConfig; + this.config = config; + this.deserializer = new DefaultSeaTunnelRowDeserializer(rowTypeInfo); } @Override public void open() { - esRestClient = EsRestClient.createInstance(this.connConfig); + esRestClient = EsRestClient.createInstance(this.config); } @Override @@ -73,10 +77,7 @@ public void pollNext(Collector output) throws Exception { synchronized (output.getCheckpointLock()) { ElasticsearchSourceSplit split = splits.poll(); if (split != null) { - SeaTunnelRowType seaTunnelRowType = split.getSeaTunnelRowType(); - SeaTunnelRowDeserializer deserializer = - new DefaultSeaTunnelRowDeserializer(seaTunnelRowType); - SourceConfig sourceIndexInfo = split.getSourceConfig(); + SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo(); ScrollResult scrollResult = esRestClient.searchByScroll( sourceIndexInfo.getIndex(), @@ -84,12 +85,12 @@ public void pollNext(Collector output) throws Exception { sourceIndexInfo.getQuery(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize()); - outputFromScrollResult(scrollResult, sourceIndexInfo, output, deserializer); + outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output); while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) { scrollResult = esRestClient.searchWithScrollId( scrollResult.getScrollId(), sourceIndexInfo.getScrollTime()); - outputFromScrollResult(scrollResult, sourceIndexInfo, output, deserializer); + outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output); } } else if (noMoreSplit) { // signal to the source that we have reached the end of the data. @@ -102,15 +103,10 @@ public void pollNext(Collector output) throws Exception { } private void outputFromScrollResult( - ScrollResult scrollResult, - SourceConfig sourceConfig, - Collector output, - SeaTunnelRowDeserializer deserializer) { - List source = sourceConfig.getSource(); - String tableId = sourceConfig.getCatalogTable().getTablePath().toString(); + ScrollResult scrollResult, List source, Collector output) { for (Map doc : scrollResult.getDocs()) { SeaTunnelRow seaTunnelRow = - deserializer.deserialize(new ElasticsearchRecord(doc, source, tableId)); + deserializer.deserialize(new ElasticsearchRecord(doc, source)); output.collect(seaTunnelRow); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java index 3c7d25b5b49..f2ad78fa5c4 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java @@ -18,8 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.source; import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; import lombok.AllArgsConstructor; import lombok.Getter; @@ -33,11 +32,7 @@ public class ElasticsearchSourceSplit implements SourceSplit { private String splitId; - @Getter private SourceConfig sourceConfig; - - public SeaTunnelRowType getSeaTunnelRowType() { - return sourceConfig.getCatalogTable().getSeaTunnelRowType(); - } + @Getter private SourceIndexInfo sourceIndexInfo; @Override public String splitId() { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java index 5e3356ebd65..107aaac322a 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java @@ -19,10 +19,11 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SourceConfig; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ElasticsearchConnectorException; import lombok.extern.slf4j.Slf4j; @@ -44,7 +45,7 @@ public class ElasticsearchSourceSplitEnumerator private final SourceSplitEnumerator.Context context; - private final ReadonlyConfig connConfig; + private final ReadonlyConfig config; private EsRestClient esRestClient; @@ -52,36 +53,36 @@ public class ElasticsearchSourceSplitEnumerator private Map> pendingSplit; - private final List sourceConfigs; + private final List source; private volatile boolean shouldEnumerate; public ElasticsearchSourceSplitEnumerator( SourceSplitEnumerator.Context context, - ReadonlyConfig connConfig, - List sourceConfigs) { - this(context, null, connConfig, sourceConfigs); + ReadonlyConfig config, + List source) { + this(context, null, config, source); } public ElasticsearchSourceSplitEnumerator( SourceSplitEnumerator.Context context, ElasticsearchSourceState sourceState, - ReadonlyConfig connConfig, - List sourceConfigs) { + ReadonlyConfig config, + List source) { this.context = context; - this.connConfig = connConfig; + this.config = config; this.pendingSplit = new HashMap<>(); this.shouldEnumerate = sourceState == null; if (sourceState != null) { this.shouldEnumerate = sourceState.isShouldEnumerate(); this.pendingSplit.putAll(sourceState.getPendingSplit()); } - this.sourceConfigs = sourceConfigs; + this.source = source; } @Override public void open() { - esRestClient = EsRestClient.createInstance(connConfig); + esRestClient = EsRestClient.createInstance(config); } @Override @@ -139,22 +140,26 @@ private void assignSplit(Collection readers) { private List getElasticsearchSplit() { List splits = new ArrayList<>(); - for (SourceConfig sourceConfig : sourceConfigs) { - - String index = sourceConfig.getIndex(); - List indexDocsCounts = esRestClient.getIndexDocsCount(index); - indexDocsCounts = - indexDocsCounts.stream() - .filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0) - .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)) - .collect(Collectors.toList()); - for (IndexDocsCount indexDocsCount : indexDocsCounts) { - SourceConfig cloneCfg = sourceConfig.clone(); - cloneCfg.setIndex(indexDocsCount.getIndex()); - splits.add( - new ElasticsearchSourceSplit( - String.valueOf(indexDocsCount.getIndex().hashCode()), cloneCfg)); - } + String scrollTime = config.get(SourceConfig.SCROLL_TIME); + int scrollSize = config.get(SourceConfig.SCROLL_SIZE); + Map query = config.get(SourceConfig.QUERY); + List indexDocsCounts = + esRestClient.getIndexDocsCount(config.get(SourceConfig.INDEX)); + indexDocsCounts = + indexDocsCounts.stream() + .filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0) + .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)) + .collect(Collectors.toList()); + for (IndexDocsCount indexDocsCount : indexDocsCounts) { + splits.add( + new ElasticsearchSourceSplit( + String.valueOf(indexDocsCount.getIndex().hashCode()), + new SourceIndexInfo( + indexDocsCount.getIndex(), + source, + query, + scrollTime, + scrollSize))); } return splits; } @@ -180,7 +185,7 @@ public int currentUnassignedSplitSize() { @Override public void handleSplitRequest(int subtaskId) { throw new ElasticsearchConnectorException( - CommonErrorCode.OPERATION_NOT_SUPPORTED, + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, "Unsupported handleSplitRequest: " + subtaskId); } diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java index 524d2310632..9ac392b6a7a 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGenerator.java @@ -18,8 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.fake.source; import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -34,11 +34,9 @@ import java.io.IOException; import java.lang.reflect.Array; -import java.math.BigDecimal; import java.util.ArrayList; import java.util.HashMap; import java.util.List; -import java.util.function.Function; public class FakeDataGenerator { private final CatalogTable catalogTable; @@ -73,11 +71,12 @@ private SeaTunnelRow convertRow(FakeConfig.RowData rowData) { } private SeaTunnelRow randomRow() { - // Generate random data according to the data type and data colum of the table - List physicalColumns = catalogTable.getTableSchema().getColumns(); - List randomRow = new ArrayList<>(physicalColumns.size()); - for (Column column : physicalColumns) { - randomRow.add(randomColumnValue(column)); + SeaTunnelRowType rowType = catalogTable.getSeaTunnelRowType(); + String[] fieldNames = rowType.getFieldNames(); + SeaTunnelDataType[] fieldTypes = rowType.getFieldTypes(); + List randomRow = new ArrayList<>(fieldNames.length); + for (SeaTunnelDataType fieldType : fieldTypes) { + randomRow.add(randomColumnValue(fieldType)); } SeaTunnelRow seaTunnelRow = new SeaTunnelRow(randomRow.toArray()); seaTunnelRow.setTableId(tableId); @@ -104,8 +103,7 @@ public List generateFakedRows(int rowNum) { } @SuppressWarnings("magicnumber") - private Object randomColumnValue(Column column) { - SeaTunnelDataType fieldType = column.getDataType(); + private Object randomColumnValue(SeaTunnelDataType fieldType) { switch (fieldType.getSqlType()) { case ARRAY: ArrayType arrayType = (ArrayType) fieldType; @@ -113,7 +111,7 @@ private Object randomColumnValue(Column column) { int length = fakeConfig.getArraySize(); Object array = Array.newInstance(elementType.getTypeClass(), length); for (int i = 0; i < length; i++) { - Object value = randomColumnValue(column.copy(elementType)); + Object value = randomColumnValue(elementType); Array.set(array, i, value); } return array; @@ -124,57 +122,59 @@ private Object randomColumnValue(Column column) { HashMap objectMap = new HashMap<>(); int mapSize = fakeConfig.getMapSize(); for (int i = 0; i < mapSize; i++) { - Object key = randomColumnValue(column.copy(keyType)); - Object value = randomColumnValue(column.copy(valueType)); + Object key = randomColumnValue(keyType); + Object value = randomColumnValue(valueType); objectMap.put(key, value); } return objectMap; case STRING: - return value(column, String::toString, fakeDataRandomUtils::randomString); + return fakeDataRandomUtils.randomString(); case BOOLEAN: - return value(column, Boolean::parseBoolean, fakeDataRandomUtils::randomBoolean); + return fakeDataRandomUtils.randomBoolean(); case TINYINT: - return value(column, Byte::parseByte, fakeDataRandomUtils::randomTinyint); + return fakeDataRandomUtils.randomTinyint(); case SMALLINT: - return value(column, Short::parseShort, fakeDataRandomUtils::randomSmallint); + return fakeDataRandomUtils.randomSmallint(); case INT: - return value(column, Integer::parseInt, fakeDataRandomUtils::randomInt); + return fakeDataRandomUtils.randomInt(); case BIGINT: - return value(column, Long::parseLong, fakeDataRandomUtils::randomBigint); + return fakeDataRandomUtils.randomBigint(); case FLOAT: - return value(column, Float::parseFloat, fakeDataRandomUtils::randomFloat); + return fakeDataRandomUtils.randomFloat(); case DOUBLE: - return value(column, Double::parseDouble, fakeDataRandomUtils::randomDouble); + return fakeDataRandomUtils.randomDouble(); case DECIMAL: - return value(column, BigDecimal::new, fakeDataRandomUtils::randomBigDecimal); + DecimalType decimalType = (DecimalType) fieldType; + return fakeDataRandomUtils.randomBigDecimal( + decimalType.getPrecision(), decimalType.getScale()); case NULL: return null; case BYTES: - return value(column, String::getBytes, fakeDataRandomUtils::randomBytes); + return fakeDataRandomUtils.randomBytes(); case DATE: - return value(column, String::toString, fakeDataRandomUtils::randomLocalDate); + return fakeDataRandomUtils.randomLocalDate(); case TIME: - return value(column, String::toString, fakeDataRandomUtils::randomLocalTime); + return fakeDataRandomUtils.randomLocalTime(); case TIMESTAMP: - return value(column, String::toString, fakeDataRandomUtils::randomLocalDateTime); + return fakeDataRandomUtils.randomLocalDateTime(); case ROW: SeaTunnelDataType[] fieldTypes = ((SeaTunnelRowType) fieldType).getFieldTypes(); Object[] objects = new Object[fieldTypes.length]; for (int i = 0; i < fieldTypes.length; i++) { - Object object = randomColumnValue(column.copy(fieldTypes[i])); + Object object = randomColumnValue(fieldTypes[i]); objects[i] = object; } return new SeaTunnelRow(objects); case BINARY_VECTOR: - return fakeDataRandomUtils.randomBinaryVector(column); + return fakeDataRandomUtils.randomBinaryVector(); case FLOAT_VECTOR: - return fakeDataRandomUtils.randomFloatVector(column); + return fakeDataRandomUtils.randomFloatVector(); case FLOAT16_VECTOR: - return fakeDataRandomUtils.randomFloat16Vector(column); + return fakeDataRandomUtils.randomFloat16Vector(); case BFLOAT16_VECTOR: - return fakeDataRandomUtils.randomBFloat16Vector(column); + return fakeDataRandomUtils.randomBFloat16Vector(); case SPARSE_FLOAT_VECTOR: - return fakeDataRandomUtils.randomSparseFloatVector(column); + return fakeDataRandomUtils.randomSparseFloatVector(); default: // never got in there throw new FakeConnectorException( @@ -182,12 +182,4 @@ private Object randomColumnValue(Column column) { "SeaTunnel Fake source connector not support this data type"); } } - - private static T value( - Column column, Function convert, Function generate) { - if (column.getDefaultValue() != null) { - return convert.apply(column.getDefaultValue().toString()); - } - return generate.apply(column); - } } diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java index 063ece63d2e..95758cb971e 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceReader.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.event.ReaderCloseEvent; +import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.fake.config.FakeConfig; import org.apache.seatunnel.connectors.seatunnel.fake.config.MultipleTableFakeSourceConfig; @@ -71,10 +73,14 @@ public FakeSourceReader( } @Override - public void open() {} + public void open() { + context.getEventListener().onEvent(new ReaderOpenEvent()); + } @Override - public void close() {} + public void close() { + context.getEventListener().onEvent(new ReaderCloseEvent()); + } @Override @SuppressWarnings("MagicNumber") diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java index 311e8183766..ecd6d509149 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java @@ -18,6 +18,8 @@ package org.apache.seatunnel.connectors.seatunnel.fake.source; import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; +import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; import org.apache.seatunnel.connectors.seatunnel.fake.config.FakeConfig; import org.apache.seatunnel.connectors.seatunnel.fake.config.MultipleTableFakeSourceConfig; import org.apache.seatunnel.connectors.seatunnel.fake.state.FakeSourceState; @@ -56,7 +58,9 @@ public FakeSourceSplitEnumerator( } @Override - public void open() {} + public void open() { + enumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); + } @Override public void run() throws Exception { @@ -65,7 +69,9 @@ public void run() throws Exception { } @Override - public void close() throws IOException {} + public void close() throws IOException { + enumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); + } @Override public void addSplitsBack(List splits, int subtaskId) { diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java index c4a038ff1a1..8a8a14dc70b 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/utils/FakeDataRandomUtils.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.fake.utils; -import org.apache.seatunnel.api.table.catalog.Column; -import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.common.utils.BufferUtils; import org.apache.seatunnel.connectors.seatunnel.fake.config.FakeConfig; @@ -27,7 +25,6 @@ import org.apache.commons.lang3.RandomUtils; import java.math.BigDecimal; -import java.math.RoundingMode; import java.nio.ByteBuffer; import java.time.LocalDate; import java.time.LocalDateTime; @@ -48,34 +45,30 @@ private static T randomFromList(List list) { return list.get(index); } - public Boolean randomBoolean(Column column) { + public Boolean randomBoolean() { return RandomUtils.nextInt(0, 2) == 1; } - public BigDecimal randomBigDecimal(Column column) { - DecimalType dataType = (DecimalType) column.getDataType(); + public BigDecimal randomBigDecimal(int precision, int scale) { return new BigDecimal( - RandomStringUtils.randomNumeric(dataType.getPrecision() - dataType.getScale()) + RandomStringUtils.randomNumeric(precision - scale) + "." - + RandomStringUtils.randomNumeric(dataType.getScale())); + + RandomStringUtils.randomNumeric(scale)); } - public byte[] randomBytes(Column column) { + public byte[] randomBytes() { return RandomStringUtils.randomAlphabetic(fakeConfig.getBytesLength()).getBytes(); } - public String randomString(Column column) { + public String randomString() { List stringTemplate = fakeConfig.getStringTemplate(); if (!CollectionUtils.isEmpty(stringTemplate)) { return randomFromList(stringTemplate); } - return RandomStringUtils.randomAlphabetic( - column.getColumnLength() != null - ? column.getColumnLength().intValue() - : fakeConfig.getStringLength()); + return RandomStringUtils.randomAlphabetic(fakeConfig.getStringLength()); } - public Byte randomTinyint(Column column) { + public Byte randomTinyint() { List tinyintTemplate = fakeConfig.getTinyintTemplate(); if (!CollectionUtils.isEmpty(tinyintTemplate)) { return randomFromList(tinyintTemplate).byteValue(); @@ -83,7 +76,7 @@ public Byte randomTinyint(Column column) { return (byte) RandomUtils.nextInt(fakeConfig.getTinyintMin(), fakeConfig.getTinyintMax()); } - public Short randomSmallint(Column column) { + public Short randomSmallint() { List smallintTemplate = fakeConfig.getSmallintTemplate(); if (!CollectionUtils.isEmpty(smallintTemplate)) { return randomFromList(smallintTemplate).shortValue(); @@ -92,7 +85,7 @@ public Short randomSmallint(Column column) { RandomUtils.nextInt(fakeConfig.getSmallintMin(), fakeConfig.getSmallintMax()); } - public Integer randomInt(Column column) { + public Integer randomInt() { List intTemplate = fakeConfig.getIntTemplate(); if (!CollectionUtils.isEmpty(intTemplate)) { return randomFromList(intTemplate); @@ -100,7 +93,7 @@ public Integer randomInt(Column column) { return RandomUtils.nextInt(fakeConfig.getIntMin(), fakeConfig.getIntMax()); } - public Long randomBigint(Column column) { + public Long randomBigint() { List bigTemplate = fakeConfig.getBigTemplate(); if (!CollectionUtils.isEmpty(bigTemplate)) { return randomFromList(bigTemplate); @@ -108,39 +101,32 @@ public Long randomBigint(Column column) { return RandomUtils.nextLong(fakeConfig.getBigintMin(), fakeConfig.getBigintMax()); } - public Float randomFloat(Column column) { + public Float randomFloat() { List floatTemplate = fakeConfig.getFloatTemplate(); if (!CollectionUtils.isEmpty(floatTemplate)) { return randomFromList(floatTemplate).floatValue(); } - float v = - RandomUtils.nextFloat( - (float) fakeConfig.getFloatMin(), (float) fakeConfig.getFloatMax()); - return column.getScale() == null - ? v - : new BigDecimal(v).setScale(column.getScale(), RoundingMode.HALF_UP).floatValue(); + return RandomUtils.nextFloat( + (float) fakeConfig.getFloatMin(), (float) fakeConfig.getFloatMax()); } - public Double randomDouble(Column column) { + public Double randomDouble() { List doubleTemplate = fakeConfig.getDoubleTemplate(); if (!CollectionUtils.isEmpty(doubleTemplate)) { return randomFromList(doubleTemplate); } - double v = RandomUtils.nextDouble(fakeConfig.getDoubleMin(), fakeConfig.getDoubleMax()); - return column.getScale() == null - ? v - : new BigDecimal(v).setScale(column.getScale(), RoundingMode.HALF_UP).floatValue(); + return RandomUtils.nextDouble(fakeConfig.getDoubleMin(), fakeConfig.getDoubleMax()); } - public LocalDate randomLocalDate(Column column) { - return randomLocalDateTime(column).toLocalDate(); + public LocalDate randomLocalDate() { + return randomLocalDateTime().toLocalDate(); } - public LocalTime randomLocalTime(Column column) { - return randomLocalDateTime(column).toLocalTime(); + public LocalTime randomLocalTime() { + return randomLocalDateTime().toLocalTime(); } - public LocalDateTime randomLocalDateTime(Column column) { + public LocalDateTime randomLocalDateTime() { int year; int month; int day; @@ -186,20 +172,15 @@ public LocalDateTime randomLocalDateTime(Column column) { return LocalDateTime.of(year, month, day, hour, minute, second); } - public ByteBuffer randomBinaryVector(Column column) { - int byteCount = - (column.getScale() != null) - ? column.getScale() / 8 - : fakeConfig.getBinaryVectorDimension() / 8; + public ByteBuffer randomBinaryVector() { + int byteCount = fakeConfig.getBinaryVectorDimension() / 8; // binary vector doesn't care endian since each byte is independent return ByteBuffer.wrap(RandomUtils.nextBytes(byteCount)); } - public ByteBuffer randomFloatVector(Column column) { - int count = - (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); - Float[] floatVector = new Float[count]; - for (int i = 0; i < count; i++) { + public ByteBuffer randomFloatVector() { + Float[] floatVector = new Float[fakeConfig.getVectorDimension()]; + for (int i = 0; i < fakeConfig.getVectorDimension(); i++) { floatVector[i] = RandomUtils.nextFloat( fakeConfig.getVectorFloatMin(), fakeConfig.getVectorFloatMax()); @@ -207,11 +188,9 @@ public ByteBuffer randomFloatVector(Column column) { return BufferUtils.toByteBuffer(floatVector); } - public ByteBuffer randomFloat16Vector(Column column) { - int count = - (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); - Short[] float16Vector = new Short[count]; - for (int i = 0; i < count; i++) { + public ByteBuffer randomFloat16Vector() { + Short[] float16Vector = new Short[fakeConfig.getVectorDimension()]; + for (int i = 0; i < fakeConfig.getVectorDimension(); i++) { float value = RandomUtils.nextFloat( fakeConfig.getVectorFloatMin(), fakeConfig.getVectorFloatMax()); @@ -220,11 +199,9 @@ public ByteBuffer randomFloat16Vector(Column column) { return BufferUtils.toByteBuffer(float16Vector); } - public ByteBuffer randomBFloat16Vector(Column column) { - int count = - (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); - Short[] bfloat16Vector = new Short[count]; - for (int i = 0; i < count; i++) { + public ByteBuffer randomBFloat16Vector() { + Short[] bfloat16Vector = new Short[fakeConfig.getVectorDimension()]; + for (int i = 0; i < fakeConfig.getVectorDimension(); i++) { float value = RandomUtils.nextFloat( fakeConfig.getVectorFloatMin(), fakeConfig.getVectorFloatMax()); @@ -233,10 +210,10 @@ public ByteBuffer randomBFloat16Vector(Column column) { return BufferUtils.toByteBuffer(bfloat16Vector); } - public Map randomSparseFloatVector(Column column) { + public Map randomSparseFloatVector() { Map sparseVector = new HashMap<>(); - int nonZeroElements = - (column.getScale() != null) ? column.getScale() : fakeConfig.getVectorDimension(); + + Integer nonZeroElements = fakeConfig.getVectorDimension(); while (nonZeroElements > 0) { Integer index = RandomUtils.nextInt(); Float value = diff --git a/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java b/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java index e33883f554e..c1cd826cb0a 100644 --- a/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java +++ b/seatunnel-connectors-v2/connector-fake/src/test/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeDataGeneratorTest.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -39,7 +38,6 @@ import java.io.FileNotFoundException; import java.net.URISyntaxException; import java.net.URL; -import java.nio.ByteBuffer; import java.nio.file.Paths; import java.util.Arrays; import java.util.List; @@ -143,59 +141,6 @@ public void testVectorParse(String conf) throws FileNotFoundException, URISyntax Assertions.assertNotNull(seaTunnelRows); } - @ParameterizedTest - @ValueSource(strings = {"fake-data.column.conf"}) - public void testColumnDataParse(String conf) throws FileNotFoundException, URISyntaxException { - ReadonlyConfig testConfig = getTestConfigFile(conf); - FakeConfig fakeConfig = FakeConfig.buildWithConfig(testConfig); - FakeDataGenerator fakeDataGenerator = new FakeDataGenerator(fakeConfig); - List seaTunnelRows = - fakeDataGenerator.generateFakedRows(fakeConfig.getRowNum()); - seaTunnelRows.forEach( - seaTunnelRow -> { - Assertions.assertEquals( - seaTunnelRow.getField(0).toString(), "Andersen's Fairy Tales"); - Assertions.assertEquals(seaTunnelRow.getField(1).toString().length(), 100); - Assertions.assertEquals(seaTunnelRow.getField(2).toString(), "10.1"); - Assertions.assertNotNull(seaTunnelRow.getField(3).toString()); - Assertions.assertNotNull(seaTunnelRow.getField(4).toString()); - // VectorType.VECTOR_FLOAT_TYPE - Assertions.assertEquals( - 8, ((ByteBuffer) seaTunnelRow.getField(5)).capacity() / 4); - // VectorType.VECTOR_BINARY_TYPE - Assertions.assertEquals( - 16, ((ByteBuffer) seaTunnelRow.getField(6)).capacity() * 8); - // VectorType.VECTOR_FLOAT16_TYPE - Assertions.assertEquals( - 8, ((ByteBuffer) seaTunnelRow.getField(7)).capacity() / 2); - // VectorType.VECTOR_BFLOAT16_TYPE - Assertions.assertEquals( - 8, ((ByteBuffer) seaTunnelRow.getField(8)).capacity() / 2); - // VectorType.VECTOR_SPARSE_FLOAT_TYPE - Assertions.assertEquals(8, ((Map) seaTunnelRow.getField(9)).size()); - Assertions.assertEquals( - 268, - seaTunnelRow.getBytesSize( - new SeaTunnelRowType( - new String[] { - "field1", "field2", "field3", "field4", "field5", - "field6", "field7", "field8", "field9", "field10" - }, - new SeaTunnelDataType[] { - BasicType.STRING_TYPE, - BasicType.STRING_TYPE, - BasicType.FLOAT_TYPE, - BasicType.FLOAT_TYPE, - BasicType.DOUBLE_TYPE, - VectorType.VECTOR_FLOAT_TYPE, - VectorType.VECTOR_BINARY_TYPE, - VectorType.VECTOR_FLOAT16_TYPE, - VectorType.VECTOR_BFLOAT16_TYPE, - VectorType.VECTOR_SPARSE_FLOAT_TYPE - }))); - }); - } - private ReadonlyConfig getTestConfigFile(String configFile) throws FileNotFoundException, URISyntaxException { if (!configFile.startsWith("/")) { diff --git a/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf b/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf index e3f0d7ee267..96e82ee41c5 100644 --- a/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf +++ b/seatunnel-connectors-v2/connector-fake/src/test/resources/complex.schema.conf @@ -23,7 +23,7 @@ FakeSource { string.length = 10 schema = { fields { - c_map = "map>" + c_map = "map>" c_array = "array" c_string = string c_boolean = boolean diff --git a/seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf b/seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf deleted file mode 100644 index 9a1515264e0..00000000000 --- a/seatunnel-connectors-v2/connector-fake/src/test/resources/fake-data.column.conf +++ /dev/null @@ -1,97 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - - FakeSource { - row.num = 5 - vector.float.max=1 - vector.float.min=0 - float.max = 2 - float.min = 0 - double.max = 4 - double.min = 2 - - # low weight - string.length = 4 - vector.dimension= 4 - binary.vector.dimension=8 - # end - - schema = { - columns = [ - { - name = book_name - type = string - defaultValue = "Andersen's Fairy Tales" - comment = "book name" - }, - { - name = book_reader_testimonials - type = string - columnLength = 100 - comment = "book reader testimonials" - }, - { - name = book_price - type = float - defaultValue = 10.1 - comment = "book price" - }, - { - name = book_percentage_popularity - type = float - columnScale = 4 - comment = "book percentage popularity" - }, - { - name = book_distribution_law - type = double - columnScale = 2 - comment = "book distribution law" - }, - { - name = book_intro_1 - type = float_vector - columnScale =8 - comment = "vector" - }, - { - name = book_intro_2 - type = binary_vector - columnScale = 16 - comment = "vector" - }, - { - name = book_intro_3 - type = float16_vector - columnScale =8 - comment = "vector" - }, - { - name = book_intro_4 - type = bfloat16_vector - columnScale =8 - comment = "vector" - }, - { - name = book_intro_5 - type = sparse_float_vector - columnScale =8 - comment = "vector" - } - ] - } - } \ No newline at end of file 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 66b4eb967b5..d63c8ef8efd 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 @@ -19,7 +19,6 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.common.config.TypesafeConfigUtils; import lombok.Builder; @@ -83,9 +82,10 @@ public class HbaseParameters implements Serializable { @Builder.Default private HbaseConfig.EnCoding enCoding = ENCODING.defaultValue(); - public static HbaseParameters buildWithConfig(ReadonlyConfig config) { + public static HbaseParameters buildWithSinkConfig(Config pluginConfig) { HbaseParametersBuilder builder = HbaseParameters.builder(); - String table = config.get(TABLE); + builder.zookeeperQuorum(pluginConfig.getString(ZOOKEEPER_QUORUM.key())); + String table = pluginConfig.getString(TABLE.key()); int colonIndex = table.indexOf(':'); if (colonIndex != -1) { String namespace = table.substring(0, colonIndex); @@ -96,21 +96,38 @@ public static HbaseParameters buildWithConfig(ReadonlyConfig config) { builder.namespace("default"); } - // required parameters - builder.zookeeperQuorum(config.get(ZOOKEEPER_QUORUM)); - builder.rowkeyColumns(config.get(ROWKEY_COLUMNS)); - builder.familyNames(config.get(FAMILY_NAME)); - - builder.rowkeyDelimiter(config.get(ROWKEY_DELIMITER)); - builder.versionColumn(config.get(VERSION_COLUMN)); - String nullMode = String.valueOf(config.get(NULL_MODE)); - builder.nullMode(HbaseConfig.NullMode.valueOf(nullMode.toUpperCase())); - builder.walWrite(config.get(WAL_WRITE)); - builder.writeBufferSize(config.get(WRITE_BUFFER_SIZE)); - String encoding = String.valueOf(config.get(ENCODING)); - builder.enCoding(HbaseConfig.EnCoding.valueOf(encoding.toUpperCase())); - builder.hbaseExtraConfig(config.get(HBASE_EXTRA_CONFIG)); - builder.ttl(config.get(HBASE_TTL_CONFIG)); + builder.rowkeyColumns(pluginConfig.getStringList(ROWKEY_COLUMNS.key())); + builder.familyNames( + TypesafeConfigUtils.configToMap(pluginConfig.getConfig(FAMILY_NAME.key()))); + + // optional parameters + if (pluginConfig.hasPath(HBASE_TTL_CONFIG.key())) { + builder.ttl(pluginConfig.getLong(HBASE_TTL_CONFIG.key())); + } + if (pluginConfig.hasPath(ROWKEY_DELIMITER.key())) { + builder.rowkeyDelimiter(pluginConfig.getString(ROWKEY_DELIMITER.key())); + } + if (pluginConfig.hasPath(VERSION_COLUMN.key())) { + builder.versionColumn(pluginConfig.getString(VERSION_COLUMN.key())); + } + if (pluginConfig.hasPath(NULL_MODE.key())) { + String nullMode = pluginConfig.getString(NULL_MODE.key()); + builder.nullMode(HbaseConfig.NullMode.valueOf(nullMode.toUpperCase())); + } + if (pluginConfig.hasPath(WAL_WRITE.key())) { + builder.walWrite(pluginConfig.getBoolean(WAL_WRITE.key())); + } + if (pluginConfig.hasPath(WRITE_BUFFER_SIZE.key())) { + builder.writeBufferSize(pluginConfig.getInt(WRITE_BUFFER_SIZE.key())); + } + if (pluginConfig.hasPath(ENCODING.key())) { + String encoding = pluginConfig.getString(ENCODING.key()); + builder.enCoding(HbaseConfig.EnCoding.valueOf(encoding.toUpperCase())); + } + if (pluginConfig.hasPath(HBASE_EXTRA_CONFIG.key())) { + Config extraConfig = pluginConfig.getConfig(HBASE_EXTRA_CONFIG.key()); + builder.hbaseExtraConfig(TypesafeConfigUtils.configToMap(extraConfig)); + } 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 0a46b1baefa..14f8ec8c7ef 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 @@ -65,7 +65,7 @@ public class HbaseSink private int versionColumnIndex = -1; public HbaseSink(ReadonlyConfig config, CatalogTable catalogTable) { - this.hbaseParameters = HbaseParameters.buildWithConfig(config); + this.hbaseParameters = HbaseParameters.buildWithSinkConfig(config.toConfig()); this.config = config; this.catalogTable = catalogTable; this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java index 0992b11d710..9fab21954a7 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java @@ -45,8 +45,6 @@ @AutoService(Factory.class) public class HbaseSinkFactory implements TableSinkFactory { - public static final String IDENTIFIER = "Hbase"; - @Override public String factoryIdentifier() { return HbaseIdentifier.IDENTIFIER_NAME; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java index 73ee19f9369..a41a2386297 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java @@ -58,7 +58,7 @@ public class HbaseSinkWriter private int versionColumnIndex; - private String defaultFamilyName = "value"; + private String writeAllColumnFamily; public HbaseSinkWriter( SeaTunnelRowType seaTunnelRowType, @@ -71,8 +71,7 @@ public HbaseSinkWriter( this.versionColumnIndex = versionColumnIndex; if (hbaseParameters.getFamilyNames().size() == 1) { - defaultFamilyName = - hbaseParameters.getFamilyNames().getOrDefault(ALL_COLUMNS, defaultFamilyName); + this.writeAllColumnFamily = hbaseParameters.getFamilyNames().get(ALL_COLUMNS); } this.hbaseClient = HbaseClient.createInstance(hbaseParameters); @@ -122,7 +121,11 @@ private Put convertRowToPut(SeaTunnelRow row) { String fieldName = seaTunnelRowType.getFieldName(writeColumnIndex); Map configurationFamilyNames = hbaseParameters.getFamilyNames(); String familyName = - hbaseParameters.getFamilyNames().getOrDefault(fieldName, defaultFamilyName); + configurationFamilyNames.getOrDefault(fieldName, writeAllColumnFamily); + if (!configurationFamilyNames.containsKey(ALL_COLUMNS) + && !configurationFamilyNames.containsKey(fieldName)) { + continue; + } byte[] bytes = convertColumnToBytes(row, writeColumnIndex); if (bytes != null) { put.addColumn(Bytes.toBytes(familyName), Bytes.toBytes(fieldName), bytes); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java index bbb590502cf..0f4610783af 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogLoader.java @@ -50,20 +50,25 @@ public class IcebergCatalogLoader implements Serializable { private static final long serialVersionUID = -6003040601422350869L; private static final List HADOOP_CONF_FILES = ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); - private final CommonConfig config; + private CommonConfig config; public IcebergCatalogLoader(CommonConfig config) { this.config = config; } public Catalog loadCatalog() { - // When using the SeaTunnel engine, set the current class loader to prevent loading failures + // When using the seatunel engine, set the current class loader to prevent loading failures Thread.currentThread().setContextClassLoader(IcebergCatalogLoader.class.getClassLoader()); return CatalogUtil.buildIcebergCatalog( config.getCatalogName(), config.getCatalogProps(), loadHadoopConfig(config)); } - /** Loading Hadoop configuration through reflection */ + /** + * Loading Hadoop configuration through reflection + * + * @param config + * @return + */ public Object loadHadoopConfig(CommonConfig config) { Class configClass = DynClasses.builder() diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java index fc28001b2ca..520f9bdbac9 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/catalog/IcebergCatalog.java @@ -58,9 +58,9 @@ @Slf4j public class IcebergCatalog implements Catalog { - private final String catalogName; - private final ReadonlyConfig readonlyConfig; - private final IcebergCatalogLoader icebergCatalogLoader; + private String catalogName; + private ReadonlyConfig readonlyConfig; + private IcebergCatalogLoader icebergCatalogLoader; private org.apache.iceberg.catalog.Catalog catalog; public IcebergCatalog(String catalogName, ReadonlyConfig readonlyConfig) { @@ -224,21 +224,22 @@ public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) public CatalogTable toCatalogTable(Table icebergTable, TablePath tablePath) { List columns = icebergTable.schema().columns(); TableSchema.Builder builder = TableSchema.builder(); - columns.forEach( - nestedField -> { - String name = nestedField.name(); - SeaTunnelDataType seaTunnelType = - SchemaUtils.toSeaTunnelType(name, nestedField.type()); - PhysicalColumn physicalColumn = - PhysicalColumn.of( - name, - seaTunnelType, - (Long) null, - true, - null, - nestedField.doc()); - builder.column(physicalColumn); - }); + columns.stream() + .forEach( + nestedField -> { + String name = nestedField.name(); + SeaTunnelDataType seaTunnelType = + SchemaUtils.toSeaTunnelType(name, nestedField.type()); + PhysicalColumn physicalColumn = + PhysicalColumn.of( + name, + seaTunnelType, + (Long) null, + true, + null, + nestedField.doc()); + builder.column(physicalColumn); + }); List partitionKeys = icebergTable.spec().fields().stream() diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java index f46928456fb..8c699b34402 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/RowConverter.java @@ -92,17 +92,17 @@ private NameMapping createNameMapping(Table table) { return nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; } - public Record convert(Object row, SeaTunnelDataType rowType) { + public Record convert(Object row, SeaTunnelDataType rowType) { return convertStructValue(row, rowType, tableSchema.asStruct(), -1, null); } - public Record convert(Object row, SeaTunnelDataType rowType, SchemaChangeWrapper wrapper) { + public Record convert(Object row, SeaTunnelDataType rowType, SchemaChangeWrapper wrapper) { return convertStructValue(row, rowType, tableSchema.asStruct(), -1, wrapper); } protected GenericRecord convertStructValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.StructType schema, int parentFieldId, SchemaChangeWrapper wrapper) { @@ -120,7 +120,15 @@ protected GenericRecord convertStructValue( } } - /** Convert RowType */ + /** + * Convert RowType + * + * @param row + * @param fromType + * @param schema + * @param structFieldId + * @return + */ private GenericRecord convertToStruct( SeaTunnelRow row, SeaTunnelRowType fromType, @@ -171,7 +179,7 @@ private GenericRecord convertToStruct( public Object convertValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Type type, int fieldId, SchemaChangeWrapper wrapper) { @@ -244,7 +252,7 @@ private Map createStructNameMap(Types.StructType sche protected List convertListValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.ListType type, SchemaChangeWrapper wrapper) { Preconditions.checkArgument(value.getClass().isArray()); @@ -261,7 +269,7 @@ protected List convertListValue( protected Map convertMapValue( Object value, - SeaTunnelDataType fromType, + SeaTunnelDataType fromType, Types.MapType type, SchemaChangeWrapper wrapper) { Preconditions.checkArgument(value instanceof Map); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java index a1d43d6acfd..65bccbdb893 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSink.java @@ -61,9 +61,9 @@ public class IcebergSink SupportSaveMode, SupportMultiTableSink { private static String PLUGIN_NAME = "Iceberg"; - private final SinkConfig config; - private final ReadonlyConfig readonlyConfig; - private final CatalogTable catalogTable; + private SinkConfig config; + private ReadonlyConfig readonlyConfig; + private CatalogTable catalogTable; public IcebergSink(ReadonlyConfig pluginConfig, CatalogTable catalogTable) { this.readonlyConfig = pluginConfig; diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java index 3a5e22b93b4..aed6522ca87 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java @@ -54,12 +54,13 @@ public class IcebergSinkWriter implements SinkWriter, SupportMultiTableSinkWriter { private SeaTunnelRowType rowType; - private final SinkConfig config; - private final IcebergTableLoader icebergTableLoader; + private SinkConfig config; + private IcebergTableLoader icebergTableLoader; private RecordWriter writer; - private final IcebergFilesCommitter filesCommitter; - private final List results = Lists.newArrayList(); + private IcebergFilesCommitter filesCommitter; + private List results = Lists.newArrayList(); private String commitUser = UUID.randomUUID().toString(); + private long checkpointId; private final DataTypeChangeEventHandler dataTypeChangeEventHandler; @@ -76,6 +77,7 @@ public IcebergSinkWriter( tryCreateRecordWriter(); if (Objects.nonNull(states) && !states.isEmpty()) { this.commitUser = states.get(0).getCommitUser(); + this.checkpointId = states.get(0).getCheckpointId(); preCommit(states); } } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java index 06b48591df1..2be206ebb6e 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergRecordWriter.java @@ -54,7 +54,7 @@ public class IcebergRecordWriter implements RecordWriter { private final List writerResults; private TaskWriter writer; private RowConverter recordConverter; - private final IcebergWriterFactory writerFactory; + private IcebergWriterFactory writerFactory; public IcebergRecordWriter(Table table, IcebergWriterFactory writerFactory, SinkConfig config) { this.config = config; @@ -122,7 +122,12 @@ private void changeColumn( } } - /** apply schema update */ + /** + * apply schema update + * + * @param updates + * @return + */ private void applySchemaUpdate(SchemaChangeWrapper updates) { // complete the current file flush(); @@ -164,4 +169,7 @@ private void flush() { table.spec().partitionType())); writer = null; } + + @Override + public void close() {} } diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java index 2ee7c3d6d74..67809088ef7 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/writer/IcebergWriterFactory.java @@ -40,6 +40,9 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import lombok.extern.slf4j.Slf4j; import java.util.List; @@ -55,6 +58,7 @@ @Slf4j public class IcebergWriterFactory { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); private final IcebergTableLoader tableLoader; private final SinkConfig config; diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java index c56f3f2f00e..7a2fdf9d4ff 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java @@ -65,12 +65,12 @@ public class IcebergSource private static final long serialVersionUID = 4343414808223919870L; - private final SourceConfig sourceConfig; - private final Schema tableSchema; - private final Schema projectedSchema; - private final SeaTunnelRowType seaTunnelRowType; + private SourceConfig sourceConfig; + private Schema tableSchema; + private Schema projectedSchema; + private SeaTunnelRowType seaTunnelRowType; private JobContext jobContext; - private final CatalogTable catalogTable; + private CatalogTable catalogTable; public IcebergSource(ReadonlyConfig config, CatalogTable catalogTable) { this.sourceConfig = SourceConfig.loadConfig(config); diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java index 01343a119f6..6c99eb409c1 100644 --- a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/utils/SchemaUtils.java @@ -40,6 +40,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.UpdateSchema; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; @@ -105,11 +106,21 @@ public static Table autoCreateTable( SinkConfig config = new SinkConfig(readonlyConfig); // build auto create table Map options = new HashMap<>(table.getOptions()); + options.put(TableProperties.FORMAT_VERSION, "2"); // override options.putAll(config.getAutoCreateProps()); return createTable(catalog, toIcebergTableIdentifier(tablePath), config, schema, options); } + /** + * For local test + * + * @param catalog + * @param tableIdentifier + * @param config + * @param rowType + * @return + */ public static Table autoCreateTable( Catalog catalog, TableIdentifier tableIdentifier, @@ -169,7 +180,7 @@ private static Table createTable( Optional pkId = structType.fields().stream() .filter(nestedField -> nestedField.name().equals(pk)) - .map(Types.NestedField::fieldId) + .map(nestedField -> nestedField.fieldId()) .findFirst(); if (!pkId.isPresent()) { throw new IllegalArgumentException( @@ -185,14 +196,23 @@ private static Table createTable( structType .fields() .forEach( - field -> - fields.add( - identifierFieldIds.contains(field.fieldId()) - ? field.asRequired() - : field.asOptional())); + field -> { + fields.add( + identifierFieldIds.contains(field.fieldId()) + ? field.asRequired() + : field.asOptional()); + }); return new Schema(fields, identifierFieldIds); } + public static TableIdentifier toIcebergTableIdentifierFromCatalogTable( + CatalogTable catalogTable) { + org.apache.seatunnel.api.table.catalog.TableIdentifier tableIdentifier = + catalogTable.getTableId(); + return TableIdentifier.of( + tableIdentifier.getDatabaseName(), tableIdentifier.getTableName()); + } + public static TableIdentifier toIcebergTableIdentifier(TablePath tablePath) { return TableIdentifier.of(tablePath.getDatabaseName(), tablePath.getTableName()); } @@ -201,7 +221,12 @@ public static TablePath toTablePath(TableIdentifier tableIdentifier) { return TablePath.of(tableIdentifier.namespace().toString(), tableIdentifier.name()); } - /** Commit table schema updates */ + /** + * Commit table schema updates + * + * @param table + * @param wrapper + */ private static void commitSchemaUpdates(Table table, SchemaChangeWrapper wrapper) { // get the latest schema in case another process updated it table.refresh(); @@ -224,7 +249,7 @@ private static void commitSchemaUpdates(Table table, SchemaChangeWrapper wrapper .collect(toList()); // Rename column name - List changeColumns = new ArrayList<>(wrapper.changeColumns()); + List changeColumns = wrapper.changeColumns().stream().collect(toList()); if (addColumns.isEmpty() && modifyColumns.isEmpty() @@ -269,7 +294,7 @@ public static SeaTunnelDataType toSeaTunnelType(String fieldName, Type type) return IcebergTypeMapper.mapping(fieldName, type); } - public static Type toIcebergType(SeaTunnelDataType rowType) { + public static Type toIcebergType(SeaTunnelDataType rowType) { return IcebergTypeMapper.toIcebergType(rowType); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java index c412ca92186..c2f2405ee00 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java @@ -20,7 +20,6 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; public interface JdbcCatalogOptions { Option BASE_URL = @@ -60,10 +59,7 @@ public interface JdbcCatalogOptions { + "For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'."); OptionRule.Builder BASE_RULE = - OptionRule.builder() - .required(BASE_URL) - .required(USERNAME, PASSWORD) - .optional(SCHEMA, JdbcOptions.DECIMAL_TYPE_NARROWING); + OptionRule.builder().required(BASE_URL).required(USERNAME, PASSWORD).optional(SCHEMA); Option TABLE_PREFIX = Options.key("tablePrefix") 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 ccbbfb509dd..5aa6dcd874d 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 @@ -25,7 +25,6 @@ import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; -import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleTypeMapper; @@ -72,32 +71,13 @@ public class OracleCatalog extends AbstractJdbcCatalog { + "ORDER BY \n" + " cols.column_id \n"; - private boolean decimalTypeNarrowing; - public OracleCatalog( String catalogName, String username, String pwd, JdbcUrlUtil.UrlInfo urlInfo, String defaultSchema) { - this( - catalogName, - username, - pwd, - urlInfo, - defaultSchema, - JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue()); - } - - public OracleCatalog( - String catalogName, - String username, - String pwd, - JdbcUrlUtil.UrlInfo urlInfo, - String defaultSchema, - boolean decimalTypeNarrowing) { super(catalogName, username, pwd, urlInfo, defaultSchema); - this.decimalTypeNarrowing = decimalTypeNarrowing; } @Override @@ -182,7 +162,7 @@ protected Column buildColumn(ResultSet resultSet) throws SQLException { .defaultValue(defaultValue) .comment(columnComment) .build(); - return new OracleTypeConverter(decimalTypeNarrowing).convert(typeDefine); + return OracleTypeConverter.INSTANCE.convert(typeDefine); } @Override @@ -203,8 +183,7 @@ private List listTables() { @Override public CatalogTable getTable(String sqlQuery) throws SQLException { Connection defaultConnection = getConnection(defaultUrl); - return CatalogUtils.getCatalogTable( - defaultConnection, sqlQuery, new OracleTypeMapper(decimalTypeNarrowing)); + return CatalogUtils.getCatalogTable(defaultConnection, sqlQuery, new OracleTypeMapper()); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java index 2b51d976212..7c90c79347a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java @@ -25,7 +25,6 @@ import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; -import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; import com.google.auto.service.AutoService; @@ -53,8 +52,7 @@ public Catalog createCatalog(String catalogName, ReadonlyConfig options) { options.get(JdbcCatalogOptions.USERNAME), options.get(JdbcCatalogOptions.PASSWORD), urlInfo, - options.get(JdbcCatalogOptions.SCHEMA), - options.get(JdbcOptions.DECIMAL_TYPE_NARROWING)); + options.get(JdbcCatalogOptions.SCHEMA)); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java index 070ef670af1..bb224c4624a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/utils/CatalogUtils.java @@ -272,15 +272,6 @@ public static CatalogTable getCatalogTable( throws SQLException { TableSchema.Builder schemaBuilder = TableSchema.builder(); Map unsupported = new LinkedHashMap<>(); - String tableName = null; - String databaseName = null; - String schemaName = null; - try { - tableName = metadata.getTableName(1); - databaseName = metadata.getCatalogName(1); - schemaName = metadata.getSchemaName(1); - } catch (SQLException ignored) { - } for (int index = 1; index <= metadata.getColumnCount(); index++) { try { Column column = columnConverter.apply(metadata, index); @@ -298,14 +289,8 @@ public static CatalogTable getCatalogTable( throw CommonError.getCatalogTableWithUnsupportedType("UNKNOWN", sqlQuery, unsupported); } String catalogName = "jdbc_catalog"; - databaseName = StringUtils.isBlank(databaseName) ? null : databaseName; - schemaName = StringUtils.isBlank(schemaName) ? null : schemaName; - TablePath tablePath = - StringUtils.isBlank(tableName) - ? TablePath.DEFAULT - : TablePath.of(databaseName, schemaName, tableName); return CatalogTable.of( - TableIdentifier.of(catalogName, tablePath), + TableIdentifier.of(catalogName, "default", "default", "default"), schemaBuilder.build(), new HashMap<>(), new ArrayList<>(), @@ -322,11 +307,11 @@ public static CatalogTable getCatalogTable( } /** + * @deprecated instead by {@link #getCatalogTable(Connection, String, JdbcDialectTypeMapper)} * @param connection * @param sqlQuery * @return * @throws SQLException - * @deprecated instead by {@link #getCatalogTable(Connection, String, JdbcDialectTypeMapper)} */ @Deprecated public static CatalogTable getCatalogTable(Connection connection, String sqlQuery) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java index 053ab71a413..dc379bb38a5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java @@ -43,8 +43,6 @@ public class JdbcConnectionConfig implements Serializable { public String xaDataSourceClassName; - public boolean decimalTypeNarrowing = JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue(); - public int maxCommitAttempts = JdbcOptions.MAX_COMMIT_ATTEMPTS.defaultValue(); public int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); @@ -83,8 +81,6 @@ public static JdbcConnectionConfig of(ReadonlyConfig config) { config.getOptional(JdbcOptions.USER).ifPresent(builder::username); config.getOptional(JdbcOptions.PASSWORD).ifPresent(builder::password); config.getOptional(JdbcOptions.PROPERTIES).ifPresent(builder::properties); - config.getOptional(JdbcOptions.DECIMAL_TYPE_NARROWING) - .ifPresent(builder::decimalTypeNarrowing); return builder.build(); } @@ -112,10 +108,6 @@ public int getMaxRetries() { return maxRetries; } - public boolean isDecimalTypeNarrowing() { - return decimalTypeNarrowing; - } - public Optional getUsername() { return Optional.ofNullable(username); } @@ -161,7 +153,6 @@ public static final class Builder { private boolean autoCommit = JdbcOptions.AUTO_COMMIT.defaultValue(); private int batchSize = JdbcOptions.BATCH_SIZE.defaultValue(); private String xaDataSourceClassName; - private boolean decimalTypeNarrowing = JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue(); private int maxCommitAttempts = JdbcOptions.MAX_COMMIT_ATTEMPTS.defaultValue(); private int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); private Map properties; @@ -192,11 +183,6 @@ public Builder connectionCheckTimeoutSeconds(int connectionCheckTimeoutSeconds) return this; } - public Builder decimalTypeNarrowing(boolean decimalTypeNarrowing) { - this.decimalTypeNarrowing = decimalTypeNarrowing; - return this; - } - public Builder maxRetries(int maxRetries) { this.maxRetries = maxRetries; return this; @@ -281,7 +267,6 @@ public JdbcConnectionConfig build() { jdbcConnectionConfig.transactionTimeoutSec = this.transactionTimeoutSec; jdbcConnectionConfig.maxCommitAttempts = this.maxCommitAttempts; jdbcConnectionConfig.xaDataSourceClassName = this.xaDataSourceClassName; - jdbcConnectionConfig.decimalTypeNarrowing = this.decimalTypeNarrowing; jdbcConnectionConfig.useKerberos = this.useKerberos; jdbcConnectionConfig.kerberosPrincipal = this.kerberosPrincipal; jdbcConnectionConfig.kerberosKeytabPath = this.kerberosKeytabPath; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java index 976650456b0..7f0ec48f365 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java @@ -101,13 +101,6 @@ public interface JdbcOptions { .defaultValue(false) .withDescription("generate sql using the database table"); - Option DECIMAL_TYPE_NARROWING = - Options.key("decimal_type_narrowing") - .booleanType() - .defaultValue(true) - .withDescription( - "decimal type narrowing, if true, the decimal type will be narrowed to the int or long type if without loss of precision. Only support for Oracle at now."); - Option XA_DATA_SOURCE_CLASS_NAME = Options.key("xa_data_source_class_name") .stringType() diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java index 09cc92d70e8..74c78013183 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceConfig.java @@ -42,7 +42,6 @@ public class JdbcSourceConfig implements Serializable { private double splitEvenDistributionFactorLowerBound; private int splitSampleShardingThreshold; private int splitInverseSamplingRate; - private boolean decimalTypeNarrowing; public static JdbcSourceConfig of(ReadonlyConfig config) { JdbcSourceConfig.Builder builder = JdbcSourceConfig.builder(); @@ -54,7 +53,7 @@ public static JdbcSourceConfig of(ReadonlyConfig config) { boolean isOldVersion = config.getOptional(JdbcOptions.QUERY).isPresent() && config.getOptional(JdbcOptions.PARTITION_COLUMN).isPresent(); - builder.useDynamicSplitter(!isOldVersion); + builder.useDynamicSplitter(isOldVersion ? false : true); builder.splitSize(config.get(JdbcSourceOptions.SPLIT_SIZE)); builder.splitEvenDistributionFactorUpperBound( @@ -65,8 +64,6 @@ public static JdbcSourceConfig of(ReadonlyConfig config) { config.get(JdbcSourceOptions.SPLIT_SAMPLE_SHARDING_THRESHOLD)); builder.splitInverseSamplingRate(config.get(JdbcSourceOptions.SPLIT_INVERSE_SAMPLING_RATE)); - builder.decimalTypeNarrowing(config.get(JdbcOptions.DECIMAL_TYPE_NARROWING)); - config.getOptional(JdbcSourceOptions.WHERE_CONDITION) .ifPresent( whereConditionClause -> { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSourceTableConfig.java index f0af8513776..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 @@ -85,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); } @@ -94,9 +96,6 @@ public static List of(ReadonlyConfig connectorConfig) { if (tableConfig.getPartitionNumber() == null) { tableConfig.setPartitionNumber(DEFAULT_PARTITION_NUMBER); } - tableConfig.setUseSelectCount( - connectorConfig.get(JdbcSourceOptions.USE_SELECT_COUNT)); - tableConfig.setSkipAnalyze(connectorConfig.get(JdbcSourceOptions.SKIP_ANALYZE)); }); if (tableList.size() > 1) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java index 8646d0d7e55..97a37c0621b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MySqlVersion.java @@ -18,27 +18,28 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql; public enum MySqlVersion { - V_5_5("5.5"), - V_5_6("5.6"), - V_5_7("5.7"), - V_8("8.0"), - V_8_1("8.1"), - V_8_2("8.2"), - V_8_3("8.3"), - V_8_4("8.4"); - - private final String versionPrefix; - - MySqlVersion(String versionPrefix) { - this.versionPrefix = versionPrefix; - } + V_5_5, + V_5_6, + V_5_7, + V_8, + V_8_4; public static MySqlVersion parse(String version) { if (version != null) { - for (MySqlVersion mySqlVersion : values()) { - if (version.startsWith(mySqlVersion.versionPrefix)) { - return mySqlVersion; - } + if (version.startsWith("5.5")) { + return V_5_5; + } + if (version.startsWith("5.6")) { + return V_5_6; + } + if (version.startsWith("5.7")) { + return V_5_7; + } + if (version.startsWith("8.0")) { + return V_8; + } + if (version.startsWith("8.4")) { + return V_8_4; } } throw new UnsupportedOperationException("Unsupported MySQL version: " + version); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java index 023fa949cf1..d359f3fef0d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverter.java @@ -86,16 +86,6 @@ public class OracleTypeConverter implements TypeConverter { public static final long BYTES_4GB = (long) Math.pow(2, 32); public static final OracleTypeConverter INSTANCE = new OracleTypeConverter(); - private final boolean decimalTypeNarrowing; - - public OracleTypeConverter() { - this(true); - } - - public OracleTypeConverter(boolean decimalTypeNarrowing) { - this.decimalTypeNarrowing = decimalTypeNarrowing; - } - @Override public String identifier() { return DatabaseIdentifier.ORACLE; @@ -129,14 +119,12 @@ public Column convert(BasicTypeDefine typeDefine) { if (scale <= 0) { int newPrecision = (int) (precision - scale); - if (newPrecision <= 18 && decimalTypeNarrowing) { - if (newPrecision == 1) { - builder.dataType(BasicType.BOOLEAN_TYPE); - } else if (newPrecision <= 9) { - builder.dataType(BasicType.INT_TYPE); - } else { - builder.dataType(BasicType.LONG_TYPE); - } + if (newPrecision == 1) { + builder.dataType(BasicType.BOOLEAN_TYPE); + } else if (newPrecision <= 9) { + builder.dataType(BasicType.INT_TYPE); + } else if (newPrecision <= 18) { + builder.dataType(BasicType.LONG_TYPE); } else if (newPrecision < 38) { builder.dataType(new DecimalType(newPrecision, 0)); builder.columnLength((long) newPrecision); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java index bbdd19af8eb..ce5ef8af889 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeMapper.java @@ -20,7 +20,6 @@ import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.connectors.seatunnel.common.source.TypeDefineUtils; -import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; import lombok.extern.slf4j.Slf4j; @@ -32,19 +31,9 @@ @Slf4j public class OracleTypeMapper implements JdbcDialectTypeMapper { - private final boolean decimalTypeNarrowing; - - public OracleTypeMapper() { - this(JdbcOptions.DECIMAL_TYPE_NARROWING.defaultValue()); - } - - public OracleTypeMapper(boolean decimalTypeNarrowing) { - this.decimalTypeNarrowing = decimalTypeNarrowing; - } - @Override public Column mappingColumn(BasicTypeDefine typeDefine) { - return new OracleTypeConverter(decimalTypeNarrowing).convert(typeDefine); + return OracleTypeConverter.INSTANCE.convert(typeDefine); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java index 9f03f652f16..0e8814726c5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferReducedBatchStatementExecutor.java @@ -96,17 +96,11 @@ public void executeBatch() throws SQLException { @Override public void closeStatements() throws SQLException { - try { - if (!buffer.isEmpty()) { - executeBatch(); - } - } finally { - if (!buffer.isEmpty()) { - buffer.clear(); - } - upsertExecutor.closeStatements(); - deleteExecutor.closeStatements(); + if (!buffer.isEmpty()) { + executeBatch(); } + upsertExecutor.closeStatements(); + deleteExecutor.closeStatements(); } private boolean changeFlag(RowKind rowKind) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java index 0ebef85f52e..86995240d79 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BufferedBatchStatementExecutor.java @@ -57,15 +57,9 @@ public void executeBatch() throws SQLException { @Override public void closeStatements() throws SQLException { - try { - if (!buffer.isEmpty()) { - executeBatch(); - } - } finally { - if (!buffer.isEmpty()) { - buffer.clear(); - } - statementExecutor.closeStatements(); + if (!buffer.isEmpty()) { + executeBatch(); } + statementExecutor.closeStatements(); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java index 9cf8b95863f..8a4b2a579c7 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/InsertOrUpdateBatchStatementExecutor.java @@ -112,16 +112,13 @@ public void executeBatch() throws SQLException { @Override public void closeStatements() throws SQLException { - try { - if (!submitted) { - executeBatch(); - } - } finally { - for (PreparedStatement statement : - Arrays.asList(existStatement, insertStatement, updateStatement)) { - if (statement != null) { - statement.close(); - } + if (!submitted) { + executeBatch(); + } + for (PreparedStatement statement : + Arrays.asList(existStatement, insertStatement, updateStatement)) { + if (statement != null) { + statement.close(); } } } 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 6eabba1edc1..860131041a9 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 @@ -34,7 +34,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; -import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceTableConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; @@ -140,8 +139,6 @@ public static Map getTables( .partitionNumber(tableConfig.getPartitionNumber()) .partitionStart(tableConfig.getPartitionStart()) .partitionEnd(tableConfig.getPartitionEnd()) - .useSelectCount(tableConfig.getUseSelectCount()) - .skipAnalyze(tableConfig.getSkipAnalyze()) .catalogTable(catalogTable) .build(); @@ -398,8 +395,6 @@ private static ReadonlyConfig extractCatalogConfig(JdbcConnectionConfig config) .ifPresent(val -> catalogConfig.put(JdbcCatalogOptions.PASSWORD.key(), val)); Optional.ofNullable(config.getCompatibleMode()) .ifPresent(val -> catalogConfig.put(JdbcCatalogOptions.COMPATIBLE_MODE.key(), val)); - catalogConfig.put( - JdbcOptions.DECIMAL_TYPE_NARROWING.key(), config.isDecimalTypeNarrowing()); return ReadonlyConfig.fromMap(catalogConfig); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java deleted file mode 100644 index cec02118928..00000000000 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlVersionTest.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class MysqlVersionTest { - - @Test - public void testMysqlVersionParse() { - Assertions.assertEquals(MySqlVersion.V_5_5, MySqlVersion.parse("5.5.0")); - Assertions.assertEquals(MySqlVersion.V_5_5, MySqlVersion.parse("5.5.1")); - Assertions.assertEquals(MySqlVersion.V_5_5, MySqlVersion.parse("5.5.12")); - - Assertions.assertEquals(MySqlVersion.V_5_6, MySqlVersion.parse("5.6.0")); - Assertions.assertEquals(MySqlVersion.V_5_6, MySqlVersion.parse("5.6.1")); - Assertions.assertEquals(MySqlVersion.V_5_6, MySqlVersion.parse("5.6.12")); - - Assertions.assertEquals(MySqlVersion.V_5_7, MySqlVersion.parse("5.7.0")); - Assertions.assertEquals(MySqlVersion.V_5_7, MySqlVersion.parse("5.7.1")); - Assertions.assertEquals(MySqlVersion.V_5_7, MySqlVersion.parse("5.7.12")); - - Assertions.assertEquals(MySqlVersion.V_8, MySqlVersion.parse("8.0.0")); - Assertions.assertEquals(MySqlVersion.V_8, MySqlVersion.parse("8.0.1")); - Assertions.assertEquals(MySqlVersion.V_8, MySqlVersion.parse("8.0.12")); - - Assertions.assertEquals(MySqlVersion.V_8_1, MySqlVersion.parse("8.1.0")); - Assertions.assertEquals(MySqlVersion.V_8_1, MySqlVersion.parse("8.1.4")); - Assertions.assertEquals(MySqlVersion.V_8_1, MySqlVersion.parse("8.1.14")); - - Assertions.assertEquals(MySqlVersion.V_8_2, MySqlVersion.parse("8.2.0")); - Assertions.assertEquals(MySqlVersion.V_8_2, MySqlVersion.parse("8.2.4")); - Assertions.assertEquals(MySqlVersion.V_8_2, MySqlVersion.parse("8.2.14")); - - Assertions.assertEquals(MySqlVersion.V_8_3, MySqlVersion.parse("8.3.0")); - Assertions.assertEquals(MySqlVersion.V_8_3, MySqlVersion.parse("8.3.4")); - Assertions.assertEquals(MySqlVersion.V_8_3, MySqlVersion.parse("8.3.14")); - - Assertions.assertEquals(MySqlVersion.V_8_4, MySqlVersion.parse("8.4.0")); - Assertions.assertEquals(MySqlVersion.V_8_4, MySqlVersion.parse("8.4.4")); - Assertions.assertEquals(MySqlVersion.V_8_4, MySqlVersion.parse("8.4.14")); - } -} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java index d4a8defddab..26238bad303 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/oracle/OracleTypeConverterTest.java @@ -36,14 +36,12 @@ public class OracleTypeConverterTest { - private static final OracleTypeConverter INSTANCE = new OracleTypeConverter(); - @Test public void testConvertUnsupported() { BasicTypeDefine typeDefine = BasicTypeDefine.builder().name("test").columnType("aaa").dataType("aaa").build(); try { - INSTANCE.convert(typeDefine); + OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.fail(); } catch (SeaTunnelRuntimeException e) { // ignore @@ -52,113 +50,6 @@ public void testConvertUnsupported() { } } - @Test - public void testConvertNumberWithoutDecimalTypeNarrowing() { - OracleTypeConverter converter = new OracleTypeConverter(false); - - BasicTypeDefine typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number") - .dataType("number") - .build(); - Column column = converter.convert(typeDefine); - - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number(38,127)") - .dataType("number") - .precision(38L) - .scale(127) - .build(); - column = converter.convert(typeDefine); - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number") - .dataType("number") - .scale(0) - .build(); - column = converter.convert(typeDefine); - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number(1,0)") - .dataType("number") - .precision(1L) - .scale(0) - .build(); - column = converter.convert(typeDefine); - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(1, 0), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number(8,0)") - .dataType("number") - .precision(8L) - .scale(0) - .build(); - column = converter.convert(typeDefine); - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(8, 0), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number(18,0)") - .dataType("number") - .precision(18L) - .scale(0) - .build(); - column = converter.convert(typeDefine); - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(18, 0), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number(38,0)") - .dataType("number") - .precision(38L) - .scale(0) - .build(); - column = converter.convert(typeDefine); - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - typeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number(39,0)") - .dataType("number") - .precision(39L) - .scale(0) - .build(); - column = converter.convert(typeDefine); - Assertions.assertEquals(typeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); - Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - } - @Test public void testConvertInteger() { BasicTypeDefine typeDefine = @@ -167,24 +58,10 @@ public void testConvertInteger() { .columnType("integer") .dataType("integer") .build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); - - // generated by int/smallint type in oracle create table sql - BasicTypeDefine numberTypeDefine = - BasicTypeDefine.builder() - .name("test") - .columnType("number") - .dataType("number") - .precision(null) - .scale(0) - .build(); - column = INSTANCE.convert(numberTypeDefine); - Assertions.assertEquals(numberTypeDefine.getName(), column.getName()); - Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); - Assertions.assertEquals(numberTypeDefine.getColumnType(), column.getSourceType()); } @Test @@ -195,7 +72,7 @@ public void testConvertNumber() { .columnType("number") .dataType("number") .build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); @@ -209,7 +86,7 @@ public void testConvertNumber() { .precision(38L) .scale(127) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -221,7 +98,7 @@ public void testConvertNumber() { .dataType("number") .scale(0) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -234,7 +111,7 @@ public void testConvertNumber() { .precision(1L) .scale(0) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.BOOLEAN_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -247,7 +124,7 @@ public void testConvertNumber() { .precision(8L) .scale(0) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.INT_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -260,7 +137,7 @@ public void testConvertNumber() { .precision(18L) .scale(0) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.LONG_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -273,7 +150,7 @@ public void testConvertNumber() { .precision(38L) .scale(0) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -286,7 +163,7 @@ public void testConvertNumber() { .precision(39L) .scale(0) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -300,7 +177,7 @@ public void testConvertFloat() { .columnType("float") .dataType("float") .build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 18), column.getDataType()); @@ -312,7 +189,7 @@ public void testConvertFloat() { .columnType("binary_float") .dataType("binary_float") .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.FLOAT_TYPE, column.getDataType()); @@ -320,7 +197,7 @@ public void testConvertFloat() { typeDefine = BasicTypeDefine.builder().name("test").columnType("real").dataType("real").build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.FLOAT_TYPE, column.getDataType()); @@ -335,7 +212,7 @@ public void testConvertDouble() { .columnType("binary_double") .dataType("binary_double") .build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.DOUBLE_TYPE, column.getDataType()); @@ -351,7 +228,7 @@ public void testConvertChar() { .dataType("char") .length(1L) .build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -365,7 +242,7 @@ public void testConvertChar() { .dataType("nchar") .length(1L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -379,7 +256,7 @@ public void testConvertChar() { .dataType("varchar") .length(1L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -393,7 +270,7 @@ public void testConvertChar() { .dataType("varchar2") .length(1L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -407,7 +284,7 @@ public void testConvertChar() { .dataType("nvarchar2") .length(1L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -420,7 +297,7 @@ public void testConvertChar() { .columnType("rowid") .dataType("rowid") .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -434,7 +311,7 @@ public void testConvertChar() { .dataType("xmltype") .length(1L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -448,7 +325,7 @@ public void testConvertChar() { .dataType("sys.xmltype") .length(1L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -462,7 +339,7 @@ public void testConvertChar() { .dataType("long") .length(1L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -471,7 +348,7 @@ public void testConvertChar() { typeDefine = BasicTypeDefine.builder().name("test").columnType("clob").dataType("clob").build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -484,7 +361,7 @@ public void testConvertChar() { .columnType("nclob") .dataType("nclob") .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.STRING_TYPE, column.getDataType()); @@ -496,7 +373,7 @@ public void testConvertChar() { public void testConvertBytes() { BasicTypeDefine typeDefine = BasicTypeDefine.builder().name("test").columnType("blob").dataType("blob").build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -505,7 +382,7 @@ public void testConvertBytes() { typeDefine = BasicTypeDefine.builder().name("test").columnType("raw").dataType("raw").build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -519,7 +396,7 @@ public void testConvertBytes() { .dataType("raw") .length(10L) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -532,7 +409,7 @@ public void testConvertBytes() { .columnType("long raw") .dataType("long raw") .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(PrimitiveByteArrayType.INSTANCE, column.getDataType()); @@ -544,7 +421,7 @@ public void testConvertBytes() { public void testConvertDatetime() { BasicTypeDefine typeDefine = BasicTypeDefine.builder().name("test").columnType("date").dataType("date").build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -558,7 +435,7 @@ public void testConvertDatetime() { .dataType("timestamp") .scale(6) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -572,7 +449,7 @@ public void testConvertDatetime() { .dataType("timestamp with time zone") .scale(6) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -586,7 +463,7 @@ public void testConvertDatetime() { .dataType("timestamp with local time zone") .scale(6) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(LocalTimeType.LOCAL_DATE_TIME_TYPE, column.getDataType()); @@ -605,7 +482,7 @@ public void testReconvertUnsupported() { null, null); try { - INSTANCE.reconvert(column); + OracleTypeConverter.INSTANCE.reconvert(column); Assertions.fail(); } catch (SeaTunnelRuntimeException e) { // ignore @@ -619,7 +496,7 @@ public void testReconvertBoolean() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.BOOLEAN_TYPE).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("%s(%s)", OracleTypeConverter.ORACLE_NUMBER, 1), @@ -632,7 +509,7 @@ public void testReconvertBoolean() { public void testReconvertByte() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.BYTE_TYPE).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -643,7 +520,7 @@ public void testReconvertShort() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.SHORT_TYPE).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -653,7 +530,7 @@ public void testReconvertShort() { public void testReconvertInt() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.INT_TYPE).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -663,7 +540,7 @@ public void testReconvertInt() { public void testReconvertLong() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.LONG_TYPE).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_INTEGER, typeDefine.getDataType()); @@ -674,7 +551,7 @@ public void testReconvertFloat() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.FLOAT_TYPE).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( OracleTypeConverter.ORACLE_BINARY_FLOAT, typeDefine.getColumnType()); @@ -686,7 +563,7 @@ public void testReconvertDouble() { Column column = PhysicalColumn.builder().name("test").dataType(BasicType.DOUBLE_TYPE).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( OracleTypeConverter.ORACLE_BINARY_DOUBLE, typeDefine.getColumnType()); @@ -698,7 +575,7 @@ public void testReconvertDecimal() { Column column = PhysicalColumn.builder().name("test").dataType(new DecimalType(0, 0)).build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format( @@ -711,7 +588,7 @@ public void testReconvertDecimal() { column = PhysicalColumn.builder().name("test").dataType(new DecimalType(10, 2)).build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("%s(%s,%s)", OracleTypeConverter.ORACLE_NUMBER, 10, 2), @@ -728,7 +605,7 @@ public void testReconvertBytes() { .columnLength(null) .build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getDataType()); @@ -740,7 +617,7 @@ public void testReconvertBytes() { .columnLength(2000L) .build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("%s(%s)", OracleTypeConverter.ORACLE_RAW, column.getColumnLength()), @@ -754,7 +631,7 @@ public void testReconvertBytes() { .columnLength(BYTES_2GB) .build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getDataType()); @@ -766,7 +643,7 @@ public void testReconvertBytes() { .columnLength(BYTES_2GB + 1) .build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_BLOB, typeDefine.getDataType()); @@ -781,7 +658,7 @@ public void testReconvertString() { .columnLength(null) .build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals("VARCHAR2(4000)", typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_VARCHAR2, typeDefine.getDataType()); @@ -793,7 +670,7 @@ public void testReconvertString() { .columnLength(2000L) .build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format( @@ -808,7 +685,7 @@ public void testReconvertString() { .columnLength(4000L) .build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format( @@ -823,7 +700,7 @@ public void testReconvertString() { .columnLength(40001L) .build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_CLOB, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_CLOB, typeDefine.getDataType()); @@ -837,7 +714,7 @@ public void testReconvertDate() { .dataType(LocalTimeType.LOCAL_DATE_TYPE) .build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals(OracleTypeConverter.ORACLE_DATE, typeDefine.getColumnType()); Assertions.assertEquals(OracleTypeConverter.ORACLE_DATE, typeDefine.getDataType()); @@ -851,7 +728,7 @@ public void testReconvertDatetime() { .dataType(LocalTimeType.LOCAL_DATE_TIME_TYPE) .build(); - BasicTypeDefine typeDefine = INSTANCE.reconvert(column); + BasicTypeDefine typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( OracleTypeConverter.ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE, @@ -867,7 +744,7 @@ public void testReconvertDatetime() { .scale(3) .build(); - typeDefine = INSTANCE.reconvert(column); + typeDefine = OracleTypeConverter.INSTANCE.reconvert(column); Assertions.assertEquals(column.getName(), typeDefine.getName()); Assertions.assertEquals( String.format("TIMESTAMP(%s) WITH LOCAL TIME ZONE", column.getScale()), @@ -888,7 +765,7 @@ public void testNumberWithNegativeScale() { .precision(38L) .scale(-1) .build(); - Column column = INSTANCE.convert(typeDefine); + Column column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(38, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -901,7 +778,7 @@ public void testNumberWithNegativeScale() { .precision(5L) .scale(-2) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.INT_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -914,7 +791,7 @@ public void testNumberWithNegativeScale() { .precision(9L) .scale(-2) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(BasicType.LONG_TYPE, column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); @@ -927,7 +804,7 @@ public void testNumberWithNegativeScale() { .precision(14L) .scale(-11) .build(); - column = INSTANCE.convert(typeDefine); + column = OracleTypeConverter.INSTANCE.convert(typeDefine); Assertions.assertEquals(typeDefine.getName(), column.getName()); Assertions.assertEquals(new DecimalType(25, 0), column.getDataType()); Assertions.assertEquals(typeDefine.getColumnType(), column.getSourceType()); diff --git a/seatunnel-connectors-v2/connector-kafka/pom.xml b/seatunnel-connectors-v2/connector-kafka/pom.xml index 9ffff7dbc0c..3251de5a8e7 100644 --- a/seatunnel-connectors-v2/connector-kafka/pom.xml +++ b/seatunnel-connectors-v2/connector-kafka/pom.xml @@ -77,11 +77,6 @@ seatunnel-format-avro ${project.version} - - org.apache.seatunnel - seatunnel-format-protobuf - ${project.version} - diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index a907c9bc212..6d6f43dc4c2 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -177,24 +177,10 @@ public class Config { .defaultValue(KafkaSemantics.NON) .withDescription( "Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON."); - public static final Option>> TABLE_LIST = Options.key("table_list") .type(new TypeReference>>() {}) .noDefaultValue() .withDescription( "Topic list config. You can configure only one `table_list` or one `topic` at the same time"); - - public static final Option PROTOBUF_SCHEMA = - Options.key("protobuf_schema") - .stringType() - .noDefaultValue() - .withDescription( - "Data serialization method protobuf metadata, used to parse protobuf data."); - - public static final Option PROTOBUF_MESSAGE_NAME = - Options.key("protobuf_message_name") - .stringType() - .noDefaultValue() - .withDescription("Parsing entity class names from Protobuf data."); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java index a877c76c37d..f02cebcbe35 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java @@ -26,6 +26,5 @@ public enum MessageFormat { COMPATIBLE_KAFKA_CONNECT_JSON, OGG_JSON, AVRO, - MAXWELL_JSON, - PROTOBUF + MAXWELL_JSON } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java index 2fb251571f1..d4a77e74b93 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.serialize; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.SerializationSchema; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -34,7 +33,6 @@ import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import org.apache.seatunnel.format.json.maxwell.MaxWellJsonSerializationSchema; import org.apache.seatunnel.format.json.ogg.OggJsonSerializationSchema; -import org.apache.seatunnel.format.protobuf.ProtobufSerializationSchema; import org.apache.seatunnel.format.text.TextSerializationSchema; import org.apache.kafka.clients.producer.ProducerRecord; @@ -48,9 +46,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_MESSAGE_NAME; -import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_SCHEMA; - @RequiredArgsConstructor public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer { private final Function topicExtractor; @@ -72,17 +67,13 @@ public ProducerRecord serializeRow(SeaTunnelRow row) { } public static DefaultSeaTunnelRowSerializer create( - String topic, - SeaTunnelRowType rowType, - MessageFormat format, - String delimiter, - ReadonlyConfig pluginConfig) { + String topic, SeaTunnelRowType rowType, MessageFormat format, String delimiter) { return new DefaultSeaTunnelRowSerializer( topicExtractor(topic, rowType, format), partitionExtractor(null), timestampExtractor(), - keyExtractor(null, rowType, format, delimiter, pluginConfig), - valueExtractor(rowType, format, delimiter, pluginConfig), + keyExtractor(null, rowType, format, delimiter), + valueExtractor(rowType, format, delimiter), headersExtractor()); } @@ -91,14 +82,13 @@ public static DefaultSeaTunnelRowSerializer create( Integer partition, SeaTunnelRowType rowType, MessageFormat format, - String delimiter, - ReadonlyConfig pluginConfig) { + String delimiter) { return new DefaultSeaTunnelRowSerializer( topicExtractor(topic, rowType, format), partitionExtractor(partition), timestampExtractor(), - keyExtractor(null, rowType, format, delimiter, pluginConfig), - valueExtractor(rowType, format, delimiter, pluginConfig), + keyExtractor(null, rowType, format, delimiter), + valueExtractor(rowType, format, delimiter), headersExtractor()); } @@ -107,14 +97,13 @@ public static DefaultSeaTunnelRowSerializer create( List keyFields, SeaTunnelRowType rowType, MessageFormat format, - String delimiter, - ReadonlyConfig pluginConfig) { + String delimiter) { return new DefaultSeaTunnelRowSerializer( topicExtractor(topic, rowType, format), partitionExtractor(null), timestampExtractor(), - keyExtractor(keyFields, rowType, format, delimiter, pluginConfig), - valueExtractor(rowType, format, delimiter, pluginConfig), + keyExtractor(keyFields, rowType, format, delimiter), + valueExtractor(rowType, format, delimiter), headersExtractor()); } @@ -168,8 +157,7 @@ private static Function keyExtractor( List keyFields, SeaTunnelRowType rowType, MessageFormat format, - String delimiter, - ReadonlyConfig pluginConfig) { + String delimiter) { if (MessageFormat.COMPATIBLE_DEBEZIUM_JSON.equals(format)) { CompatibleDebeziumJsonSerializationSchema serializationSchema = new CompatibleDebeziumJsonSerializationSchema(rowType, true); @@ -184,17 +172,14 @@ private static Function keyExtractor( Function keyRowExtractor = createKeyRowExtractor(keyType, rowType); SerializationSchema serializationSchema = - createSerializationSchema(keyType, format, delimiter, true, pluginConfig); + createSerializationSchema(keyType, format, delimiter, true); return row -> serializationSchema.serialize(keyRowExtractor.apply(row)); } private static Function valueExtractor( - SeaTunnelRowType rowType, - MessageFormat format, - String delimiter, - ReadonlyConfig pluginConfig) { + SeaTunnelRowType rowType, MessageFormat format, String delimiter) { SerializationSchema serializationSchema = - createSerializationSchema(rowType, format, delimiter, false, pluginConfig); + createSerializationSchema(rowType, format, delimiter, false); return row -> serializationSchema.serialize(row); } @@ -227,11 +212,7 @@ private static Function createKeyRowExtractor( } private static SerializationSchema createSerializationSchema( - SeaTunnelRowType rowType, - MessageFormat format, - String delimiter, - boolean isKey, - ReadonlyConfig pluginConfig) { + SeaTunnelRowType rowType, MessageFormat format, String delimiter, boolean isKey) { switch (format) { case JSON: return new JsonSerializationSchema(rowType); @@ -252,11 +233,6 @@ private static SerializationSchema createSerializationSchema( return new CompatibleDebeziumJsonSerializationSchema(rowType, isKey); case AVRO: return new AvroSerializationSchema(rowType); - case PROTOBUF: - String protobufMessageName = pluginConfig.get(PROTOBUF_MESSAGE_NAME); - String protobufSchema = pluginConfig.get(PROTOBUF_SCHEMA); - return new ProtobufSerializationSchema( - rowType, protobufMessageName, protobufSchema); default: throw new SeaTunnelJsonFormatException( CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java index 6639a34a0bd..44676f29d4c 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java @@ -188,21 +188,15 @@ private SeaTunnelRowSerializer getSerializer( getPartitionKeyFields(pluginConfig, seaTunnelRowType), seaTunnelRowType, messageFormat, - delimiter, - pluginConfig); + delimiter); } if (pluginConfig.get(PARTITION) != null) { return DefaultSeaTunnelRowSerializer.create( - topic, - pluginConfig.get(PARTITION), - seaTunnelRowType, - messageFormat, - delimiter, - pluginConfig); + topic, pluginConfig.get(PARTITION), seaTunnelRowType, messageFormat, delimiter); } // By default, all partitions are sent randomly return DefaultSeaTunnelRowSerializer.create( - topic, Arrays.asList(), seaTunnelRowType, messageFormat, delimiter, pluginConfig); + topic, Arrays.asList(), seaTunnelRowType, messageFormat, delimiter); } private KafkaSemantics getKafkaSemantics(ReadonlyConfig pluginConfig) { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java index c129fbdc93c..8ce9dbb870a 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java @@ -37,6 +37,7 @@ public class ConsumerMetadata implements Serializable { private String topic; private boolean isPattern = false; private Properties properties; + private String consumerGroup; private StartMode startMode = StartMode.GROUP_OFFSETS; private Map specificStartOffsets; private Long startOffsetsTimestamp; diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java new file mode 100644 index 00000000000..99b6baeaf16 --- /dev/null +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaConsumerThread.java @@ -0,0 +1,111 @@ +/* + * 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.kafka.source; + +import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorException; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import java.util.Properties; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +public class KafkaConsumerThread implements Runnable { + + private final KafkaConsumer consumer; + private static final String CLIENT_ID_PREFIX = "seatunnel"; + private final ConsumerMetadata metadata; + + private final LinkedBlockingQueue>> tasks; + + public KafkaConsumerThread(KafkaSourceConfig kafkaSourceConfig, ConsumerMetadata metadata) { + this.metadata = metadata; + this.tasks = new LinkedBlockingQueue<>(); + this.consumer = + initConsumer( + kafkaSourceConfig.getBootstrap(), + metadata.getConsumerGroup(), + kafkaSourceConfig.getProperties(), + !kafkaSourceConfig.isCommitOnCheckpoint()); + } + + @Override + public void run() { + try { + while (!Thread.currentThread().isInterrupted()) { + try { + Consumer> task = tasks.poll(1, TimeUnit.SECONDS); + if (task != null) { + task.accept(consumer); + } + } catch (Exception e) { + throw new KafkaConnectorException( + KafkaConnectorErrorCode.CONSUME_THREAD_RUN_ERROR, e); + } + } + } finally { + try { + if (consumer != null) { + consumer.close(); + } + } catch (Throwable t) { + throw new KafkaConnectorException(KafkaConnectorErrorCode.CONSUMER_CLOSE_FAILED, t); + } + } + } + + public LinkedBlockingQueue>> getTasks() { + return tasks; + } + + private KafkaConsumer initConsumer( + String bootstrapServer, + String consumerGroup, + Properties properties, + boolean autoCommit) { + Properties props = new Properties(); + properties.forEach( + (key, value) -> props.setProperty(String.valueOf(key), String.valueOf(value))); + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, consumerGroup); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer); + if (this.metadata.getProperties().get("client.id") == null) { + props.setProperty( + ConsumerConfig.CLIENT_ID_CONFIG, + CLIENT_ID_PREFIX + "-consumer-" + this.hashCode()); + } else { + props.setProperty( + ConsumerConfig.CLIENT_ID_CONFIG, + this.metadata.getProperties().get("client.id").toString()); + } + props.setProperty( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + props.setProperty( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, String.valueOf(autoCommit)); + + // Disable auto create topics feature + props.setProperty(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); + return new KafkaConsumer<>(props); + } +} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java deleted file mode 100644 index 8bca82999c7..00000000000 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaPartitionSplitReader.java +++ /dev/null @@ -1,458 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.kafka.source; - -import org.apache.seatunnel.shade.com.google.common.base.Preconditions; - -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.common.utils.TemporaryClassLoaderContext; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitReader; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitsAddition; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitsChange; -import org.apache.seatunnel.connectors.seatunnel.kafka.config.StartMode; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.WakeupException; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.StringJoiner; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -public class KafkaPartitionSplitReader - implements SplitReader, KafkaSourceSplit> { - - private static final Logger LOG = LoggerFactory.getLogger(KafkaPartitionSplitReader.class); - - private static final long POLL_TIMEOUT = 10000L; - private static final String CLIENT_ID_PREFIX = "seatunnel"; - private final KafkaSourceConfig kafkaSourceConfig; - - private final KafkaConsumer consumer; - - private final Map stoppingOffsets; - - private final String groupId; - - private final Set emptySplits = new HashSet<>(); - - public KafkaPartitionSplitReader( - KafkaSourceConfig kafkaSourceConfig, SourceReader.Context context) { - this.kafkaSourceConfig = kafkaSourceConfig; - this.consumer = initConsumer(kafkaSourceConfig, context.getIndexOfSubtask()); - this.stoppingOffsets = new HashMap<>(); - this.groupId = - kafkaSourceConfig.getProperties().getProperty(ConsumerConfig.GROUP_ID_CONFIG); - } - - @Override - public RecordsWithSplitIds> fetch() throws IOException { - ConsumerRecords consumerRecords; - try { - consumerRecords = consumer.poll(Duration.ofMillis(POLL_TIMEOUT)); - } catch (WakeupException | IllegalStateException e) { - // IllegalStateException will be thrown if the consumer is not assigned any partitions. - // This happens if all assigned partitions are invalid or empty (starting offset >= - // stopping offset). We just mark empty partitions as finished and return an empty - // record container, and this consumer will be closed by SplitFetcherManager. - KafkaPartitionSplitRecords recordsBySplits = - new KafkaPartitionSplitRecords(ConsumerRecords.empty()); - markEmptySplitsAsFinished(recordsBySplits); - return recordsBySplits; - } - KafkaPartitionSplitRecords recordsBySplits = - new KafkaPartitionSplitRecords(consumerRecords); - List finishedPartitions = new ArrayList<>(); - for (TopicPartition tp : consumerRecords.partitions()) { - long stoppingOffset = getStoppingOffset(tp); - final List> recordsFromPartition = - consumerRecords.records(tp); - - if (recordsFromPartition.size() > 0) { - final ConsumerRecord lastRecord = - recordsFromPartition.get(recordsFromPartition.size() - 1); - - // After processing a record with offset of "stoppingOffset - 1", the split reader - // should not continue fetching because the record with stoppingOffset may not - // exist. Keep polling will just block forever. - if (lastRecord.offset() >= stoppingOffset - 1) { - recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); - finishSplitAtRecord( - tp, - stoppingOffset, - lastRecord.offset(), - finishedPartitions, - recordsBySplits); - } - } - } - - markEmptySplitsAsFinished(recordsBySplits); - - if (!finishedPartitions.isEmpty()) { - unassignPartitions(finishedPartitions); - } - - return recordsBySplits; - } - - private void finishSplitAtRecord( - TopicPartition tp, - long stoppingOffset, - long currentOffset, - List finishedPartitions, - KafkaPartitionSplitRecords recordsBySplits) { - LOG.debug( - "{} has reached stopping offset {}, current offset is {}", - tp, - stoppingOffset, - currentOffset); - finishedPartitions.add(tp); - recordsBySplits.addFinishedSplit(tp.toString()); - } - - private void markEmptySplitsAsFinished(KafkaPartitionSplitRecords recordsBySplits) { - // Some splits are discovered as empty when handling split additions. These splits should be - // added to finished splits to clean up states in split fetcher and source reader. - if (!emptySplits.isEmpty()) { - recordsBySplits.finishedSplits.addAll(emptySplits); - emptySplits.clear(); - } - } - - @Override - public void handleSplitsChanges(SplitsChange splitsChange) { - // Get all the partition assignments and stopping offsets. - if (!(splitsChange instanceof SplitsAddition)) { - throw new UnsupportedOperationException( - String.format( - "The SplitChange type of %s is not supported.", - splitsChange.getClass())); - } - - // Assignment. - List newPartitionAssignments = new ArrayList<>(); - // Starting offsets. - Map partitionsStartingFromSpecifiedOffsets = new HashMap<>(); - List partitionsStartingFromEarliest = new ArrayList<>(); - List partitionsStartingFromLatest = new ArrayList<>(); - // Stopping offsets. - List partitionsStoppingAtLatest = new ArrayList<>(); - - // Parse the starting and stopping offsets. - splitsChange - .splits() - .forEach( - s -> { - newPartitionAssignments.add(s.getTopicPartition()); - parseStartingOffsets( - s, - partitionsStartingFromEarliest, - partitionsStartingFromLatest, - partitionsStartingFromSpecifiedOffsets); - parseStoppingOffsets(s, partitionsStoppingAtLatest); - }); - - // Assign new partitions. - newPartitionAssignments.addAll(consumer.assignment()); - consumer.assign(newPartitionAssignments); - - // Seek on the newly assigned partitions to their stating offsets. - seekToStartingOffsets( - partitionsStartingFromEarliest, - partitionsStartingFromLatest, - partitionsStartingFromSpecifiedOffsets); - // Setup the stopping offsets. - acquireAndSetStoppingOffsets(partitionsStoppingAtLatest); - - // After acquiring the starting and stopping offsets, remove the empty splits if necessary. - removeEmptySplits(); - - maybeLogSplitChangesHandlingResult(splitsChange); - } - - private void maybeLogSplitChangesHandlingResult(SplitsChange splitsChange) { - if (LOG.isDebugEnabled()) { - StringJoiner splitsInfo = new StringJoiner(","); - Set assginment = consumer.assignment(); - for (KafkaSourceSplit split : splitsChange.splits()) { - if (!assginment.contains(split.getTopicPartition())) { - continue; - } - - long startingOffset = - retryOnWakeup( - () -> consumer.position(split.getTopicPartition()), - "logging starting position"); - long stoppingOffset = getStoppingOffset(split.getTopicPartition()); - splitsInfo.add( - String.format( - "[%s, start:%d, stop: %d]", - split.getTopicPartition(), startingOffset, stoppingOffset)); - } - LOG.debug("SplitsChange handling result: {}", splitsInfo); - } - } - - private void removeEmptySplits() { - List emptyPartitions = new ArrayList<>(); - // If none of the partitions have any records, - for (TopicPartition tp : consumer.assignment()) { - if (retryOnWakeup( - () -> consumer.position(tp), - "getting starting offset to check if split is empty") - >= getStoppingOffset(tp)) { - emptyPartitions.add(tp); - } - } - if (!emptyPartitions.isEmpty()) { - LOG.debug( - "These assigning splits are empty and will be marked as finished in later fetch: {}", - emptyPartitions); - // Add empty partitions to empty split set for later cleanup in fetch() - emptySplits.addAll( - emptyPartitions.stream() - .map(TopicPartition::toString) - .collect(Collectors.toSet())); - // Un-assign partitions from Kafka consumer - unassignPartitions(emptyPartitions); - } - } - - private void unassignPartitions(Collection partitionsToUnassign) { - Collection newAssignment = new HashSet<>(consumer.assignment()); - newAssignment.removeAll(partitionsToUnassign); - consumer.assign(newAssignment); - } - - private void acquireAndSetStoppingOffsets(List partitionsStoppingAtLatest) { - Map endOffset = consumer.endOffsets(partitionsStoppingAtLatest); - stoppingOffsets.putAll(endOffset); - } - - private void seekToStartingOffsets( - List partitionsStartingFromEarliest, - List partitionsStartingFromLatest, - Map partitionsStartingFromSpecifiedOffsets) { - - if (!partitionsStartingFromEarliest.isEmpty()) { - LOG.trace("Seeking starting offsets to beginning: {}", partitionsStartingFromEarliest); - consumer.seekToBeginning(partitionsStartingFromEarliest); - } - - if (!partitionsStartingFromLatest.isEmpty()) { - LOG.trace("Seeking starting offsets to end: {}", partitionsStartingFromLatest); - consumer.seekToEnd(partitionsStartingFromLatest); - } - - if (!partitionsStartingFromSpecifiedOffsets.isEmpty()) { - LOG.trace( - "Seeking starting offsets to specified offsets: {}", - partitionsStartingFromSpecifiedOffsets); - partitionsStartingFromSpecifiedOffsets.forEach(consumer::seek); - } - } - - private void parseStoppingOffsets( - KafkaSourceSplit split, List partitionsStoppingAtLatest) { - TopicPartition tp = split.getTopicPartition(); - if (split.getEndOffset() >= 0) { - stoppingOffsets.put(tp, split.getEndOffset()); - } else { - partitionsStoppingAtLatest.add(tp); - } - } - - private long getStoppingOffset(TopicPartition tp) { - return stoppingOffsets.getOrDefault(tp, Long.MAX_VALUE); - } - - private void parseStartingOffsets( - KafkaSourceSplit split, - List partitionsStartingFromEarliest, - List partitionsStartingFromLatest, - Map partitionsStartingFromSpecifiedOffsets) { - TopicPartition tp = split.getTopicPartition(); - // Parse starting offsets. - ConsumerMetadata metadata = kafkaSourceConfig.getMapMetadata().get(split.getTablePath()); - if (metadata.getStartMode() == StartMode.EARLIEST) { - partitionsStartingFromEarliest.add(tp); - } else if (metadata.getStartMode() == StartMode.LATEST) { - partitionsStartingFromLatest.add(tp); - } else if (metadata.getStartMode() == StartMode.GROUP_OFFSETS) { - // Do nothing here, the consumer will first try to get the committed offsets of - // these partitions by default. - } else { - partitionsStartingFromSpecifiedOffsets.put(tp, split.getStartOffset()); - } - } - - @Override - public void wakeUp() { - consumer.wakeup(); - } - - @Override - public void close() throws Exception { - consumer.close(); - } - - public void notifyCheckpointComplete( - Map offsetsToCommit, - OffsetCommitCallback offsetCommitCallback) { - consumer.commitAsync(offsetsToCommit, offsetCommitCallback); - } - - private KafkaConsumer initConsumer( - KafkaSourceConfig kafkaSourceConfig, int subtaskId) { - - try (TemporaryClassLoaderContext ignored = - TemporaryClassLoaderContext.of(kafkaSourceConfig.getClass().getClassLoader())) { - Properties props = new Properties(); - kafkaSourceConfig - .getProperties() - .forEach( - (key, value) -> - props.setProperty(String.valueOf(key), String.valueOf(value))); - props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, kafkaSourceConfig.getConsumerGroup()); - props.setProperty( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaSourceConfig.getBootstrap()); - if (this.kafkaSourceConfig.getProperties().get("client.id") == null) { - props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, - CLIENT_ID_PREFIX + "-consumer-" + subtaskId); - } else { - props.setProperty( - ConsumerConfig.CLIENT_ID_CONFIG, - this.kafkaSourceConfig.getProperties().get("client.id").toString() - + "-" - + subtaskId); - } - props.setProperty( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - ByteArrayDeserializer.class.getName()); - props.setProperty( - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - ByteArrayDeserializer.class.getName()); - props.setProperty( - ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, - String.valueOf(kafkaSourceConfig.isCommitOnCheckpoint())); - - // Disable auto create topics feature - props.setProperty(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); - return new KafkaConsumer<>(props); - } - } - - private V retryOnWakeup(Supplier consumerCall, String description) { - try { - return consumerCall.get(); - } catch (WakeupException we) { - LOG.info( - "Caught WakeupException while executing Kafka consumer call for {}. Will retry the consumer call.", - description); - return consumerCall.get(); - } - } - - private static class KafkaPartitionSplitRecords - implements RecordsWithSplitIds> { - - private final Set finishedSplits = new HashSet<>(); - private final Map stoppingOffsets = new HashMap<>(); - private final ConsumerRecords consumerRecords; - private final Iterator splitIterator; - private Iterator> recordIterator; - private TopicPartition currentTopicPartition; - private Long currentSplitStoppingOffset; - - private KafkaPartitionSplitRecords(ConsumerRecords consumerRecords) { - this.consumerRecords = consumerRecords; - this.splitIterator = consumerRecords.partitions().iterator(); - } - - private void setPartitionStoppingOffset( - TopicPartition topicPartition, long stoppingOffset) { - stoppingOffsets.put(topicPartition, stoppingOffset); - } - - private void addFinishedSplit(String splitId) { - finishedSplits.add(splitId); - } - - @Nullable @Override - public String nextSplit() { - if (splitIterator.hasNext()) { - currentTopicPartition = splitIterator.next(); - recordIterator = consumerRecords.records(currentTopicPartition).iterator(); - currentSplitStoppingOffset = - stoppingOffsets.getOrDefault(currentTopicPartition, Long.MAX_VALUE); - return currentTopicPartition.toString(); - } else { - currentTopicPartition = null; - recordIterator = null; - currentSplitStoppingOffset = null; - return null; - } - } - - @Nullable @Override - public ConsumerRecord nextRecordFromSplit() { - Preconditions.checkNotNull( - currentTopicPartition, - "Make sure nextSplit() did not return null before " - + "iterate over the records split."); - if (recordIterator.hasNext()) { - final ConsumerRecord record = recordIterator.next(); - // Only emit records before stopping offset - if (record.offset() < currentSplitStoppingOffset) { - return record; - } - } - return null; - } - - @Override - public Set finishedSplits() { - return finishedSplits; - } - } -} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java deleted file mode 100644 index 6593137aff7..00000000000 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitter.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.kafka.source; - -import org.apache.seatunnel.api.serialization.DeserializationSchema; -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.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordEmitter; -import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; -import org.apache.seatunnel.format.compatible.kafka.connect.json.CompatibleKafkaConnectDeserializationSchema; - -import org.apache.kafka.clients.consumer.ConsumerRecord; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Map; - -public class KafkaRecordEmitter - implements RecordEmitter< - ConsumerRecord, SeaTunnelRow, KafkaSourceSplitState> { - - private static final Logger logger = LoggerFactory.getLogger(KafkaRecordEmitter.class); - private final Map mapMetadata; - private final OutputCollector outputCollector; - private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; - - public KafkaRecordEmitter( - Map mapMetadata, - MessageFormatErrorHandleWay messageFormatErrorHandleWay) { - this.mapMetadata = mapMetadata; - this.messageFormatErrorHandleWay = messageFormatErrorHandleWay; - this.outputCollector = new OutputCollector<>(); - } - - @Override - public void emitRecord( - ConsumerRecord consumerRecord, - Collector collector, - KafkaSourceSplitState splitState) - throws Exception { - outputCollector.output = collector; - // todo there is an additional loss in this place for non-multi-table scenarios - DeserializationSchema deserializationSchema = - mapMetadata.get(splitState.getTablePath()).getDeserializationSchema(); - try { - if (deserializationSchema instanceof CompatibleKafkaConnectDeserializationSchema) { - ((CompatibleKafkaConnectDeserializationSchema) deserializationSchema) - .deserialize(consumerRecord, outputCollector); - } else { - deserializationSchema.deserialize(consumerRecord.value(), outputCollector); - } - // consumerRecord.offset + 1 is the offset commit to Kafka and also the start offset - // for the next run - splitState.setCurrentOffset(consumerRecord.offset() + 1); - } catch (IOException e) { - if (this.messageFormatErrorHandleWay == MessageFormatErrorHandleWay.SKIP) { - logger.warn( - "Deserialize message failed, skip this message, message: {}", - new String(consumerRecord.value())); - } - throw e; - } - } - - private static class OutputCollector implements Collector { - private Collector output; - - @Override - public void collect(T record) { - output.collect(record); - } - - @Override - public void collect(SchemaChangeEvent event) { - output.collect(event); - } - - @Override - public void markSchemaChangeBeforeCheckpoint() { - output.markSchemaChangeBeforeCheckpoint(); - } - - @Override - public void markSchemaChangeAfterCheckpoint() { - output.markSchemaChangeAfterCheckpoint(); - } - - @Override - public Object getCheckpointLock() { - return output.getCheckpointLock(); - } - } -} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index 5688fde5b64..0c8cb4d496a 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -27,31 +27,20 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.constants.JobMode; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.SourceReaderOptions; -import org.apache.seatunnel.connectors.seatunnel.kafka.source.fetch.KafkaSourceFetcherManager; import org.apache.seatunnel.connectors.seatunnel.kafka.state.KafkaSourceState; -import org.apache.kafka.clients.consumer.ConsumerRecord; - -import com.google.common.base.Supplier; - import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; public class KafkaSource implements SeaTunnelSource, SupportParallelism { - private final ReadonlyConfig readonlyConfig; private JobContext jobContext; private final KafkaSourceConfig kafkaSourceConfig; public KafkaSource(ReadonlyConfig readonlyConfig) { - this.readonlyConfig = readonlyConfig; kafkaSourceConfig = new KafkaSourceConfig(readonlyConfig); } @@ -77,28 +66,10 @@ public List getProducedCatalogTables() { @Override public SourceReader createReader( SourceReader.Context readerContext) { - - BlockingQueue>> elementsQueue = - new LinkedBlockingQueue<>(); - - Supplier kafkaPartitionSplitReaderSupplier = - () -> new KafkaPartitionSplitReader(kafkaSourceConfig, readerContext); - - KafkaSourceFetcherManager kafkaSourceFetcherManager = - new KafkaSourceFetcherManager( - elementsQueue, kafkaPartitionSplitReaderSupplier::get); - KafkaRecordEmitter kafkaRecordEmitter = - new KafkaRecordEmitter( - kafkaSourceConfig.getMapMetadata(), - kafkaSourceConfig.getMessageFormatErrorHandleWay()); - return new KafkaSourceReader( - elementsQueue, - kafkaSourceFetcherManager, - kafkaRecordEmitter, - new SourceReaderOptions(readonlyConfig), kafkaSourceConfig, - readerContext); + readerContext, + kafkaSourceConfig.getMessageFormatErrorHandleWay()); } @Override diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java index 960a0184029..1c782ca6ab7 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceConfig.java @@ -44,7 +44,6 @@ import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import org.apache.seatunnel.format.json.maxwell.MaxWellJsonDeserializationSchema; import org.apache.seatunnel.format.json.ogg.OggJsonDeserializationSchema; -import org.apache.seatunnel.format.protobuf.ProtobufDeserializationSchema; import org.apache.seatunnel.format.text.TextDeserializationSchema; import org.apache.seatunnel.format.text.constant.TextFormatConstant; @@ -72,8 +71,6 @@ import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PATTERN; -import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_MESSAGE_NAME; -import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PROTOBUF_SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_OFFSETS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.START_MODE_TIMESTAMP; @@ -89,7 +86,6 @@ public class KafkaSourceConfig implements Serializable { @Getter private final Properties properties; @Getter private final long discoveryIntervalMillis; @Getter private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; - @Getter private final String consumerGroup; public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { this.bootstrap = readonlyConfig.get(BOOTSTRAP_SERVERS); @@ -99,7 +95,6 @@ public KafkaSourceConfig(ReadonlyConfig readonlyConfig) { this.discoveryIntervalMillis = readonlyConfig.get(KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS); this.messageFormatErrorHandleWay = readonlyConfig.get(MESSAGE_FORMAT_ERROR_HANDLE_WAY_OPTION); - this.consumerGroup = readonlyConfig.get(CONSUMER_GROUP); } private Properties createKafkaProperties(ReadonlyConfig readonlyConfig) { @@ -133,6 +128,7 @@ private ConsumerMetadata createConsumerMetadata(ReadonlyConfig readonlyConfig) { ConsumerMetadata consumerMetadata = new ConsumerMetadata(); consumerMetadata.setTopic(readonlyConfig.get(TOPIC)); consumerMetadata.setPattern(readonlyConfig.get(PATTERN)); + consumerMetadata.setConsumerGroup(readonlyConfig.get(CONSUMER_GROUP)); consumerMetadata.setProperties(new Properties()); // Create a catalog CatalogTable catalogTable = createCatalogTable(readonlyConfig); @@ -220,15 +216,7 @@ private CatalogTable createCatalogTable(ReadonlyConfig readonlyConfig) { return CatalogTable.of( TableIdentifier.of("", tablePath), tableSchema, - new HashMap() { - { - Optional.ofNullable(readonlyConfig.get(PROTOBUF_MESSAGE_NAME)) - .ifPresent(value -> put(PROTOBUF_MESSAGE_NAME.key(), value)); - - Optional.ofNullable(readonlyConfig.get(PROTOBUF_SCHEMA)) - .ifPresent(value -> put(PROTOBUF_SCHEMA.key(), value)); - } - }, + Collections.emptyMap(), Collections.emptyList(), null); } @@ -237,8 +225,6 @@ private DeserializationSchema createDeserializationSchema( CatalogTable catalogTable, ReadonlyConfig readonlyConfig) { SeaTunnelRowType seaTunnelRowType = catalogTable.getSeaTunnelRowType(); - MessageFormat format = readonlyConfig.get(FORMAT); - if (!readonlyConfig.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { return TextDeserializationSchema.builder() .seaTunnelRowType(seaTunnelRowType) @@ -247,6 +233,7 @@ private DeserializationSchema createDeserializationSchema( .build(); } + MessageFormat format = readonlyConfig.get(FORMAT); switch (format) { case JSON: return new JsonDeserializationSchema(catalogTable, false, false); @@ -283,8 +270,6 @@ private DeserializationSchema createDeserializationSchema( return new DebeziumJsonDeserializationSchema(catalogTable, true, includeSchema); case AVRO: return new AvroDeserializationSchema(catalogTable); - case PROTOBUF: - return new ProtobufDeserializationSchema(catalogTable); default: throw new SeaTunnelJsonFormatException( CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, 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 82a0522c411..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 @@ -17,156 +17,283 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordEmitter; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.SourceReaderOptions; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SingleThreadFetcherManager; -import org.apache.seatunnel.connectors.seatunnel.kafka.source.fetch.KafkaSourceFetcherManager; +import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; +import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.kafka.exception.KafkaConnectorException; +import org.apache.seatunnel.format.compatible.kafka.connect.json.CompatibleKafkaConnectDeserializationSchema; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import com.google.common.collect.Sets; +import lombok.extern.slf4j.Slf4j; -import java.util.Collections; +import java.io.IOException; +import java.time.Duration; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.concurrent.BlockingQueue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Collectors; -public class KafkaSourceReader - extends SingleThreadMultiplexSourceReaderBase< - ConsumerRecord, - SeaTunnelRow, - KafkaSourceSplit, - KafkaSourceSplitState> { +@Slf4j +public class KafkaSourceReader implements SourceReader { - private static final Logger logger = LoggerFactory.getLogger(KafkaSourceReader.class); - private final SourceReader.Context context; + private static final long THREAD_WAIT_TIME = 500L; + private static final long POLL_TIMEOUT = 10000L; + private final SourceReader.Context context; private final KafkaSourceConfig kafkaSourceConfig; - private final SortedMap> checkpointOffsetMap; - private final ConcurrentMap offsetsOfFinishedSplits; + private final Map tablePathMetadataMap; + private final Set sourceSplits; + private final Map> checkpointOffsetMap; + private final Map consumerThreadMap; + private final ExecutorService executorService; + private final MessageFormatErrorHandleWay messageFormatErrorHandleWay; + + private final LinkedBlockingQueue pendingPartitionsQueue; + + private volatile boolean running = false; KafkaSourceReader( - BlockingQueue>> elementsQueue, - SingleThreadFetcherManager, KafkaSourceSplit> - splitFetcherManager, - RecordEmitter, SeaTunnelRow, KafkaSourceSplitState> - recordEmitter, - SourceReaderOptions options, KafkaSourceConfig kafkaSourceConfig, - Context context) { - super(elementsQueue, splitFetcherManager, recordEmitter, options, context); + Context context, + MessageFormatErrorHandleWay messageFormatErrorHandleWay) { this.kafkaSourceConfig = kafkaSourceConfig; + this.tablePathMetadataMap = kafkaSourceConfig.getMapMetadata(); this.context = context; - this.checkpointOffsetMap = Collections.synchronizedSortedMap(new TreeMap<>()); - this.offsetsOfFinishedSplits = new ConcurrentHashMap<>(); + this.messageFormatErrorHandleWay = messageFormatErrorHandleWay; + this.sourceSplits = new HashSet<>(); + this.consumerThreadMap = new ConcurrentHashMap<>(); + this.checkpointOffsetMap = new ConcurrentHashMap<>(); + this.executorService = + Executors.newCachedThreadPool(r -> new Thread(r, "Kafka Source Data Consumer")); + pendingPartitionsQueue = new LinkedBlockingQueue<>(); } @Override - protected void onSplitFinished(Map finishedSplitIds) { - finishedSplitIds.forEach( - (ignored, splitState) -> { - if (splitState.getCurrentOffset() > 0) { - offsetsOfFinishedSplits.put( - splitState.getTopicPartition(), - new OffsetAndMetadata(splitState.getCurrentOffset())); - } else if (splitState.getEndOffset() > 0) { - offsetsOfFinishedSplits.put( - splitState.getTopicPartition(), - new OffsetAndMetadata(splitState.getEndOffset())); - } - }); - } + public void open() {} @Override - protected KafkaSourceSplitState initializedState(KafkaSourceSplit split) { - return new KafkaSourceSplitState(split); + public void close() throws IOException { + if (executorService != null) { + executorService.shutdownNow(); + } } @Override - protected KafkaSourceSplit toSplitType(String splitId, KafkaSourceSplitState splitState) { - return splitState.toKafkaSourceSplit(); - } + public void pollNext(Collector output) throws Exception { + if (!running) { + Thread.sleep(THREAD_WAIT_TIME); + return; + } - @Override - public List snapshotState(long checkpointId) { - List sourceSplits = super.snapshotState(checkpointId); - if (!kafkaSourceConfig.isCommitOnCheckpoint()) { - return sourceSplits; + while (!pendingPartitionsQueue.isEmpty()) { + sourceSplits.add(pendingPartitionsQueue.poll()); } - if (sourceSplits.isEmpty() && offsetsOfFinishedSplits.isEmpty()) { - logger.debug( - "checkpoint {} does not have an offset to submit for splits", checkpointId); - checkpointOffsetMap.put(checkpointId, Collections.emptyMap()); - } else { - Map offsetAndMetadataMap = - checkpointOffsetMap.computeIfAbsent(checkpointId, id -> new HashMap<>()); - for (KafkaSourceSplit kafkaSourceSplit : sourceSplits) { - if (kafkaSourceSplit.getStartOffset() >= 0) { - offsetAndMetadataMap.put( - kafkaSourceSplit.getTopicPartition(), - new OffsetAndMetadata(kafkaSourceSplit.getStartOffset())); + sourceSplits.forEach( + sourceSplit -> + consumerThreadMap.computeIfAbsent( + sourceSplit.getTopicPartition(), + s -> { + ConsumerMetadata currentSplitConsumerMetaData = + tablePathMetadataMap.get(sourceSplit.getTablePath()); + KafkaConsumerThread thread = + new KafkaConsumerThread( + kafkaSourceConfig, + currentSplitConsumerMetaData); + executorService.submit(thread); + return thread; + })); + List finishedSplits = new CopyOnWriteArrayList<>(); + sourceSplits.forEach( + sourceSplit -> { + CompletableFuture completableFuture = new CompletableFuture<>(); + TablePath tablePath = sourceSplit.getTablePath(); + DeserializationSchema deserializationSchema = + tablePathMetadataMap.get(tablePath).getDeserializationSchema(); + try { + consumerThreadMap + .get(sourceSplit.getTopicPartition()) + .getTasks() + .put( + consumer -> { + try { + Set partitions = + Sets.newHashSet( + sourceSplit.getTopicPartition()); + consumer.assign(partitions); + if (sourceSplit.getStartOffset() >= 0) { + consumer.seek( + sourceSplit.getTopicPartition(), + sourceSplit.getStartOffset()); + } + ConsumerRecords records = + consumer.poll( + Duration.ofMillis(POLL_TIMEOUT)); + for (TopicPartition partition : partitions) { + List> + recordList = records.records(partition); + if (Boundedness.BOUNDED.equals( + context.getBoundedness()) + && recordList.isEmpty()) { + completableFuture.complete(true); + return; + } + for (ConsumerRecord record : + recordList) { + try { + if (deserializationSchema + instanceof + CompatibleKafkaConnectDeserializationSchema) { + ((CompatibleKafkaConnectDeserializationSchema) + deserializationSchema) + .deserialize( + record, output); + } else { + deserializationSchema.deserialize( + record.value(), output); + } + } catch (IOException e) { + if (this.messageFormatErrorHandleWay + == MessageFormatErrorHandleWay + .SKIP) { + log.warn( + "Deserialize message failed, skip this message, message: {}", + new String(record.value())); + continue; + } + throw e; + } + + if (Boundedness.BOUNDED.equals( + context.getBoundedness()) + && record.offset() + >= sourceSplit + .getEndOffset()) { + completableFuture.complete(true); + return; + } + } + long lastOffset = -1; + if (!recordList.isEmpty()) { + lastOffset = + recordList + .get(recordList.size() - 1) + .offset(); + sourceSplit.setStartOffset(lastOffset + 1); + } + + if (lastOffset >= sourceSplit.getEndOffset()) { + sourceSplit.setEndOffset(lastOffset); + } + } + } catch (Exception e) { + completableFuture.completeExceptionally(e); + } + completableFuture.complete(false); + }); + if (completableFuture.get()) { + finishedSplits.add(sourceSplit); + } + } catch (Exception e) { + throw new KafkaConnectorException( + KafkaConnectorErrorCode.CONSUME_DATA_FAILED, e); + } + }); + if (Boundedness.BOUNDED.equals(context.getBoundedness())) { + for (KafkaSourceSplit split : finishedSplits) { + split.setFinish(true); + if (split.getStartOffset() == -1) { + // log next running read start offset + split.setStartOffset(split.getEndOffset()); } } - offsetAndMetadataMap.putAll(offsetsOfFinishedSplits); + if (sourceSplits.stream().allMatch(KafkaSourceSplit::isFinish)) { + context.signalNoMoreElement(); + } } - return sourceSplits; } @Override - public void notifyCheckpointComplete(long checkpointId) { - logger.debug("Committing offsets for checkpoint {}", checkpointId); - if (!kafkaSourceConfig.isCommitOnCheckpoint()) { - logger.debug("Submitting offsets after snapshot completion is prohibited"); - return; - } - Map committedPartitions = - checkpointOffsetMap.get(checkpointId); - - if (committedPartitions == null) { - logger.debug("Offsets for checkpoint {} have already been committed.", checkpointId); - return; - } + public List snapshotState(long checkpointId) { + checkpointOffsetMap.put( + checkpointId, + sourceSplits.stream() + .collect( + Collectors.toMap( + KafkaSourceSplit::getTopicPartition, + KafkaSourceSplit::getStartOffset))); + return sourceSplits.stream().map(KafkaSourceSplit::copy).collect(Collectors.toList()); + } - if (committedPartitions.isEmpty()) { - logger.debug("There are no offsets to commit for checkpoint {}.", checkpointId); - removeAllOffsetsToCommitUpToCheckpoint(checkpointId); - return; - } + @Override + public void addSplits(List splits) { + running = true; + splits.forEach( + s -> { + try { + pendingPartitionsQueue.put(s); + } catch (InterruptedException e) { + throw new KafkaConnectorException( + KafkaConnectorErrorCode.ADD_SPLIT_CHECKPOINT_FAILED, e); + } + }); + } - ((KafkaSourceFetcherManager) splitFetcherManager) - .commitOffsets( - committedPartitions, - (ignored, e) -> { - if (e != null) { - logger.warn( - "Failed to commit consumer offsets for checkpoint {}", - checkpointId, - e); - return; - } - offsetsOfFinishedSplits - .keySet() - .removeIf(committedPartitions::containsKey); - removeAllOffsetsToCommitUpToCheckpoint(checkpointId); - }); + @Override + public void handleNoMoreSplits() { + log.info("receive no more splits message, this reader will not add new split."); } - private void removeAllOffsetsToCommitUpToCheckpoint(long checkpointId) { - while (!checkpointOffsetMap.isEmpty() && checkpointOffsetMap.firstKey() <= checkpointId) { - checkpointOffsetMap.remove(checkpointOffsetMap.firstKey()); + @Override + public void notifyCheckpointComplete(long checkpointId) { + if (!checkpointOffsetMap.containsKey(checkpointId)) { + log.warn("checkpoint {} do not exist or have already been committed.", checkpointId); + } else { + checkpointOffsetMap + .remove(checkpointId) + .forEach( + (topicPartition, offset) -> { + try { + consumerThreadMap + .get(topicPartition) + .getTasks() + .put( + consumer -> { + if (kafkaSourceConfig + .isCommitOnCheckpoint()) { + Map + offsets = new HashMap<>(); + if (offset >= 0) { + offsets.put( + topicPartition, + new OffsetAndMetadata( + offset)); + consumer.commitSync(offsets); + } + } + }); + } catch (InterruptedException e) { + log.error("commit offset to kafka failed", e); + } + }); } } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java index 06ce4565c3b..f868eaed20c 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java @@ -151,7 +151,8 @@ private void setPartitionStartOffset() throws ExecutionException, InterruptedExc listOffsets(topicPartitions, OffsetSpec.earliest())); break; case GROUP_OFFSETS: - topicPartitionOffsets.putAll(listConsumerGroupOffsets(topicPartitions)); + topicPartitionOffsets.putAll( + listConsumerGroupOffsets(topicPartitions, metadata)); break; case LATEST: topicPartitionOffsets.putAll(listOffsets(topicPartitions, OffsetSpec.latest())); @@ -365,12 +366,13 @@ private Map listOffsets( .get(); } - public Map listConsumerGroupOffsets(Collection partitions) + public Map listConsumerGroupOffsets( + Collection partitions, ConsumerMetadata metadata) throws ExecutionException, InterruptedException { ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions().topicPartitions(new ArrayList<>(partitions)); return adminClient - .listConsumerGroupOffsets(kafkaSourceConfig.getConsumerGroup(), options) + .listConsumerGroupOffsets(metadata.getConsumerGroup(), options) .partitionsToOffsetAndMetadata() .thenApply( result -> { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java deleted file mode 100644 index ab2592cc38e..00000000000 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitState.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.kafka.source; - -public class KafkaSourceSplitState extends KafkaSourceSplit { - - private long currentOffset; - - public KafkaSourceSplitState(KafkaSourceSplit sourceSplit) { - super( - sourceSplit.getTablePath(), - sourceSplit.getTopicPartition(), - sourceSplit.getStartOffset(), - sourceSplit.getEndOffset()); - this.currentOffset = sourceSplit.getStartOffset(); - } - - public long getCurrentOffset() { - return currentOffset; - } - - public void setCurrentOffset(long currentOffset) { - this.currentOffset = currentOffset; - } - - public KafkaSourceSplit toKafkaSourceSplit() { - return new KafkaSourceSplit( - getTablePath(), getTopicPartition(), getCurrentOffset(), getEndOffset()); - } -} diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java deleted file mode 100644 index bc80455725d..00000000000 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/fetch/KafkaSourceFetcherManager.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.kafka.source.fetch; - -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.RecordsWithSplitIds; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SingleThreadFetcherManager; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcher; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.fetcher.SplitFetcherTask; -import org.apache.seatunnel.connectors.seatunnel.common.source.reader.splitreader.SplitReader; -import org.apache.seatunnel.connectors.seatunnel.kafka.source.KafkaPartitionSplitReader; -import org.apache.seatunnel.connectors.seatunnel.kafka.source.KafkaSourceSplit; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.common.TopicPartition; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.BlockingQueue; -import java.util.function.Consumer; -import java.util.function.Supplier; - -public class KafkaSourceFetcherManager - extends SingleThreadFetcherManager, KafkaSourceSplit> { - - private static final Logger logger = LoggerFactory.getLogger(KafkaSourceFetcherManager.class); - - public KafkaSourceFetcherManager( - BlockingQueue>> elementsQueue, - Supplier, KafkaSourceSplit>> - splitReaderSupplier) { - super(elementsQueue, splitReaderSupplier); - } - - public KafkaSourceFetcherManager( - BlockingQueue>> elementsQueue, - Supplier, KafkaSourceSplit>> - splitReaderSupplier, - Consumer> splitFinishedHook) { - super(elementsQueue, splitReaderSupplier, splitFinishedHook); - } - - public void commitOffsets( - Map offsetsToCommit, OffsetCommitCallback callback) { - logger.debug("Committing offsets {}", offsetsToCommit); - if (offsetsToCommit.isEmpty()) { - return; - } - SplitFetcher, KafkaSourceSplit> splitFetcher = - fetchers.get(0); - if (splitFetcher != null) { - // The fetcher thread is still running. This should be the majority of the cases. - enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback); - } else { - splitFetcher = createSplitFetcher(); - enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit, callback); - startFetcher(splitFetcher); - } - } - - private void enqueueOffsetsCommitTask( - SplitFetcher, KafkaSourceSplit> splitFetcher, - Map offsetsToCommit, - OffsetCommitCallback callback) { - KafkaPartitionSplitReader kafkaReader = - (KafkaPartitionSplitReader) splitFetcher.getSplitReader(); - - splitFetcher.addTask( - new SplitFetcherTask() { - @Override - public void run() throws IOException { - kafkaReader.notifyCheckpointComplete(offsetsToCommit, callback); - } - - @Override - public void wakeUp() {} - }); - } -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java index 8eda6612c70..505b30fcbde 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java @@ -401,7 +401,7 @@ private static byte[] convertToBinary(BsonValue bsonValue) { } private static long convertToLong(BsonValue bsonValue) { - if (bsonValue.isInt64() || bsonValue.isInt32()) { + if (bsonValue.isInt64()) { return bsonValue.asNumber().longValue(); } throw new MongodbConnectorException( diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java index 4ef08c53988..6ce9596daa9 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java @@ -108,12 +108,7 @@ private ImmutablePair getDocumentNumAndAvgSize() { clientProvider.getDefaultCollection().getNamespace().getCollectionName(); BsonDocument statsCmd = new BsonDocument("collStats", new BsonString(collectionName)); Document res = clientProvider.getDefaultDatabase().runCommand(statsCmd); - Object count = res.get("count"); - // fix issue https://github.com/apache/seatunnel/issues/7575 - long total = - Optional.ofNullable(count) - .map(v -> Long.parseLong(String.valueOf(count))) - .orElse(0L); + long total = res.getInteger("count"); Object avgDocumentBytes = res.get("avgObjSize"); long avgObjSize = Optional.ofNullable(avgDocumentBytes) diff --git a/seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java b/seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java deleted file mode 100644 index b47769c0aca..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/test/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConvertersTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.mongodb.serde; - -import org.apache.seatunnel.api.table.type.BasicType; - -import org.bson.BsonDouble; -import org.bson.BsonInt32; -import org.bson.BsonInt64; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class BsonToRowDataConvertersTest { - private final BsonToRowDataConverters converterFactory = new BsonToRowDataConverters(); - - @Test - public void testConvertAnyNumberToDouble() { - // It covered #6997 - BsonToRowDataConverters.BsonToRowDataConverter converter = - converterFactory.createConverter(BasicType.DOUBLE_TYPE); - - Assertions.assertEquals(1.0d, converter.convert(new BsonInt32(1))); - Assertions.assertEquals(1.0d, converter.convert(new BsonInt64(1L))); - - Assertions.assertEquals(4.0d, converter.convert(new BsonDouble(4.0d))); - Assertions.assertEquals(4.4d, converter.convert(new BsonDouble(4.4d))); - } - - @Test - public void testConvertBsonIntToBigInt() { - // It covered #7567 - BsonToRowDataConverters.BsonToRowDataConverter converter = - converterFactory.createConverter(BasicType.LONG_TYPE); - - Assertions.assertEquals(123456L, converter.convert(new BsonInt32(123456))); - - Assertions.assertEquals( - (long) Integer.MAX_VALUE, converter.convert(new BsonInt64(Integer.MAX_VALUE))); - } -} diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java index 9e09035e2f0..d896e015398 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalog.java @@ -40,7 +40,6 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; -import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import java.io.Closeable; @@ -184,35 +183,6 @@ public void createDatabase(TablePath tablePath, boolean ignoreIfExists) } } - @Override - public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) - throws TableNotExistException, CatalogException { - try { - Identifier identifier = toIdentifier(tablePath); - FileStoreTable table = (FileStoreTable) catalog.getTable(identifier); - Schema schema = buildPaimonSchema(table.schema()); - dropTable(tablePath, ignoreIfNotExists); - catalog.createTable(identifier, schema, ignoreIfNotExists); - } catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) { - throw new TableNotExistException(this.catalogName, tablePath); - } catch (org.apache.paimon.catalog.Catalog.TableAlreadyExistException e) { - throw new DatabaseAlreadyExistException(this.catalogName, tablePath.getDatabaseName()); - } catch (org.apache.paimon.catalog.Catalog.DatabaseNotExistException e) { - throw new DatabaseNotExistException(this.catalogName, tablePath.getDatabaseName()); - } - } - - private Schema buildPaimonSchema(@NonNull org.apache.paimon.schema.TableSchema schema) { - Schema.Builder builder = Schema.newBuilder(); - schema.fields() - .forEach(field -> builder.column(field.name(), field.type(), field.description())); - builder.options(schema.options()); - builder.primaryKey(schema.primaryKeys()); - builder.partitionKeys(schema.partitionKeys()); - builder.comment(schema.comment()); - return builder.build(); - } - @Override public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) throws DatabaseNotExistException, CatalogException { diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java index ed4c80a40d0..ef37e52c01e 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/exception/PaimonConnectorErrorCode.java @@ -26,8 +26,7 @@ public enum PaimonConnectorErrorCode implements SeaTunnelErrorCode { TABLE_PRE_COMMIT_FAILED("PAIMON-03", "Paimon pre commit failed"), GET_TABLE_FAILED("PAIMON-04", "Get table from database failed"), AUTHENTICATE_KERBEROS_FAILED("PAIMON-05", "Authenticate kerberos failed"), - LOAD_CATALOG("PAIMON-06", "Load catalog failed"), - GET_FILED_FAILED("PAIMON-07", "Get field failed"); + LOAD_CATALOG("PAIMON-06", "Load catalog failed"); private final String code; private final String description; diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java index 580f55b5817..9c576018a32 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverter.java @@ -346,18 +346,17 @@ public static SeaTunnelRow convert( * * @param seaTunnelRow SeaTunnel row object * @param seaTunnelRowType SeaTunnel row type - * @param sinkTableSchema Paimon table schema + * @param tableSchema Paimon table schema * @return Paimon row object */ public static InternalRow reconvert( - SeaTunnelRow seaTunnelRow, - SeaTunnelRowType seaTunnelRowType, - TableSchema sinkTableSchema) { - List sinkTotalFields = sinkTableSchema.fields(); + SeaTunnelRow seaTunnelRow, SeaTunnelRowType seaTunnelRowType, TableSchema tableSchema) { + List sinkTotalFields = tableSchema.fields(); int sourceTotalFields = seaTunnelRowType.getTotalFields(); if (sourceTotalFields != sinkTotalFields.size()) { - throw CommonError.writeRowErrorWithFiledsCountNotMatch( - "Paimon", sourceTotalFields, sinkTotalFields.size()); + throw new CommonError() + .writeRowErrorWithFiledsCountNotMatch( + "Paimon", sourceTotalFields, sinkTotalFields.size()); } BinaryRow binaryRow = new BinaryRow(sourceTotalFields); BinaryWriter binaryWriter = new BinaryRowWriter(binaryRow); @@ -378,7 +377,7 @@ public static InternalRow reconvert( binaryWriter.setNullAt(i); continue; } - checkCanWriteWithSchema(i, seaTunnelRowType, sinkTotalFields); + checkCanWriteWithType(i, seaTunnelRowType, sinkTotalFields); String fieldName = seaTunnelRowType.getFieldName(i); switch (fieldTypes[i].getSqlType()) { case TINYINT: @@ -400,17 +399,14 @@ public static InternalRow reconvert( binaryWriter.writeDouble(i, (Double) seaTunnelRow.getField(i)); break; case DECIMAL: - DataField decimalDataField = - SchemaUtil.getDataField(sinkTotalFields, fieldName); - org.apache.paimon.types.DecimalType decimalType = - (org.apache.paimon.types.DecimalType) decimalDataField.type(); + DecimalType fieldType = (DecimalType) seaTunnelRowType.getFieldType(i); binaryWriter.writeDecimal( i, Decimal.fromBigDecimal( (BigDecimal) seaTunnelRow.getField(i), - decimalType.getPrecision(), - decimalType.getScale()), - decimalType.getPrecision()); + fieldType.getPrecision(), + fieldType.getScale()), + fieldType.getPrecision()); break; case STRING: binaryWriter.writeString( @@ -468,12 +464,9 @@ public static InternalRow reconvert( SeaTunnelDataType rowType = seaTunnelRowType.getFieldType(i); Object row = seaTunnelRow.getField(i); InternalRow paimonRow = - reconvert( - (SeaTunnelRow) row, - (SeaTunnelRowType) rowType, - sinkTableSchema); + reconvert((SeaTunnelRow) row, (SeaTunnelRowType) rowType, tableSchema); RowType paimonRowType = - RowTypeConverter.reconvert((SeaTunnelRowType) rowType, sinkTableSchema); + RowTypeConverter.reconvert((SeaTunnelRowType) rowType, tableSchema); binaryWriter.writeRow(i, paimonRow, new InternalRowSerializer(paimonRowType)); break; default: @@ -486,7 +479,7 @@ public static InternalRow reconvert( return binaryRow; } - private static void checkCanWriteWithSchema( + private static void checkCanWriteWithType( int i, SeaTunnelRowType seaTunnelRowType, List fields) { String sourceFieldName = seaTunnelRowType.getFieldName(i); SeaTunnelDataType sourceFieldType = seaTunnelRowType.getFieldType(i); @@ -495,27 +488,13 @@ private static void checkCanWriteWithSchema( RowTypeConverter.reconvert(sourceFieldName, seaTunnelRowType.getFieldType(i)); DataField exceptDataField = new DataField(i, sourceFieldName, exceptDataType); DataType sinkDataType = sinkDataField.type(); - if (!exceptDataType.getTypeRoot().equals(sinkDataType.getTypeRoot()) - || !StringUtils.equals(sourceFieldName, sinkDataField.name())) { - throw CommonError.writeRowErrorWithSchemaIncompatibleSchema( - "Paimon", - sourceFieldName + StringUtils.SPACE + sourceFieldType.getSqlType(), - exceptDataField.asSQLString(), - sinkDataField.asSQLString()); - } - if (sourceFieldType instanceof DecimalType - && sinkDataType instanceof org.apache.paimon.types.DecimalType) { - DecimalType sourceDecimalType = (DecimalType) sourceFieldType; - org.apache.paimon.types.DecimalType sinkDecimalType = - (org.apache.paimon.types.DecimalType) sinkDataType; - if (sinkDecimalType.getPrecision() < sourceDecimalType.getPrecision() - || sinkDecimalType.getScale() < sourceDecimalType.getScale()) { - throw CommonError.writeRowErrorWithSchemaIncompatibleSchema( - "Paimon", - sourceFieldName + StringUtils.SPACE + sourceFieldType.getSqlType(), - exceptDataField.asSQLString(), - sinkDataField.asSQLString()); - } + if (!exceptDataType.getTypeRoot().equals(sinkDataType.getTypeRoot())) { + throw new CommonError() + .writeRowErrorWithSchemaIncompatibleSchema( + "Paimon", + sourceFieldName + StringUtils.SPACE + sourceFieldType.getSqlType(), + exceptDataField.asSQLString(), + sinkDataField.asSQLString()); } } } diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java index fa8ed338208..0da047244fa 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/SchemaUtil.java @@ -22,8 +22,6 @@ import org.apache.seatunnel.api.table.converter.BasicTypeDefine; import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig; import org.apache.seatunnel.connectors.seatunnel.paimon.data.PaimonTypeMapper; -import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorException; import org.apache.paimon.schema.Schema; import org.apache.paimon.types.DataField; @@ -32,7 +30,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; /** The util seatunnel schema to paimon schema */ public class SchemaUtil { @@ -72,13 +69,9 @@ public static Column toSeaTunnelType(BasicTypeDefine typeDefine) { } public static DataField getDataField(List fields, String fieldName) { - Optional firstField = - fields.stream().filter(field -> field.name().equals(fieldName)).findFirst(); - if (!firstField.isPresent()) { - throw new PaimonConnectorException( - PaimonConnectorErrorCode.GET_FILED_FAILED, - "Can not get the filed [" + fieldName + "] from source table"); - } - return firstField.get(); + return fields.parallelStream() + .filter(field -> field.name().equals(fieldName)) + .findFirst() + .get(); } } diff --git a/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java b/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java index c574b77e125..ebde744d032 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java +++ b/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/utils/RowConverterTest.java @@ -26,10 +26,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonError; -import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; -import org.apache.commons.lang3.StringUtils; import org.apache.paimon.data.BinaryArray; import org.apache.paimon.data.BinaryArrayWriter; import org.apache.paimon.data.BinaryMap; @@ -42,7 +39,6 @@ import org.apache.paimon.data.serializer.InternalArraySerializer; import org.apache.paimon.data.serializer.InternalMapSerializer; import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -52,8 +48,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import lombok.extern.slf4j.Slf4j; - import java.math.BigDecimal; import java.time.LocalDate; import java.time.LocalDateTime; @@ -64,7 +58,6 @@ import java.util.Map; /** Unit tests for {@link RowConverter} */ -@Slf4j public class RowConverterTest { private SeaTunnelRow seaTunnelRow; @@ -73,64 +66,49 @@ public class RowConverterTest { private SeaTunnelRowType seaTunnelRowType; - private volatile boolean isCaseSensitive = false; - private volatile boolean subtractOneFiledInSource = false; - private volatile int index = 0; - private static final String[] filedNames = { - "c_tinyint", - "c_smallint", - "c_int", - "c_bigint", - "c_float", - "c_double", - "c_decimal", - "c_string", - "c_bytes", - "c_boolean", - "c_date", - "c_timestamp", - "c_map", - "c_array" - }; - - public static final SeaTunnelDataType[] seaTunnelDataTypes = { - BasicType.BYTE_TYPE, - BasicType.SHORT_TYPE, - BasicType.INT_TYPE, - BasicType.LONG_TYPE, - BasicType.FLOAT_TYPE, - BasicType.DOUBLE_TYPE, - new DecimalType(30, 8), - BasicType.STRING_TYPE, - PrimitiveByteArrayType.INSTANCE, - BasicType.BOOLEAN_TYPE, - LocalTimeType.LOCAL_DATE_TYPE, - LocalTimeType.LOCAL_DATE_TIME_TYPE, - new MapType<>(BasicType.STRING_TYPE, BasicType.STRING_TYPE), - ArrayType.STRING_ARRAY_TYPE - }; + private TableSchema tableSchema; + + public static final RowType DEFAULT_ROW_TYPE = + RowType.of( + new DataType[] { + DataTypes.TINYINT(), + DataTypes.SMALLINT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.DECIMAL(10, 10), + DataTypes.STRING(), + DataTypes.BYTES(), + DataTypes.BOOLEAN(), + DataTypes.DATE(), + DataTypes.TIMESTAMP(), + DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING()), + DataTypes.ARRAY(DataTypes.STRING()) + }, + new String[] { + "c_tinyint", + "c_smallint", + "c_int", + "c_bigint", + "c_float", + "c_double", + "c_decimal", + "c_string", + "c_bytes", + "c_boolean", + "c_date", + "c_timestamp", + "c_map", + "c_array" + }); public static final List KEY_NAME_LIST = Arrays.asList("c_tinyint"); - public TableSchema getTableSchema(int decimalPrecision, int decimalScale) { - RowType rowType = - RowType.of( - new DataType[] { - DataTypes.TINYINT(), - DataTypes.SMALLINT(), - DataTypes.INT(), - DataTypes.BIGINT(), - DataTypes.FLOAT(), - DataTypes.DOUBLE(), - DataTypes.DECIMAL(decimalPrecision, decimalScale), - DataTypes.STRING(), - DataTypes.BYTES(), - DataTypes.BOOLEAN(), - DataTypes.DATE(), - DataTypes.TIMESTAMP(), - DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING()), - DataTypes.ARRAY(DataTypes.STRING()) - }, + @BeforeEach + public void before() { + seaTunnelRowType = + new SeaTunnelRowType( new String[] { "c_tinyint", "c_smallint", @@ -146,21 +124,23 @@ public TableSchema getTableSchema(int decimalPrecision, int decimalScale) { "c_timestamp", "c_map", "c_array" + }, + new SeaTunnelDataType[] { + BasicType.BYTE_TYPE, + BasicType.SHORT_TYPE, + BasicType.INT_TYPE, + BasicType.LONG_TYPE, + BasicType.FLOAT_TYPE, + BasicType.DOUBLE_TYPE, + new DecimalType(30, 8), + BasicType.STRING_TYPE, + PrimitiveByteArrayType.INSTANCE, + BasicType.BOOLEAN_TYPE, + LocalTimeType.LOCAL_DATE_TYPE, + LocalTimeType.LOCAL_DATE_TIME_TYPE, + new MapType<>(BasicType.STRING_TYPE, BasicType.STRING_TYPE), + ArrayType.STRING_ARRAY_TYPE }); - - return new TableSchema( - 0, - TableSchema.newFields(rowType), - rowType.getFieldCount(), - Collections.EMPTY_LIST, - KEY_NAME_LIST, - Collections.EMPTY_MAP, - ""); - } - - @BeforeEach - public void generateTestData() { - initSeaTunnelRowTypeCaseSensitive(isCaseSensitive, index, subtractOneFiledInSource); byte tinyint = 1; short smallint = 2; int intNum = 3; @@ -235,96 +215,27 @@ public void generateTestData() { binaryRowWriter.writeArray( 13, binaryArray2, new InternalArraySerializer(DataTypes.STRING())); internalRow = binaryRow; - } - private void initSeaTunnelRowTypeCaseSensitive( - boolean isUpperCase, int index, boolean subtractOneFiledInSource) { - String[] oneUpperCaseFiledNames = - Arrays.copyOf( - filedNames, - subtractOneFiledInSource ? filedNames.length - 1 : filedNames.length); - if (isUpperCase) { - oneUpperCaseFiledNames[index] = oneUpperCaseFiledNames[index].toUpperCase(); - } - SeaTunnelDataType[] newSeaTunnelDataTypes = - Arrays.copyOf( - seaTunnelDataTypes, - subtractOneFiledInSource - ? seaTunnelDataTypes.length - 1 - : filedNames.length); - seaTunnelRowType = new SeaTunnelRowType(oneUpperCaseFiledNames, newSeaTunnelDataTypes); + tableSchema = + new TableSchema( + 0, + TableSchema.newFields(DEFAULT_ROW_TYPE), + DEFAULT_ROW_TYPE.getFieldCount(), + Collections.EMPTY_LIST, + KEY_NAME_LIST, + Collections.EMPTY_MAP, + ""); } @Test public void seaTunnelToPaimon() { - TableSchema sinkTableSchema = getTableSchema(30, 8); - SeaTunnelRuntimeException actualException = - Assertions.assertThrows( - SeaTunnelRuntimeException.class, - () -> - RowConverter.reconvert( - seaTunnelRow, seaTunnelRowType, getTableSchema(10, 10))); - SeaTunnelRuntimeException exceptedException = - CommonError.writeRowErrorWithSchemaIncompatibleSchema( - "Paimon", - "c_decimal" + StringUtils.SPACE + "DECIMAL", - "`c_decimal` DECIMAL(30, 8)", - "`c_decimal` DECIMAL(10, 10)"); - Assertions.assertEquals(exceptedException.getMessage(), actualException.getMessage()); - - InternalRow reconvert = - RowConverter.reconvert(seaTunnelRow, seaTunnelRowType, sinkTableSchema); - Assertions.assertEquals(reconvert, internalRow); - - subtractOneFiledInSource = true; - generateTestData(); - SeaTunnelRuntimeException filedNumsActualException = - Assertions.assertThrows( - SeaTunnelRuntimeException.class, - () -> - RowConverter.reconvert( - seaTunnelRow, seaTunnelRowType, sinkTableSchema)); - SeaTunnelRuntimeException filedNumsExceptException = - CommonError.writeRowErrorWithFiledsCountNotMatch( - "Paimon", - seaTunnelRowType.getTotalFields(), - sinkTableSchema.fields().size()); - Assertions.assertEquals( - filedNumsExceptException.getMessage(), filedNumsActualException.getMessage()); - - subtractOneFiledInSource = false; - isCaseSensitive = true; - - for (int i = 0; i < filedNames.length; i++) { - index = i; - generateTestData(); - String sourceFiledname = seaTunnelRowType.getFieldName(i); - DataType exceptDataType = - RowTypeConverter.reconvert(sourceFiledname, seaTunnelRowType.getFieldType(i)); - DataField exceptDataField = new DataField(i, sourceFiledname, exceptDataType); - SeaTunnelRuntimeException actualException1 = - Assertions.assertThrows( - SeaTunnelRuntimeException.class, - () -> - RowConverter.reconvert( - seaTunnelRow, seaTunnelRowType, sinkTableSchema)); - Assertions.assertEquals( - CommonError.writeRowErrorWithSchemaIncompatibleSchema( - "Paimon", - sourceFiledname - + StringUtils.SPACE - + seaTunnelRowType.getFieldType(i).getSqlType(), - exceptDataField.asSQLString(), - sinkTableSchema.fields().get(i).asSQLString()) - .getMessage(), - actualException1.getMessage()); - } + InternalRow convert = RowConverter.reconvert(seaTunnelRow, seaTunnelRowType, tableSchema); + Assertions.assertEquals(convert, internalRow); } @Test public void paimonToSeaTunnel() { - SeaTunnelRow convert = - RowConverter.convert(internalRow, seaTunnelRowType, getTableSchema(10, 10)); + SeaTunnelRow convert = RowConverter.convert(internalRow, seaTunnelRowType, tableSchema); Assertions.assertEquals(convert, seaTunnelRow); } } diff --git a/seatunnel-connectors-v2/connector-qdrant/pom.xml b/seatunnel-connectors-v2/connector-qdrant/pom.xml deleted file mode 100644 index 686f0bdb7a4..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/pom.xml +++ /dev/null @@ -1,63 +0,0 @@ - - - - 4.0.0 - - org.apache.seatunnel - seatunnel-connectors-v2 - ${revision} - - - connector-qdrant - SeaTunnel : Connectors V2 : Qdrant - - - connector.qdrant - - - - - org.apache.seatunnel - connector-common - ${project.version} - - - - io.grpc - grpc-protobuf - 1.65.1 - - - - org.apache.seatunnel - seatunnel-guava - ${project.version} - optional - - - - io.qdrant - client - 1.11.0 - - - - diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java deleted file mode 100644 index 1be03a13111..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantConfig.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.config; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; - -public class QdrantConfig { - - public static final String CONNECTOR_IDENTITY = "Qdrant"; - - public static final Option HOST = - Options.key("host") - .stringType() - .defaultValue("localhost") - .withDescription("Qdrant gRPC host"); - - public static final Option PORT = - Options.key("port").intType().defaultValue(6334).withDescription("Qdrant gRPC port"); - - public static final Option API_KEY = - Options.key("api_key").stringType().defaultValue("").withDescription("Qdrant API key"); - - public static final Option COLLECTION_NAME = - Options.key("collection_name") - .stringType() - .noDefaultValue() - .withDescription("Qdrant collection name"); - - public static final Option USE_TLS = - Options.key("use_tls") - .booleanType() - .defaultValue(false) - .withDescription("Whether to use TLS"); -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java deleted file mode 100644 index 1ae612fafbb..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/config/QdrantParameters.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.config; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; - -import io.qdrant.client.QdrantClient; -import io.qdrant.client.QdrantGrpcClient; -import lombok.Data; - -import java.io.Serializable; - -@Data -public class QdrantParameters implements Serializable { - private String host; - private int port; - private String apiKey; - private String collectionName; - private boolean useTls; - - public QdrantParameters(ReadonlyConfig config) { - this.host = config.get(QdrantConfig.HOST); - this.port = config.get(QdrantConfig.PORT); - this.apiKey = config.get(QdrantConfig.API_KEY); - this.collectionName = config.get(QdrantConfig.COLLECTION_NAME); - this.useTls = config.get(QdrantConfig.USE_TLS); - } - - public QdrantClient buildQdrantClient() { - return new QdrantClient(QdrantGrpcClient.newBuilder(host, port, useTls).build()); - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.java deleted file mode 100644 index becf31abf5f..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/exception/QdrantConnectorException.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.exception; - -import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; -import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; - -public class QdrantConnectorException extends SeaTunnelRuntimeException { - public QdrantConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { - super(seaTunnelErrorCode, errorMessage); - } - - public QdrantConnectorException( - SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { - super(seaTunnelErrorCode, errorMessage, cause); - } - - public QdrantConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, Throwable cause) { - super(seaTunnelErrorCode, cause); - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java deleted file mode 100644 index 7ca4428c81e..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantBatchWriter.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.sink; - -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.PrimaryKey; -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.CommonErrorCode; -import org.apache.seatunnel.common.utils.BufferUtils; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; -import org.apache.seatunnel.connectors.seatunnel.qdrant.exception.QdrantConnectorException; - -import org.apache.commons.collections4.CollectionUtils; - -import io.qdrant.client.QdrantClient; -import io.qdrant.client.ValueFactory; -import io.qdrant.client.VectorFactory; -import io.qdrant.client.grpc.JsonWithInt; -import io.qdrant.client.grpc.Points; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; - -import static io.qdrant.client.PointIdFactory.id; -import static org.apache.seatunnel.api.table.catalog.PrimaryKey.isPrimaryKeyField; - -public class QdrantBatchWriter { - - private final int batchSize; - private final CatalogTable catalogTable; - private final String collectionName; - private final QdrantClient qdrantClient; - - private final List qdrantDataCache; - private volatile int writeCount = 0; - - public QdrantBatchWriter( - CatalogTable catalogTable, Integer batchSize, QdrantParameters params) { - this.catalogTable = catalogTable; - this.qdrantClient = params.buildQdrantClient(); - this.collectionName = params.getCollectionName(); - this.batchSize = batchSize; - this.qdrantDataCache = new ArrayList<>(batchSize); - } - - public void addToBatch(SeaTunnelRow element) { - Points.PointStruct point = buildPoint(element); - qdrantDataCache.add(point); - writeCount++; - } - - public boolean needFlush() { - return this.writeCount >= this.batchSize; - } - - public synchronized void flush() { - if (CollectionUtils.isEmpty(this.qdrantDataCache)) { - return; - } - upsert(); - this.qdrantDataCache.clear(); - this.writeCount = 0; - } - - public void close() { - this.qdrantClient.close(); - } - - private Points.PointStruct buildPoint(SeaTunnelRow element) { - SeaTunnelRowType seaTunnelRowType = catalogTable.getSeaTunnelRowType(); - PrimaryKey primaryKey = catalogTable.getTableSchema().getPrimaryKey(); - - Points.PointStruct.Builder point = Points.PointStruct.newBuilder(); - Points.NamedVectors.Builder namedVectors = Points.NamedVectors.newBuilder(); - for (int i = 0; i < seaTunnelRowType.getFieldNames().length; i++) { - String fieldName = seaTunnelRowType.getFieldNames()[i]; - SeaTunnelDataType fieldType = seaTunnelRowType.getFieldType(i); - Object value = element.getField(i); - - if (isPrimaryKeyField(primaryKey, fieldName)) { - point.setId(pointId(fieldType, value)); - continue; - } - - JsonWithInt.Value payloadValue = buildPayload(fieldType, value); - if (payloadValue != null) { - point.putPayload(fieldName, payloadValue); - continue; - } - - Points.Vector vector = buildVector(fieldType, value); - if (vector != null) { - namedVectors.putVectors(fieldName, vector); - } - } - - if (!point.hasId()) { - point.setId(id(UUID.randomUUID())); - } - - point.setVectors(Points.Vectors.newBuilder().setVectors(namedVectors).build()); - return point.build(); - } - - private void upsert() { - try { - qdrantClient - .upsertAsync( - Points.UpsertPoints.newBuilder() - .setCollectionName(collectionName) - .addAllPoints(qdrantDataCache) - .build()) - .get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Upsert failed", e); - } - } - - public static Points.PointId pointId(SeaTunnelDataType fieldType, Object value) { - SqlType sqlType = fieldType.getSqlType(); - switch (sqlType) { - case INT: - return id(Integer.parseInt(value.toString())); - case STRING: - return id(UUID.fromString(value.toString())); - default: - throw new QdrantConnectorException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, - "Unexpected value type for point ID: " + sqlType.name()); - } - } - - public static JsonWithInt.Value buildPayload(SeaTunnelDataType fieldType, Object value) { - SqlType sqlType = fieldType.getSqlType(); - switch (sqlType) { - case SMALLINT: - case INT: - case BIGINT: - return ValueFactory.value(Integer.parseInt(value.toString())); - case FLOAT: - case DOUBLE: - return ValueFactory.value(Long.parseLong(value.toString())); - case STRING: - case DATE: - return ValueFactory.value(value.toString()); - case BOOLEAN: - return ValueFactory.value(Boolean.parseBoolean(value.toString())); - default: - return null; - } - } - - public static Points.Vector buildVector(SeaTunnelDataType fieldType, Object value) { - SqlType sqlType = fieldType.getSqlType(); - switch (sqlType) { - case FLOAT_VECTOR: - case FLOAT16_VECTOR: - case BFLOAT16_VECTOR: - case BINARY_VECTOR: - ByteBuffer floatVectorBuffer = (ByteBuffer) value; - Float[] floats = BufferUtils.toFloatArray(floatVectorBuffer); - return VectorFactory.vector(Arrays.stream(floats).collect(Collectors.toList())); - default: - return null; - } - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java deleted file mode 100644 index 85119032c86..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSink.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.sink; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.sink.SinkWriter; -import org.apache.seatunnel.api.sink.SupportMultiTableSink; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; - -import java.io.IOException; - -public class QdrantSink extends AbstractSimpleSink - implements SupportMultiTableSink { - private final QdrantParameters qdrantParameters; - private final CatalogTable catalogTable; - - public QdrantSink(ReadonlyConfig config, CatalogTable table) { - this.qdrantParameters = new QdrantParameters(config); - this.catalogTable = table; - } - - @Override - public String getPluginName() { - return QdrantConfig.CONNECTOR_IDENTITY; - } - - @Override - public QdrantSinkWriter createWriter(SinkWriter.Context context) throws IOException { - return new QdrantSinkWriter(catalogTable, qdrantParameters); - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java deleted file mode 100644 index a7ed5599d50..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkFactory.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.sink; - -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.sink.SinkCommonOptions; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.connector.TableSink; -import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableSinkFactory; -import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; - -import com.google.auto.service.AutoService; - -@AutoService(Factory.class) -public class QdrantSinkFactory implements TableSinkFactory { - @Override - public String factoryIdentifier() { - return QdrantConfig.CONNECTOR_IDENTITY; - } - - @Override - public TableSink createSink(TableSinkFactoryContext context) { - CatalogTable catalogTable = context.getCatalogTable(); - return () -> new QdrantSink(context.getOptions(), catalogTable); - } - - @Override - public OptionRule optionRule() { - return OptionRule.builder() - .optional( - QdrantConfig.HOST, - QdrantConfig.PORT, - QdrantConfig.API_KEY, - QdrantConfig.USE_TLS, - SinkCommonOptions.MULTI_TABLE_SINK_REPLICA) - .build(); - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java deleted file mode 100644 index a0e00838b6f..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/sink/QdrantSinkWriter.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.sink; - -import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; - -import java.io.IOException; -import java.util.Optional; - -public class QdrantSinkWriter extends AbstractSinkWriter - implements SupportMultiTableSinkWriter { - - private final QdrantBatchWriter batchWriter; - - public QdrantSinkWriter(CatalogTable catalog, QdrantParameters qdrantParameters) { - int batchSize = 64; - this.batchWriter = new QdrantBatchWriter(catalog, batchSize, qdrantParameters); - } - - @Override - public void write(SeaTunnelRow element) throws IOException { - batchWriter.addToBatch(element); - if (batchWriter.needFlush()) { - batchWriter.flush(); - } - } - - @Override - public Optional prepareCommit() { - batchWriter.flush(); - return Optional.empty(); - } - - private void clearBuffer() {} - - @Override - public void close() throws IOException { - batchWriter.flush(); - batchWriter.close(); - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java deleted file mode 100644 index 39aeb3a8793..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSource.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.source; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.source.Boundedness; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitReader; -import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitSource; -import org.apache.seatunnel.connectors.seatunnel.common.source.SingleSplitReaderContext; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; - -import java.util.Collections; -import java.util.List; - -public class QdrantSource extends AbstractSingleSplitSource { - private final QdrantParameters qdrantParameters; - private final CatalogTable catalogTable; - - @Override - public String getPluginName() { - return QdrantConfig.CONNECTOR_IDENTITY; - } - - public QdrantSource(ReadonlyConfig readonlyConfig) { - this.qdrantParameters = new QdrantParameters(readonlyConfig); - this.catalogTable = CatalogTableUtil.buildWithConfig(readonlyConfig); - } - - @Override - public Boundedness getBoundedness() { - return Boundedness.BOUNDED; - } - - @Override - public List getProducedCatalogTables() { - return Collections.singletonList(catalogTable); - } - - @Override - public AbstractSingleSplitReader createReader( - SingleSplitReaderContext readerContext) { - return new QdrantSourceReader(qdrantParameters, readerContext, catalogTable); - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java deleted file mode 100644 index 0639fc07e24..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceFactory.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.source; - -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; -import org.apache.seatunnel.api.table.connector.TableSource; -import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableSourceFactory; -import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantConfig; - -import com.google.auto.service.AutoService; - -import java.io.Serializable; - -@AutoService(Factory.class) -public class QdrantSourceFactory implements TableSourceFactory { - @Override - public String factoryIdentifier() { - return QdrantConfig.CONNECTOR_IDENTITY; - } - - @Override - public - TableSource createSource(TableSourceFactoryContext context) { - return () -> (SeaTunnelSource) new QdrantSource(context.getOptions()); - } - - @Override - public OptionRule optionRule() { - return OptionRule.builder() - .required(QdrantConfig.COLLECTION_NAME, TableSchemaOptions.SCHEMA) - .optional( - QdrantConfig.HOST, - QdrantConfig.PORT, - QdrantConfig.API_KEY, - QdrantConfig.USE_TLS) - .build(); - } - - @Override - public Class getSourceClass() { - return QdrantSource.class; - } -} diff --git a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java b/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java deleted file mode 100644 index 2c371631295..00000000000 --- a/seatunnel-connectors-v2/connector-qdrant/src/main/java/org/apache/seatunnel/connectors/seatunnel/qdrant/source/QdrantSourceReader.java +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.qdrant.source; - -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.PrimaryKey; -import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.TableSchema; -import org.apache.seatunnel.api.table.type.RowKind; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCode; -import org.apache.seatunnel.common.utils.BufferUtils; -import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitReader; -import org.apache.seatunnel.connectors.seatunnel.common.source.SingleSplitReaderContext; -import org.apache.seatunnel.connectors.seatunnel.qdrant.config.QdrantParameters; -import org.apache.seatunnel.connectors.seatunnel.qdrant.exception.QdrantConnectorException; - -import io.qdrant.client.QdrantClient; -import io.qdrant.client.WithVectorsSelectorFactory; -import io.qdrant.client.grpc.JsonWithInt; -import io.qdrant.client.grpc.Points; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import static io.qdrant.client.WithPayloadSelectorFactory.enable; -import static org.apache.seatunnel.api.table.catalog.PrimaryKey.isPrimaryKeyField; - -public class QdrantSourceReader extends AbstractSingleSplitReader { - private final QdrantParameters qdrantParameters; - private final SingleSplitReaderContext context; - private final TableSchema tableSchema; - private final TablePath tablePath; - private QdrantClient qdrantClient; - - public QdrantSourceReader( - QdrantParameters qdrantParameters, - SingleSplitReaderContext context, - CatalogTable catalogTable) { - this.qdrantParameters = qdrantParameters; - this.context = context; - this.tableSchema = catalogTable.getTableSchema(); - this.tablePath = catalogTable.getTablePath(); - } - - @Override - public void open() throws Exception { - qdrantClient = qdrantParameters.buildQdrantClient(); - qdrantClient.healthCheckAsync().get(); - } - - @Override - public void close() { - if (Objects.nonNull(qdrantClient)) { - qdrantClient.close(); - } - } - - @Override - public void internalPollNext(Collector output) throws Exception { - int SCROLL_SIZE = 64; - Points.ScrollPoints request = - Points.ScrollPoints.newBuilder() - .setCollectionName(qdrantParameters.getCollectionName()) - .setLimit(SCROLL_SIZE) - .setWithPayload(enable(true)) - .setWithVectors(WithVectorsSelectorFactory.enable(true)) - .build(); - - while (true) { - Points.ScrollResponse response = qdrantClient.scrollAsync(request).get(); - List points = response.getResultList(); - - for (Points.RetrievedPoint point : points) { - SeaTunnelRow seaTunnelRow = convertToSeaTunnelRow(point); - output.collect(seaTunnelRow); - } - - Points.PointId offset = response.getNextPageOffset(); - - if (!offset.hasNum() && !offset.hasUuid()) break; - - request = request.toBuilder().setOffset(offset).build(); - } - - context.signalNoMoreElement(); - } - - private SeaTunnelRow convertToSeaTunnelRow(Points.RetrievedPoint point) { - SeaTunnelRowType typeInfo = tableSchema.toPhysicalRowDataType(); - PrimaryKey primaryKey = tableSchema.getPrimaryKey(); - Map payloadMap = point.getPayloadMap(); - Points.Vectors vectors = point.getVectors(); - Map vectorsMap = new HashMap<>(); - String DEFAULT_VECTOR_KEY = "default_vector"; - - if (vectors.hasVector()) { - vectorsMap.put(DEFAULT_VECTOR_KEY, vectors.getVector()); - } else if (vectors.hasVectors()) { - vectorsMap = vectors.getVectors().getVectorsMap(); - } - Object[] fields = new Object[typeInfo.getTotalFields()]; - String[] fieldNames = typeInfo.getFieldNames(); - for (int fieldIndex = 0; fieldIndex < typeInfo.getTotalFields(); fieldIndex++) { - SeaTunnelDataType seaTunnelDataType = typeInfo.getFieldType(fieldIndex); - String fieldName = fieldNames[fieldIndex]; - - if (isPrimaryKeyField(primaryKey, fieldName)) { - Points.PointId id = point.getId(); - if (id.hasNum()) { - fields[fieldIndex] = id.getNum(); - } else if (id.hasUuid()) { - fields[fieldIndex] = id.getUuid(); - } - continue; - } - JsonWithInt.Value value = payloadMap.get(fieldName); - Points.Vector vector = vectorsMap.get(fieldName); - switch (seaTunnelDataType.getSqlType()) { - case NULL: - fields[fieldIndex] = null; - break; - case STRING: - fields[fieldIndex] = value.getStringValue(); - break; - case BOOLEAN: - fields[fieldIndex] = value.getBoolValue(); - break; - case TINYINT: - case SMALLINT: - case INT: - case BIGINT: - fields[fieldIndex] = value.getIntegerValue(); - break; - case FLOAT: - case DECIMAL: - case DOUBLE: - fields[fieldIndex] = value.getDoubleValue(); - break; - case BINARY_VECTOR: - case FLOAT_VECTOR: - case FLOAT16_VECTOR: - case BFLOAT16_VECTOR: - List list = vector.getDataList(); - Float[] vectorArray = new Float[list.size()]; - list.toArray(vectorArray); - fields[fieldIndex] = BufferUtils.toByteBuffer(vectorArray); - break; - default: - throw new QdrantConnectorException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, - "Unexpected value: " + seaTunnelDataType.getSqlType().name()); - } - } - - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(fields); - seaTunnelRow.setTableId(tablePath.getFullName()); - seaTunnelRow.setRowKind(RowKind.INSERT); - return seaTunnelRow; - } -} diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java index 3f5c862cadf..82ae2728d67 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/client/RabbitmqClient.java @@ -189,16 +189,11 @@ public void close() { protected void setupQueue() throws IOException { if (config.getQueueName() != null) { - declareQueueDefaults(channel, config); + declareQueueDefaults(channel, config.getQueueName()); } } - private void declareQueueDefaults(Channel channel, RabbitmqConfig config) throws IOException { - channel.queueDeclare( - config.getQueueName(), - config.getDurable(), - config.getExclusive(), - config.getAutoDelete(), - null); + private void declareQueueDefaults(Channel channel, String queueName) throws IOException { + channel.queueDeclare(queueName, true, false, false, null); } } diff --git a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java index 8475817457a..e8e2ce55c30 100644 --- a/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java +++ b/seatunnel-connectors-v2/connector-rabbitmq/src/main/java/org/apache/seatunnel/connectors/seatunnel/rabbitmq/config/RabbitmqConfig.java @@ -53,9 +53,6 @@ public class RabbitmqConfig implements Serializable { private Integer prefetchCount; private long deliveryTimeout; private String queueName; - private Boolean durable; - private Boolean exclusive; - private Boolean autoDelete; private String routingKey; private boolean logFailuresOnly = false; private String exchange = ""; @@ -198,30 +195,6 @@ public class RabbitmqConfig implements Serializable { "Whether the messages received are supplied with a unique" + "id to deduplicate messages (in case of failed acknowledgments)."); - public static final Option DURABLE = - Options.key("durable") - .booleanType() - .defaultValue(true) - .withDescription( - "true: The queue will survive a server restart." - + "false: The queue will be deleted on server restart."); - - public static final Option EXCLUSIVE = - Options.key("exclusive") - .booleanType() - .defaultValue(false) - .withDescription( - "true: The queue is used only by the current connection and will be deleted when the connection closes." - + "false: The queue can be used by multiple connections."); - - public static final Option AUTO_DELETE = - Options.key("auto_delete") - .booleanType() - .defaultValue(false) - .withDescription( - "true: The queue will be deleted automatically when the last consumer unsubscribes." - + "false: The queue will not be automatically deleted."); - private void parseSinkOptionProperties(Config pluginConfig) { if (CheckConfigUtil.isValidParam(pluginConfig, RABBITMQ_CONFIG.key())) { pluginConfig @@ -286,15 +259,6 @@ public RabbitmqConfig(Config config) { if (config.hasPath(USE_CORRELATION_ID.key())) { this.usesCorrelationId = config.getBoolean(USE_CORRELATION_ID.key()); } - if (config.hasPath(DURABLE.key())) { - this.durable = config.getBoolean(DURABLE.key()); - } - if (config.hasPath(EXCLUSIVE.key())) { - this.exclusive = config.getBoolean(EXCLUSIVE.key()); - } - if (config.hasPath(AUTO_DELETE.key())) { - this.autoDelete = config.getBoolean(AUTO_DELETE.key()); - } parseSinkOptionProperties(config); } diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java index be121818932..7b2aa6bae67 100644 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java +++ b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java @@ -19,14 +19,11 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; - import lombok.AllArgsConstructor; import lombok.Data; import java.io.Serializable; import java.util.List; -import java.util.Map; import static org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreConfig.BATCH_SIZE; @@ -48,8 +45,6 @@ public class TablestoreOptions implements Serializable { public int batchSize = Integer.parseInt(BATCH_SIZE.defaultValue()); - public TablestoreOptions() {} - public TablestoreOptions(Config config) { this.endpoint = config.getString(TablestoreConfig.END_POINT.key()); this.instanceName = config.getString(TablestoreConfig.INSTANCE_NAME.key()); @@ -62,18 +57,4 @@ public TablestoreOptions(Config config) { this.batchSize = config.getInt(BATCH_SIZE.key()); } } - - public static TablestoreOptions of(ReadonlyConfig config) { - Map map = config.getSourceMap(); - TablestoreOptions tablestoreOptions = new TablestoreOptions(); - tablestoreOptions.setEndpoint(config.get(TablestoreConfig.END_POINT)); - tablestoreOptions.setInstanceName(config.get(TablestoreConfig.INSTANCE_NAME)); - tablestoreOptions.setAccessKeyId(config.get(TablestoreConfig.ACCESS_KEY_ID)); - tablestoreOptions.setAccessKeySecret(config.get(TablestoreConfig.ACCESS_KEY_SECRET)); - tablestoreOptions.setTable(config.get(TablestoreConfig.TABLE)); - List keys = (List) map.get(TablestoreConfig.PRIMARY_KEYS.key()); - - tablestoreOptions.setPrimaryKeys(keys); - return tablestoreOptions; - } } diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java deleted file mode 100644 index 9bdb060a49d..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/DefaultSeaTunnelRowDeserializer.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.serialize; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -import com.alicloud.openservices.tablestore.model.StreamRecord; - -import java.util.ArrayList; -import java.util.List; - -public class DefaultSeaTunnelRowDeserializer implements SeaTunnelRowDeserializer { - - @Override - public SeaTunnelRow deserialize(StreamRecord r) { - List fields = new ArrayList<>(); - r.getColumns() - .forEach( - k -> { - fields.add(k.getColumn().getValue()); - }); - return new SeaTunnelRow(fields.toArray()); - } -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java deleted file mode 100644 index 44a2560693f..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/serialize/SeaTunnelRowDeserializer.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.serialize; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -import com.alicloud.openservices.tablestore.model.StreamRecord; - -public interface SeaTunnelRowDeserializer { - - SeaTunnelRow deserialize(StreamRecord streamRecord); -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java deleted file mode 100644 index 85c0062ed32..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSource.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.source; - -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.source.Boundedness; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.source.SourceReader.Context; -import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.source.SupportColumnProjection; -import org.apache.seatunnel.api.source.SupportParallelism; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.constants.JobMode; -import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions; - -import lombok.extern.slf4j.Slf4j; - -import java.util.List; - -@Slf4j -public class TableStoreDBSource - implements SeaTunnelSource, - SupportParallelism, - SupportColumnProjection { - - private TablestoreOptions tablestoreOptions; - private SeaTunnelRowType typeInfo; - private JobContext jobContext; - - @Override - public String getPluginName() { - return "Tablestore"; - } - - @Override - public List getProducedCatalogTables() { - return SeaTunnelSource.super.getProducedCatalogTables(); - } - - public TableStoreDBSource(ReadonlyConfig config) { - this.tablestoreOptions = TablestoreOptions.of(config); - CatalogTableUtil.buildWithConfig(config); - this.typeInfo = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); - } - - @Override - public Boundedness getBoundedness() { - return JobMode.BATCH.equals(jobContext.getJobMode()) - ? Boundedness.BOUNDED - : Boundedness.UNBOUNDED; - } - - @Override - public SourceReader createReader(Context readerContext) - throws Exception { - return new TableStoreDBSourceReader(readerContext, tablestoreOptions, typeInfo); - } - - @Override - public SourceSplitEnumerator createEnumerator( - org.apache.seatunnel.api.source.SourceSplitEnumerator.Context - enumeratorContext) - throws Exception { - return new TableStoreDBSourceSplitEnumerator(enumeratorContext, tablestoreOptions); - } - - @Override - public SourceSplitEnumerator - restoreEnumerator( - org.apache.seatunnel.api.source.SourceSplitEnumerator.Context< - TableStoreDBSourceSplit> - enumeratorContext, - TableStoreDBSourceState checkpointState) - throws Exception { - return new TableStoreDBSourceSplitEnumerator( - enumeratorContext, tablestoreOptions, checkpointState); - } - - @Override - public void setJobContext(JobContext jobContext) { - this.jobContext = jobContext; - } -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java deleted file mode 100644 index eefd4aae031..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceReader.java +++ /dev/null @@ -1,175 +0,0 @@ -package org.apache.seatunnel.connectors.seatunnel.tablestore.source; -/* - * 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.source.Collector; -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions; - -import com.alicloud.openservices.tablestore.SyncClient; -import com.alicloud.openservices.tablestore.TunnelClient; -import com.alicloud.openservices.tablestore.model.tunnel.CreateTunnelRequest; -import com.alicloud.openservices.tablestore.model.tunnel.CreateTunnelResponse; -import com.alicloud.openservices.tablestore.model.tunnel.DeleteTunnelRequest; -import com.alicloud.openservices.tablestore.model.tunnel.DeleteTunnelResponse; -import com.alicloud.openservices.tablestore.model.tunnel.DescribeTunnelRequest; -import com.alicloud.openservices.tablestore.model.tunnel.DescribeTunnelResponse; -import com.alicloud.openservices.tablestore.model.tunnel.TunnelType; -import com.alicloud.openservices.tablestore.tunnel.worker.TunnelWorker; -import com.alicloud.openservices.tablestore.tunnel.worker.TunnelWorkerConfig; -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedDeque; - -@Slf4j -public class TableStoreDBSourceReader - implements SourceReader { - - protected SourceReader.Context context; - protected TablestoreOptions tablestoreOptions; - protected SeaTunnelRowType seaTunnelRowType; - Queue pendingSplits = new ConcurrentLinkedDeque<>(); - private SyncClient client; - private volatile boolean noMoreSplit; - private TunnelClient tunnelClient; - - public TableStoreDBSourceReader( - SourceReader.Context context, - TablestoreOptions options, - SeaTunnelRowType seaTunnelRowType) { - - this.context = context; - this.tablestoreOptions = options; - this.seaTunnelRowType = seaTunnelRowType; - } - - @Override - public void open() throws Exception { - client = - new SyncClient( - tablestoreOptions.getEndpoint(), - tablestoreOptions.getAccessKeyId(), - tablestoreOptions.getAccessKeySecret(), - tablestoreOptions.getInstanceName()); - tunnelClient = - new TunnelClient( - tablestoreOptions.getEndpoint(), - tablestoreOptions.getAccessKeyId(), - tablestoreOptions.getAccessKeySecret(), - tablestoreOptions.getInstanceName()); - } - - @Override - public void close() throws IOException { - tunnelClient.shutdown(); - client.shutdown(); - } - - @Override - public void pollNext(Collector output) throws Exception { - synchronized (output.getCheckpointLock()) { - TableStoreDBSourceSplit split = pendingSplits.poll(); - if (Objects.nonNull(split)) { - read(split, output); - } - /*if (split == null) { - log.info( - "TableStore Source Reader [{}] waiting for splits", - context.getIndexOfSubtask()); - }*/ - if (noMoreSplit) { - // signal to the source that we have reached the end of the data. - log.info("Closed the bounded tablestore source"); - context.signalNoMoreElement(); - Thread.sleep(2000L); - } else { - Thread.sleep(1000L); - } - } - } - - private void read(TableStoreDBSourceSplit split, Collector output) { - String tunnelId = getTunel(split); - TableStoreProcessor processor = - new TableStoreProcessor(split.getTableName(), split.getPrimaryKey(), output); - TunnelWorkerConfig workerConfig = new TunnelWorkerConfig(processor); - TunnelWorker worker = new TunnelWorker(tunnelId, tunnelClient, workerConfig); - try { - worker.connectAndWorking(); - } catch (Exception e) { - log.error("Start OTS tunnel failed.", e); - worker.shutdown(); - } - } - - public String getTunel(TableStoreDBSourceSplit split) { - deleteTunel(split); - String tunnelId = null; - String tunnelName = split.getTableName() + "_migration2aws_tunnel4" + split.getSplitId(); - - try { - DescribeTunnelRequest drequest = new DescribeTunnelRequest("test", tunnelName); - DescribeTunnelResponse dresp = tunnelClient.describeTunnel(drequest); - tunnelId = dresp.getTunnelInfo().getTunnelId(); - } catch (Exception be) { - CreateTunnelRequest crequest = - new CreateTunnelRequest( - split.getTableName(), tunnelName, TunnelType.valueOf("BaseAndStream")); - CreateTunnelResponse cresp = tunnelClient.createTunnel(crequest); - tunnelId = cresp.getTunnelId(); - } - log.info("Tunnel found, Id: " + tunnelId); - return tunnelId; - } - - public void deleteTunel(TableStoreDBSourceSplit split) { - String tunnelName = split.getTableName() + "_migration2aws_tunnel4" + split.getSplitId(); - try { - DeleteTunnelRequest drequest = - new DeleteTunnelRequest(split.getTableName(), tunnelName); - DeleteTunnelResponse dresp = tunnelClient.deleteTunnel(drequest); - log.info("Tunnel has been deleted: " + dresp.toString()); - } catch (Exception be) { - log.warn("Tunnel deletion failed due to not found: " + tunnelName); - } - } - - @Override - public List snapshotState(long checkpointId) throws Exception { - return new ArrayList<>(pendingSplits); - } - - @Override - public void addSplits(List splits) { - this.pendingSplits.addAll(splits); - } - - @Override - public void handleNoMoreSplits() { - log.info("Reader [{}] received noMoreSplit event.", context.getIndexOfSubtask()); - noMoreSplit = true; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception {} -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java deleted file mode 100644 index 24328b0a6f9..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplit.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.source; - -import org.apache.seatunnel.api.source.SourceSplit; - -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.Setter; - -@AllArgsConstructor -@Getter -@Setter -public class TableStoreDBSourceSplit implements SourceSplit { - - private Integer splitId; - private String tableName; - private String primaryKey; - - @Override - public String splitId() { - return splitId.toString(); - } -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java deleted file mode 100644 index 3dd58b7e69b..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceSplitEnumerator.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.source; - -import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions; - -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -@Slf4j -public class TableStoreDBSourceSplitEnumerator - implements SourceSplitEnumerator { - - private final SourceSplitEnumerator.Context enumeratorContext; - private final Map> pendingSplits; - private final TablestoreOptions tablestoreOptions; - - private final Object stateLock = new Object(); - private volatile boolean shouldEnumerate; - - /** - * @param enumeratorContext - * @param tablestoreOptions - */ - public TableStoreDBSourceSplitEnumerator( - Context enumeratorContext, - TablestoreOptions tablestoreOptions) { - this(enumeratorContext, tablestoreOptions, null); - } - - public TableStoreDBSourceSplitEnumerator( - Context enumeratorContext, - TablestoreOptions tablestoreOptions, - TableStoreDBSourceState sourceState) { - this.enumeratorContext = enumeratorContext; - this.tablestoreOptions = tablestoreOptions; - this.pendingSplits = new HashMap<>(); - this.shouldEnumerate = sourceState == null; - if (sourceState != null) { - this.shouldEnumerate = sourceState.isShouldEnumerate(); - this.pendingSplits.putAll(sourceState.getPendingSplits()); - } - } - - @Override - public void open() {} - - @Override - public void run() throws Exception { - Set readers = enumeratorContext.registeredReaders(); - if (shouldEnumerate) { - Set newSplits = getTableStoreDBSourceSplit(); - synchronized (stateLock) { - addPendingSplit(newSplits); - shouldEnumerate = false; - } - assignSplit(readers); - } - } - - private void assignSplit(Set readers) { - for (int reader : readers) { - List assignmentForReader = pendingSplits.remove(reader); - if (assignmentForReader != null && !assignmentForReader.isEmpty()) { - log.info("Assign splits {} to reader {}", assignmentForReader, reader); - try { - enumeratorContext.assignSplit(reader, assignmentForReader); - } catch (Exception e) { - log.error( - "Failed to assign splits {} to reader {}", - assignmentForReader, - reader, - e); - pendingSplits.put(reader, assignmentForReader); - } - } - } - } - - private Set getTableStoreDBSourceSplit() { - - Set allSplit = new HashSet<>(); - String tables = tablestoreOptions.getTable(); - String[] tableArr = tables.split(","); - for (int i = 0; i < tableArr.length; i++) { - allSplit.add( - new TableStoreDBSourceSplit( - i, tableArr[i], tablestoreOptions.getPrimaryKeys().get(i))); - } - return allSplit; - } - - private void addPendingSplit(Collection splits) { - int readerCount = enumeratorContext.currentParallelism(); - for (TableStoreDBSourceSplit split : splits) { - int ownerReader = split.getSplitId() % readerCount; - pendingSplits.computeIfAbsent(ownerReader, k -> new ArrayList<>()).add(split); - } - } - - @Override - public void close() throws IOException { - // TODO Auto-generated method stub - throw new UnsupportedOperationException("Unimplemented method 'close'"); - } - - @Override - public void addSplitsBack(List splits, int subtaskId) { - log.debug("Add back splits {} to tablestore.", splits); - if (!splits.isEmpty()) { - addPendingSplit(splits); - assignSplit(Collections.singleton(subtaskId)); - enumeratorContext.signalNoMoreSplits(subtaskId); - } - } - - @Override - public int currentUnassignedSplitSize() { - return pendingSplits.size(); - } - - @Override - public void handleSplitRequest(int subtaskId) {} - - @Override - public void registerReader(int subtaskId) { - log.debug("Register reader {} to TablestoreSplitEnumerator.", subtaskId); - if (!pendingSplits.isEmpty()) { - assignSplit(Collections.singleton(subtaskId)); - } - } - - @Override - public TableStoreDBSourceState snapshotState(long checkpointId) throws Exception { - synchronized (stateLock) { - return new TableStoreDBSourceState(shouldEnumerate, pendingSplits); - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception {} -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java deleted file mode 100644 index 05a73a63101..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDBSourceState.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.source; - -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.Setter; - -import java.io.Serializable; -import java.util.List; -import java.util.Map; - -@Getter -@Setter -@AllArgsConstructor -public class TableStoreDBSourceState implements Serializable { - - private boolean shouldEnumerate; - private Map> pendingSplits; -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java deleted file mode 100644 index f93ae4bfe32..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreDbSourceFactory.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.source; - -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.table.connector.TableSource; -import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableSourceFactory; -import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; -import org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreConfig; - -import com.google.auto.service.AutoService; - -import java.io.Serializable; - -@AutoService(Factory.class) -public class TableStoreDbSourceFactory implements TableSourceFactory { - - @Override - public String factoryIdentifier() { - return "Tablestore"; - } - - @Override - public OptionRule optionRule() { - return OptionRule.builder() - .required( - TablestoreConfig.END_POINT, - TablestoreConfig.INSTANCE_NAME, - TablestoreConfig.ACCESS_KEY_ID, - TablestoreConfig.ACCESS_KEY_SECRET, - TablestoreConfig.TABLE, - TablestoreConfig.PRIMARY_KEYS) - .build(); - } - - @Override - public - TableSource createSource(TableSourceFactoryContext context) { - return () -> - (SeaTunnelSource) new TableStoreDBSource(context.getOptions()); - } - - @Override - public Class getSourceClass() { - return TableStoreDBSource.class; - } -} diff --git a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java b/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java deleted file mode 100644 index ba5334a85eb..00000000000 --- a/seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/source/TableStoreProcessor.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.connectors.seatunnel.tablestore.source; - -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.type.RowKind; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.tablestore.serialize.SeaTunnelRowDeserializer; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alicloud.openservices.tablestore.model.StreamRecord; -import com.alicloud.openservices.tablestore.tunnel.worker.IChannelProcessor; -import com.alicloud.openservices.tablestore.tunnel.worker.ProcessRecordsInput; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class TableStoreProcessor implements IChannelProcessor { - private String tableName = null; - private String primaryKey = null; - private Collector output = null; - protected SeaTunnelRowDeserializer seaTunnelRowDeserializer; - private static final Logger log = LoggerFactory.getLogger(TableStoreProcessor.class); - - public TableStoreProcessor( - String tableName, String primaryKey, Collector output) { - this.tableName = tableName; - this.primaryKey = primaryKey; - this.output = output; - } - - @Override - public void process(ProcessRecordsInput input) { - log.info("Default record processor, would print records count"); - - log.info( - String.format( - "Process %d records, NextToken: %s", - input.getRecords().size(), input.getNextToken())); - - for (StreamRecord r : input.getRecords()) { - try { - List fields = new ArrayList<>(); - Arrays.stream(r.getPrimaryKey().getPrimaryKeyColumns()) - .forEach( - k -> { - fields.add(k.getValue().toString()); - }); - r.getColumns() - .forEach( - k -> { - fields.add(k.getColumn().getValue().toString()); - }); - SeaTunnelRow row = new SeaTunnelRow(fields.toArray()); - row.setTableId(tableName); - switch ((r.getRecordType())) { - case PUT: - row.setRowKind(RowKind.INSERT); - break; - case UPDATE: - row.setRowKind(RowKind.UPDATE_AFTER); - break; - case DELETE: - row.setRowKind(RowKind.DELETE); - break; - } - output.collect(row); - } catch (Exception e) { - log.error("send to target failed with record: " + r.toString(), e); - } - } - } - - @Override - public void shutdown() { - log.info("process shutdown du to finished for table: " + tableName); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/pom.xml b/seatunnel-connectors-v2/connector-typesense/pom.xml deleted file mode 100644 index 57a8682cb44..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/pom.xml +++ /dev/null @@ -1,90 +0,0 @@ - - - - 4.0.0 - - org.apache.seatunnel - seatunnel-connectors-v2 - ${revision} - - - connector-typesense - SeaTunnel : Connectors V2 : Typesense - - - 0.8.1 - - - - - com.fasterxml.jackson.core - jackson-databind - 2.14.1 - - - - org.typesense - typesense-java - 0.8.1 - - - org.apache.seatunnel - connector-common - ${project.version} - compile - - - - org.apache.seatunnel - seatunnel-format-json - ${project.version} - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - - shade - - package - - - - okhttp3 - shaded.okhttp3 - - - okio - shaded.okio - - - false - - - - - - - - diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java deleted file mode 100644 index fd73c247497..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalog.java +++ /dev/null @@ -1,214 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.catalog; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.configuration.util.ConfigUtil; -import org.apache.seatunnel.api.table.catalog.Catalog; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.InfoPreviewResult; -import org.apache.seatunnel.api.table.catalog.PhysicalColumn; -import org.apache.seatunnel.api.table.catalog.PreviewResult; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; -import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.TableSchema; -import org.apache.seatunnel.api.table.catalog.exception.CatalogException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseAlreadyExistException; -import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; -import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException; -import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; -import org.apache.seatunnel.api.table.converter.BasicTypeDefine; -import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; -import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType; - -import lombok.extern.slf4j.Slf4j; - -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -import static com.google.common.base.Preconditions.checkNotNull; - -@Slf4j -public class TypesenseCatalog implements Catalog { - - private final String catalogName; - private final String defaultDatabase; - - private final ReadonlyConfig config; - private TypesenseClient typesenseClient; - - public TypesenseCatalog(String catalogName, String defaultDatabase, ReadonlyConfig config) { - this.catalogName = checkNotNull(catalogName, "catalogName cannot be null"); - this.defaultDatabase = defaultDatabase; - this.config = checkNotNull(config, "Typesense Config cannot be null"); - } - - @Override - public void open() throws CatalogException { - typesenseClient = TypesenseClient.createInstance(config); - } - - @Override - public void close() throws CatalogException { - // Nothing - } - - @Override - public String name() { - return catalogName; - } - - @Override - public String getDefaultDatabase() throws CatalogException { - return defaultDatabase; - } - - @Override - public boolean databaseExists(String databaseName) throws CatalogException { - return typesenseClient.collectionExists(databaseName); - } - - @Override - public List listDatabases() throws CatalogException { - return typesenseClient.collectionList(); - } - - @Override - public List listTables(String databaseName) - throws CatalogException, DatabaseNotExistException { - if (!databaseExists(databaseName)) { - throw new DatabaseNotExistException(catalogName, databaseName); - } - return Arrays.asList(databaseName); - } - - @Override - public boolean tableExists(TablePath tablePath) throws CatalogException { - checkNotNull(tablePath); - return databaseExists(tablePath.getTableName()); - } - - @Override - public CatalogTable getTable(TablePath tablePath) - throws CatalogException, TableNotExistException { - checkNotNull(tablePath, "tablePath cannot be null"); - TableSchema.Builder builder = TableSchema.builder(); - Map> fieldTypeMapping = - typesenseClient.getFieldTypeMapping(tablePath.getTableName()); - buildColumnsWithErrorCheck( - tablePath, - builder, - fieldTypeMapping.entrySet().iterator(), - nameAndType -> { - return PhysicalColumn.of( - nameAndType.getKey(), - TypesenseTypeConverter.INSTANCE - .convert(nameAndType.getValue()) - .getDataType(), - (Long) null, - true, - null, - null); - }); - - return CatalogTable.of( - TableIdentifier.of( - catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), - builder.build(), - buildTableOptions(tablePath), - Collections.emptyList(), - ""); - } - - private Map buildTableOptions(TablePath tablePath) { - Map options = new HashMap<>(); - options.put("connector", "typesense"); - options.put("config", ConfigUtil.convertToJsonString(tablePath)); - return options; - } - - @Override - public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists) - throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { - checkNotNull(tablePath, "tablePath cannot be null"); - typesenseClient.createCollection(tablePath.getTableName()); - } - - @Override - public void dropTable(TablePath tablePath, boolean ignoreIfNotExists) - throws TableNotExistException, CatalogException { - checkNotNull(tablePath); - if (!tableExists(tablePath) && !ignoreIfNotExists) { - throw new TableNotExistException(catalogName, tablePath); - } - try { - typesenseClient.dropCollection(tablePath.getTableName()); - } catch (Exception ex) { - throw new CatalogException( - String.format( - "Failed to drop table %s in catalog %s", - tablePath.getTableName(), catalogName), - ex); - } - } - - @Override - public void createDatabase(TablePath tablePath, boolean ignoreIfExists) - throws DatabaseAlreadyExistException, CatalogException { - createTable(tablePath, null, ignoreIfExists); - } - - @Override - public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists) - throws DatabaseNotExistException, CatalogException { - dropTable(tablePath, ignoreIfNotExists); - } - - @Override - public void truncateTable(TablePath tablePath, boolean ignoreIfNotExists) { - typesenseClient.truncateCollectionData(tablePath.getTableName()); - } - - @Override - public boolean isExistsData(TablePath tablePath) { - return typesenseClient.collectionDocNum(tablePath.getTableName()) > 0; - } - - @Override - public PreviewResult previewAction( - ActionType actionType, TablePath tablePath, Optional catalogTable) { - if (actionType == ActionType.CREATE_TABLE) { - return new InfoPreviewResult("create collection " + tablePath.getTableName()); - } else if (actionType == ActionType.DROP_TABLE) { - return new InfoPreviewResult("delete collection " + tablePath.getTableName()); - } else if (actionType == ActionType.TRUNCATE_TABLE) { - return new InfoPreviewResult( - "delete and create collection " + tablePath.getTableName()); - } else if (actionType == ActionType.CREATE_DATABASE) { - return new InfoPreviewResult("create collection " + tablePath.getTableName()); - } else if (actionType == ActionType.DROP_DATABASE) { - return new InfoPreviewResult("delete collection " + tablePath.getTableName()); - } else { - throw new UnsupportedOperationException("Unsupported action type: " + actionType); - } - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java deleted file mode 100644 index 0b619ad3052..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseCatalogFactory.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.catalog; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.table.catalog.Catalog; -import org.apache.seatunnel.api.table.factory.CatalogFactory; -import org.apache.seatunnel.api.table.factory.Factory; - -import com.google.auto.service.AutoService; - -@AutoService(Factory.class) -public class TypesenseCatalogFactory implements CatalogFactory { - - @Override - public Catalog createCatalog(String catalogName, ReadonlyConfig options) { - return new TypesenseCatalog(catalogName, "", options); - } - - @Override - public String factoryIdentifier() { - return "Typesense"; - } - - @Override - public OptionRule optionRule() { - return OptionRule.builder().build(); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java deleted file mode 100644 index c4cb862e29b..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/catalog/TypesenseTypeConverter.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.catalog; - -import org.apache.seatunnel.api.table.catalog.Column; -import org.apache.seatunnel.api.table.catalog.PhysicalColumn; -import org.apache.seatunnel.api.table.converter.BasicTypeConverter; -import org.apache.seatunnel.api.table.converter.BasicTypeDefine; -import org.apache.seatunnel.api.table.converter.TypeConverter; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType; - -import com.google.auto.service.AutoService; - -import java.util.Map; - -import static org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType.INT32; -import static org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseType.INT64; - -@AutoService(TypeConverter.class) -public class TypesenseTypeConverter implements BasicTypeConverter> { - public static final TypesenseTypeConverter INSTANCE = new TypesenseTypeConverter(); - - @Override - public String identifier() { - return "Typesense"; - } - - @Override - public Column convert(BasicTypeDefine typeDefine) { - PhysicalColumn.PhysicalColumnBuilder builder = - PhysicalColumn.builder() - .name(typeDefine.getName()) - .sourceType(typeDefine.getColumnType()) - .nullable(typeDefine.isNullable()) - .defaultValue(typeDefine.getDefaultValue()) - .comment(typeDefine.getComment()); - String type = typeDefine.getDataType().toLowerCase(); - switch (type) { - case INT32: - builder.dataType(BasicType.INT_TYPE); - break; - case INT64: - builder.dataType(BasicType.LONG_TYPE); - break; - case TypesenseType.FLOAT: - builder.dataType(BasicType.FLOAT_TYPE); - break; - case TypesenseType.BOOL: - builder.dataType(BasicType.BOOLEAN_TYPE); - break; - case TypesenseType.OBJET: - Map> typeInfo = - (Map) typeDefine.getNativeType().getOptions(); - SeaTunnelRowType object = - new SeaTunnelRowType( - typeInfo.keySet().toArray(new String[0]), - typeInfo.values().stream() - .map(this::convert) - .map(Column::getDataType) - .toArray(SeaTunnelDataType[]::new)); - builder.dataType(object); - break; - case TypesenseType.STRING: - case TypesenseType.IMAGE: - default: - builder.dataType(BasicType.STRING_TYPE); - break; - } - return builder.build(); - } - - @Override - public BasicTypeDefine reconvert(Column column) { - throw new UnsupportedOperationException("Unsupported operation"); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java deleted file mode 100644 index 01b848492fb..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseClient.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.client; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.table.converter.BasicTypeDefine; -import org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig; -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; -import org.apache.seatunnel.connectors.seatunnel.typesense.util.URLParamsConverter; - -import org.apache.commons.lang3.StringUtils; - -import org.typesense.api.Client; -import org.typesense.api.Collections; -import org.typesense.api.Configuration; -import org.typesense.api.FieldTypes; -import org.typesense.model.CollectionResponse; -import org.typesense.model.CollectionSchema; -import org.typesense.model.DeleteDocumentsParameters; -import org.typesense.model.Field; -import org.typesense.model.ImportDocumentsParameters; -import org.typesense.model.SearchParameters; -import org.typesense.model.SearchResult; -import org.typesense.resources.Node; - -import com.fasterxml.jackson.databind.ObjectMapper; -import lombok.extern.slf4j.Slf4j; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig.QUERY_BATCH_SIZE; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.CREATE_COLLECTION_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.DELETE_COLLECTION_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.DROP_COLLECTION_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.FIELD_TYPE_MAPPING_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.INSERT_DOC_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_EXISTS_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_LIST_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.QUERY_COLLECTION_NUM_ERROR; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.TRUNCATE_COLLECTION_ERROR; - -@Slf4j -public class TypesenseClient { - private final Client tsClient; - - TypesenseClient(Client tsClient) { - this.tsClient = tsClient; - } - - public static TypesenseClient createInstance(ReadonlyConfig config) { - List hosts = config.get(TypesenseConnectionConfig.HOSTS); - String protocol = config.get(TypesenseConnectionConfig.protocol); - String apiKey = config.get(TypesenseConnectionConfig.APIKEY); - return createInstance(hosts, apiKey, protocol); - } - - public static TypesenseClient createInstance( - List hosts, String apiKey, String protocol) { - List nodes = new ArrayList<>(); - - hosts.stream() - .map(host -> host.split(":")) - .forEach( - split -> - nodes.add( - new Node( - protocol, - split[0], - StringUtils.isBlank(split[1]) - ? "8018" - : split[1]))); - - Configuration configuration = new Configuration(nodes, Duration.ofSeconds(5), apiKey); - Client client = new Client(configuration); - return new TypesenseClient(client); - } - - public void insert(String collection, List documentList) { - - ImportDocumentsParameters queryParameters = new ImportDocumentsParameters(); - queryParameters.action("upsert"); - String text = ""; - for (String s : documentList) { - text = text + s + "\n"; - } - try { - tsClient.collections(collection).documents().import_(text, queryParameters); - } catch (Exception e) { - log.error(INSERT_DOC_ERROR.getDescription()); - throw new TypesenseConnectorException( - INSERT_DOC_ERROR, INSERT_DOC_ERROR.getDescription()); - } - } - - public SearchResult search(String collection, String query, int offset) throws Exception { - return search(collection, query, offset, QUERY_BATCH_SIZE.defaultValue()); - } - - public SearchResult search(String collection, String query, int offset, int pageSize) - throws Exception { - SearchParameters searchParameters; - if (StringUtils.isNotBlank(query)) { - String jsonQuery = URLParamsConverter.convertParamsToJson(query); - ObjectMapper objectMapper = new ObjectMapper(); - searchParameters = objectMapper.readValue(jsonQuery, SearchParameters.class); - } else { - searchParameters = new SearchParameters().q("*"); - } - log.debug("Typesense query param:{}", searchParameters); - searchParameters.offset(offset); - searchParameters.perPage(pageSize); - SearchResult searchResult = - tsClient.collections(collection).documents().search(searchParameters); - return searchResult; - } - - public boolean collectionExists(String collection) { - try { - Collections collections = tsClient.collections(); - CollectionResponse[] collectionResponses = collections.retrieve(); - for (CollectionResponse collectionRespons : collectionResponses) { - String collectionName = collectionRespons.getName(); - if (collection.equals(collectionName)) { - return true; - } - } - } catch (Exception e) { - log.error(QUERY_COLLECTION_EXISTS_ERROR.getDescription()); - throw new TypesenseConnectorException( - QUERY_COLLECTION_EXISTS_ERROR, QUERY_COLLECTION_EXISTS_ERROR.getDescription()); - } - return false; - } - - public List collectionList() { - try { - Collections collections = tsClient.collections(); - CollectionResponse[] collectionResponses = collections.retrieve(); - List list = new ArrayList<>(); - for (CollectionResponse collectionRespons : collectionResponses) { - String collectionName = collectionRespons.getName(); - list.add(collectionName); - } - return list; - } catch (Exception e) { - log.error(QUERY_COLLECTION_LIST_ERROR.getDescription()); - throw new TypesenseConnectorException( - QUERY_COLLECTION_LIST_ERROR, QUERY_COLLECTION_LIST_ERROR.getDescription()); - } - } - - public Map getField(String collection) { - if (collectionExists(collection)) { - Map fieldMap = new HashMap<>(); - try { - CollectionResponse collectionResponse = tsClient.collections(collection).retrieve(); - List fields = collectionResponse.getFields(); - for (Field field : fields) { - String fieldName = field.getName(); - String type = field.getType(); - fieldMap.put(fieldName, type); - } - } catch (Exception e) { - log.error(FIELD_TYPE_MAPPING_ERROR.getDescription()); - throw new TypesenseConnectorException( - FIELD_TYPE_MAPPING_ERROR, FIELD_TYPE_MAPPING_ERROR.getDescription()); - } - return fieldMap; - } else { - return null; - } - } - - public Map> getFieldTypeMapping(String collection) { - Map> allTypesenseSearchFieldTypeInfoMap = - new HashMap<>(); - try { - CollectionResponse collectionResponse = tsClient.collections(collection).retrieve(); - List fields = collectionResponse.getFields(); - for (Field field : fields) { - String fieldName = field.getName(); - String type = field.getType(); - BasicTypeDefine.BasicTypeDefineBuilder typeDefine = - BasicTypeDefine.builder() - .name(fieldName) - .columnType(type) - .dataType(type) - .nativeType(new TypesenseType(type, new HashMap<>())); - allTypesenseSearchFieldTypeInfoMap.put(fieldName, typeDefine.build()); - } - } catch (Exception e) { - log.error(FIELD_TYPE_MAPPING_ERROR.getDescription()); - throw new TypesenseConnectorException( - FIELD_TYPE_MAPPING_ERROR, FIELD_TYPE_MAPPING_ERROR.getDescription()); - } - return allTypesenseSearchFieldTypeInfoMap; - } - - public boolean createCollection(String collection) { - if (collectionExists(collection)) { - return true; - } - List fields = new ArrayList<>(); - fields.add(new Field().name(".*").type(FieldTypes.AUTO)); - return createCollection(collection, fields); - } - - public boolean createCollection(String collection, List fields) { - CollectionSchema collectionSchema = new CollectionSchema(); - collectionSchema.name(collection).fields(fields).enableNestedFields(true); - try { - tsClient.collections().create(collectionSchema); - return true; - } catch (Exception e) { - log.error(CREATE_COLLECTION_ERROR.getDescription()); - throw new TypesenseConnectorException( - CREATE_COLLECTION_ERROR, CREATE_COLLECTION_ERROR.getDescription()); - } - } - - public boolean dropCollection(String collection) { - try { - tsClient.collections(collection).delete(); - return true; - } catch (Exception e) { - log.error(DROP_COLLECTION_ERROR.getDescription()); - throw new TypesenseConnectorException( - DROP_COLLECTION_ERROR, DROP_COLLECTION_ERROR.getDescription()); - } - } - - public boolean truncateCollectionData(String collection) { - DeleteDocumentsParameters deleteDocumentsParameters = new DeleteDocumentsParameters(); - deleteDocumentsParameters.filterBy("id:!=1||id:=1"); - try { - tsClient.collections(collection).documents().delete(deleteDocumentsParameters); - } catch (Exception e) { - log.error(TRUNCATE_COLLECTION_ERROR.getDescription()); - throw new TypesenseConnectorException( - TRUNCATE_COLLECTION_ERROR, TRUNCATE_COLLECTION_ERROR.getDescription()); - } - return true; - } - - public boolean deleteCollectionData(String collection, String id) { - try { - tsClient.collections(collection).documents(id).delete(); - } catch (Exception e) { - log.error(DELETE_COLLECTION_ERROR.getDescription()); - throw new TypesenseConnectorException( - DELETE_COLLECTION_ERROR, DELETE_COLLECTION_ERROR.getDescription()); - } - return true; - } - - public long collectionDocNum(String collection) { - SearchParameters q = new SearchParameters().q("*"); - try { - SearchResult searchResult = tsClient.collections(collection).documents().search(q); - return searchResult.getFound(); - } catch (Exception e) { - log.error(QUERY_COLLECTION_NUM_ERROR.getDescription()); - throw new TypesenseConnectorException( - QUERY_COLLECTION_NUM_ERROR, QUERY_COLLECTION_NUM_ERROR.getDescription()); - } - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java deleted file mode 100644 index 2ac05a6d312..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/client/TypesenseType.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.client; - -import lombok.AllArgsConstructor; -import lombok.Getter; - -import java.util.Map; - -@Getter -@AllArgsConstructor -public class TypesenseType { - - public static final String STRING = "string"; - public static final String INT32 = "int32"; - public static final String INT64 = "int64"; - public static final String FLOAT = "float"; - public static final String BOOL = "bool"; - public static final String IMAGE = "image"; - public static final String OBJET = "object"; - private String type; - private Map options; -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java deleted file mode 100644 index 9912c6aa2e1..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SinkConfig.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.config; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; -import org.apache.seatunnel.api.sink.DataSaveMode; -import org.apache.seatunnel.api.sink.SchemaSaveMode; - -import java.util.Arrays; -import java.util.List; - -import static org.apache.seatunnel.api.sink.DataSaveMode.APPEND_DATA; -import static org.apache.seatunnel.api.sink.DataSaveMode.DROP_DATA; -import static org.apache.seatunnel.api.sink.DataSaveMode.ERROR_WHEN_DATA_EXISTS; - -public class SinkConfig { - - public static final Option COLLECTION = - Options.key("collection") - .stringType() - .noDefaultValue() - .withDescription("Typesense collection name"); - - public static final Option> PRIMARY_KEYS = - Options.key("primary_keys") - .listType(String.class) - .noDefaultValue() - .withDescription("Primary key fields used to generate the document `id`"); - - public static final Option KEY_DELIMITER = - Options.key("key_delimiter") - .stringType() - .defaultValue("_") - .withDescription( - "Delimiter for composite keys (\"_\" by default), e.g., \"$\" would result in document `id` \"KEY1$KEY2$KEY3\"."); - - public static final Option MAX_BATCH_SIZE = - Options.key("max_batch_size") - .intType() - .defaultValue(10) - .withDescription("batch bulk doc max size"); - - public static final Option MAX_RETRY_COUNT = - Options.key("max_retry_count") - .intType() - .defaultValue(3) - .withDescription("one bulk request max try count"); - - public static final Option SCHEMA_SAVE_MODE = - Options.key("schema_save_mode") - .enumType(SchemaSaveMode.class) - .defaultValue(SchemaSaveMode.CREATE_SCHEMA_WHEN_NOT_EXIST) - .withDescription("schema_save_mode"); - - public static final Option DATA_SAVE_MODE = - Options.key("data_save_mode") - .singleChoice( - DataSaveMode.class, - Arrays.asList(DROP_DATA, APPEND_DATA, ERROR_WHEN_DATA_EXISTS)) - .defaultValue(APPEND_DATA) - .withDescription("data_save_mode"); -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java deleted file mode 100644 index a1642331a6c..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/SourceConfig.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.config; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; - -public class SourceConfig { - - public static final Option COLLECTION = - Options.key("collection") - .stringType() - .noDefaultValue() - .withDescription("Typesense collection name"); - - public static final Option QUERY = - Options.key("query") - .stringType() - .noDefaultValue() - .withDescription("Typesense query param"); - - public static final Option QUERY_BATCH_SIZE = - Options.key("batch_size") - .intType() - .defaultValue(100) - .withDescription("Typesense query batch size"); -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java deleted file mode 100644 index 2f36dc044c5..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/config/TypesenseConnectionConfig.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.config; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; - -import java.util.List; - -public class TypesenseConnectionConfig { - - public static final Option> HOSTS = - Options.key("hosts") - .listType() - .noDefaultValue() - .withDescription( - "Typesense cluster http address, the format is host:port, allowing multiple hosts to be specified. Such as [\"host1:8018\", \"host2:8018\"]"); - - public static final Option APIKEY = - Options.key("api_key") - .stringType() - .noDefaultValue() - .withDescription("Typesense api key"); - - public static final Option protocol = - Options.key("protocol") - .stringType() - .defaultValue("http") - .withDescription("Default is http , for Typesense Cloud use https"); -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java deleted file mode 100644 index 9fd4892979f..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/CollectionInfo.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.dto; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; - -import lombok.Data; - -@Data -public class CollectionInfo { - - private String collection; - private String type; - private String[] primaryKeys; - private String keyDelimiter; - - public CollectionInfo(String collection, ReadonlyConfig config) { - this.collection = collection; - if (config.getOptional(SinkConfig.PRIMARY_KEYS).isPresent()) { - primaryKeys = config.get(SinkConfig.PRIMARY_KEYS).toArray(new String[0]); - } - keyDelimiter = config.get(SinkConfig.KEY_DELIMITER); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java deleted file mode 100644 index 9e1b55f8cbd..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/dto/SourceCollectionInfo.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.dto; - -import lombok.AllArgsConstructor; -import lombok.Data; - -import java.io.Serializable; - -@Data -@AllArgsConstructor -public class SourceCollectionInfo implements Serializable { - private String collection; - private String query; - private long found; - private int offset; - private int queryBatchSize; -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java deleted file mode 100644 index 8edcf608d55..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorErrorCode.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.exception; - -import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; - -public enum TypesenseConnectorErrorCode implements SeaTunnelErrorCode { - QUERY_PARAM_ERROR("TYPESENSE-01", "Query parameter error"), - QUERY_COLLECTION_EXISTS_ERROR("TYPESENSE-02", "Whether the collection stores query exceptions"), - QUERY_COLLECTION_LIST_ERROR("TYPESENSE-03", "Collection list acquisition exception"), - FIELD_TYPE_MAPPING_ERROR("TYPESENSE-04", "Failed to obtain the field"), - CREATE_COLLECTION_ERROR("TYPESENSE-05", "Create collection failed"), - DROP_COLLECTION_ERROR("TYPESENSE-06", "Drop collection failed"), - TRUNCATE_COLLECTION_ERROR("TYPESENSE-07", "Truncate collection failed"), - QUERY_COLLECTION_NUM_ERROR("TYPESENSE-08", "Query collection doc number failed"), - INSERT_DOC_ERROR("TYPESENSE-09", "Insert documents failed"), - DELETE_COLLECTION_ERROR("TYPESENSE-10", "Truncate collection failed"); - private final String code; - private final String description; - - TypesenseConnectorErrorCode(String code, String description) { - this.code = code; - this.description = description; - } - - @Override - public String getCode() { - return code; - } - - @Override - public String getDescription() { - return description; - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java deleted file mode 100644 index 3c1cb1cc167..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/exception/TypesenseConnectorException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.exception; - -import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; -import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; - -public class TypesenseConnectorException extends SeaTunnelRuntimeException { - public TypesenseConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { - super(seaTunnelErrorCode, errorMessage); - } - - public TypesenseConnectorException( - SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { - super(seaTunnelErrorCode, errorMessage, cause); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java deleted file mode 100644 index 5dc56f0bb77..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/KeyExtractor.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize; - -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; - -import lombok.AllArgsConstructor; - -import java.io.Serializable; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.ArrayList; -import java.util.List; -import java.util.function.Function; - -@AllArgsConstructor -public class KeyExtractor implements Function, Serializable { - private final FieldFormatter[] fieldFormatters; - private final String keyDelimiter; - - @Override - public String apply(SeaTunnelRow row) { - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < fieldFormatters.length; i++) { - if (i > 0) { - builder.append(keyDelimiter); - } - String value = fieldFormatters[i].format(row); - builder.append(value); - } - return builder.toString(); - } - - public static Function createKeyExtractor( - SeaTunnelRowType rowType, String[] primaryKeys, String keyDelimiter) { - if (primaryKeys == null) { - return row -> null; - } - - List fieldFormatters = new ArrayList<>(primaryKeys.length); - for (String fieldName : primaryKeys) { - int fieldIndex = rowType.indexOf(fieldName); - SeaTunnelDataType fieldType = rowType.getFieldType(fieldIndex); - FieldFormatter fieldFormatter = createFieldFormatter(fieldIndex, fieldType); - fieldFormatters.add(fieldFormatter); - } - return new KeyExtractor(fieldFormatters.toArray(new FieldFormatter[0]), keyDelimiter); - } - - private static FieldFormatter createFieldFormatter( - int fieldIndex, SeaTunnelDataType fieldType) { - return row -> { - switch (fieldType.getSqlType()) { - case ROW: - case ARRAY: - case MAP: - throw new TypesenseConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "Unsupported type: " + fieldType); - case DATE: - LocalDate localDate = (LocalDate) row.getField(fieldIndex); - return localDate.toString(); - case TIME: - LocalTime localTime = (LocalTime) row.getField(fieldIndex); - return localTime.toString(); - case TIMESTAMP: - LocalDateTime localDateTime = (LocalDateTime) row.getField(fieldIndex); - return localDateTime.toString(); - default: - return row.getField(fieldIndex).toString(); - } - }; - } - - private interface FieldFormatter extends Serializable { - String format(SeaTunnelRow row); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java deleted file mode 100644 index 94da50cf09b..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/SeaTunnelRowSerializer.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -public interface SeaTunnelRowSerializer { - String serializeRow(SeaTunnelRow row); - - String serializeRowForDelete(SeaTunnelRow row); -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java deleted file mode 100644 index 07faca4cc92..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/TypesenseRowSerializer.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonError; -import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.KeyExtractor; - -import org.apache.commons.lang3.StringUtils; - -import java.time.temporal.Temporal; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.function.Function; - -public class TypesenseRowSerializer implements SeaTunnelRowSerializer { - - private final SeaTunnelRowType seaTunnelRowType; - - private final ObjectMapper objectMapper = new ObjectMapper(); - - private final Function keyExtractor; - - public TypesenseRowSerializer( - CollectionInfo collectionInfo, SeaTunnelRowType seaTunnelRowType) { - this.seaTunnelRowType = seaTunnelRowType; - this.keyExtractor = - KeyExtractor.createKeyExtractor( - seaTunnelRowType, - collectionInfo.getPrimaryKeys(), - collectionInfo.getKeyDelimiter()); - } - - @Override - public String serializeRow(SeaTunnelRow row) { - String key = keyExtractor.apply(row); - Map document = toDocumentMap(row, seaTunnelRowType); - if (StringUtils.isNotBlank(key)) { - document.put("id", key); - } - String documentStr; - try { - documentStr = objectMapper.writeValueAsString(document); - } catch (JsonProcessingException e) { - throw CommonError.jsonOperationError("Typesense", "document:" + document.toString(), e); - } - return documentStr; - } - - @Override - public String serializeRowForDelete(SeaTunnelRow row) { - String key = keyExtractor.apply(row); - Map document = toDocumentMap(row, seaTunnelRowType); - String id = document.get("id").toString(); - if (StringUtils.isNotBlank(key)) { - id = key; - } - return id; - } - - private Map toDocumentMap(SeaTunnelRow row, SeaTunnelRowType rowType) { - String[] fieldNames = rowType.getFieldNames(); - Map doc = new HashMap<>(fieldNames.length); - Object[] fields = row.getFields(); - for (int i = 0; i < fieldNames.length; i++) { - Object value = fields[i]; - if (value == null) { - } else if (value instanceof SeaTunnelRow) { - doc.put( - fieldNames[i], - toDocumentMap( - (SeaTunnelRow) value, (SeaTunnelRowType) rowType.getFieldType(i))); - } else { - doc.put(fieldNames[i], convertValue(value)); - } - } - return doc; - } - - private Object convertValue(Object value) { - if (value instanceof Temporal) { - // jackson not support jdk8 new time api - return value.toString(); - } else if (value instanceof Map) { - for (Map.Entry entry : ((Map) value).entrySet()) { - ((Map) value).put(entry.getKey(), convertValue(entry.getValue())); - } - return value; - } else if (value instanceof List) { - for (int i = 0; i < ((List) value).size(); i++) { - ((List) value).set(i, convertValue(((List) value).get(i))); - } - return value; - } else { - return value; - } - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java deleted file mode 100644 index d13901b7081..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/CollectionSerializer.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.collection; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -public interface CollectionSerializer { - String serialize(SeaTunnelRow row); -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java deleted file mode 100644 index a2b0da248e1..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/sink/collection/FixedValueCollectionSerializer.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.collection; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -public class FixedValueCollectionSerializer implements CollectionSerializer { - - private final String index; - - public FixedValueCollectionSerializer(String index) { - this.index = index; - } - - @Override - public String serialize(SeaTunnelRow row) { - return index; - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java deleted file mode 100644 index 762506d4980..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/DefaultSeaTunnelRowDeserializer.java +++ /dev/null @@ -1,253 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; - -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.DecimalType; -import org.apache.seatunnel.api.table.type.LocalTimeType; -import org.apache.seatunnel.api.table.type.MapType; -import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; - -import java.lang.reflect.Array; -import java.math.BigDecimal; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.format.DateTimeFormatter; -import java.util.Base64; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.seatunnel.api.table.type.BasicType.BOOLEAN_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.BYTE_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.DOUBLE_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.LONG_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.SHORT_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; -import static org.apache.seatunnel.api.table.type.BasicType.VOID_TYPE; - -public class DefaultSeaTunnelRowDeserializer implements SeaTunnelRowDeserializer { - - private final SeaTunnelRowType rowTypeInfo; - - private final ObjectMapper mapper = new ObjectMapper(); - - private final String nullDefault = "null"; - - private final Map dateTimeFormatterMap = - new HashMap() { - { - put("yyyy-MM-dd HH".length(), DateTimeFormatter.ofPattern("yyyy-MM-dd HH")); - put( - "yyyy-MM-dd HH:mm".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm")); - put( - "yyyyMMdd HH:mm:ss".length(), - DateTimeFormatter.ofPattern("yyyyMMdd HH:mm:ss")); - put( - "yyyy-MM-dd HH:mm:ss".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss")); - put( - "yyyy-MM-dd HH:mm:ss.S".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.S")); - put( - "yyyy-MM-dd HH:mm:ss.SS".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SS")); - put( - "yyyy-MM-dd HH:mm:ss.SSS".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS")); - put( - "yyyy-MM-dd HH:mm:ss.SSSS".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSS")); - put( - "yyyy-MM-dd HH:mm:ss.SSSSSS".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSS")); - put( - "yyyy-MM-dd HH:mm:ss.SSSSSSSSS".length(), - DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSSSSS")); - } - }; - - public DefaultSeaTunnelRowDeserializer(SeaTunnelRowType rowTypeInfo) { - this.rowTypeInfo = rowTypeInfo; - } - - @Override - public SeaTunnelRow deserialize(TypesenseRecord rowRecord) { - return convert(rowRecord); - } - - SeaTunnelRow convert(TypesenseRecord rowRecord) { - Object[] seaTunnelFields = new Object[rowTypeInfo.getTotalFields()]; - String fieldName = null; - Object value = null; - SeaTunnelDataType seaTunnelDataType = null; - Map doc = rowRecord.getDoc(); - try { - for (int i = 0; i < rowTypeInfo.getTotalFields(); i++) { - fieldName = rowTypeInfo.getFieldName(i); - value = doc.get(fieldName); // 字段值 - if (value != null) { - seaTunnelDataType = - rowTypeInfo.getFieldType(i); // seaTunnelDataType 为SeaTunnel类型 - seaTunnelFields[i] = convertValue(seaTunnelDataType, value); - } - } - } catch (Exception ex) { - throw new TypesenseConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - String.format( - "error fieldName=%s,fieldValue=%s,seaTunnelDataType=%s,rowRecord=%s", - fieldName, value, seaTunnelDataType, JsonUtils.toJsonString(rowRecord)), - ex); - } - return new SeaTunnelRow(seaTunnelFields); - } - - Object convertValue(SeaTunnelDataType fieldType, Object fieldValue) - throws JsonProcessingException { - if (STRING_TYPE.equals(fieldType)) { - return fieldValue.toString(); - } else { - if (nullDefault.equals(fieldValue.toString())) { - return null; - } - if (BOOLEAN_TYPE.equals(fieldType)) { - return Boolean.parseBoolean(fieldValue.toString()); - } else if (BYTE_TYPE.equals(fieldType)) { - return Byte.valueOf(fieldValue.toString()); - } else if (SHORT_TYPE.equals(fieldType)) { - return Short.parseShort(fieldValue.toString()); - } else if (INT_TYPE.equals(fieldType)) { - return Integer.parseInt(fieldValue.toString()); - } else if (LONG_TYPE.equals(fieldType)) { - return Long.parseLong(fieldValue.toString()); - } else if (FLOAT_TYPE.equals(fieldType)) { - return Float.parseFloat(fieldValue.toString()); - } else if (DOUBLE_TYPE.equals(fieldType)) { - return Double.parseDouble(fieldValue.toString()); - } else if (LocalTimeType.LOCAL_DATE_TYPE.equals(fieldType)) { - LocalDateTime localDateTime = parseDate(fieldValue.toString()); - return localDateTime.toLocalDate(); - } else if (LocalTimeType.LOCAL_TIME_TYPE.equals(fieldType)) { - LocalDateTime localDateTime = parseDate(fieldValue.toString()); - return localDateTime.toLocalTime(); - } else if (LocalTimeType.LOCAL_DATE_TIME_TYPE.equals(fieldType)) { - return parseDate(fieldValue.toString()); - } else if (fieldType instanceof DecimalType) { - return new BigDecimal(fieldValue.toString()); - } else if (fieldType instanceof ArrayType) { - ArrayType arrayType = (ArrayType) fieldType; - SeaTunnelDataType elementType = arrayType.getElementType(); - List stringList = (List) fieldValue; - Object arr = Array.newInstance(elementType.getTypeClass(), stringList.size()); - for (int i = 0; i < stringList.size(); i++) { - Object convertValue = convertValue(elementType, stringList.get(i)); - Array.set(arr, i, convertValue); - } - return arr; - } else if (fieldType instanceof MapType) { - MapType mapType = (MapType) fieldType; - SeaTunnelDataType keyType = mapType.getKeyType(); - - SeaTunnelDataType valueType = mapType.getValueType(); - Map stringMap = - mapper.readValue( - fieldValue.toString(), - new TypeReference>() {}); - Map convertMap = new HashMap(); - for (Map.Entry entry : stringMap.entrySet()) { - Object convertKey = convertValue(keyType, entry.getKey()); - Object convertValue = convertValue(valueType, entry.getValue()); - convertMap.put(convertKey, convertValue); - } - return convertMap; - } else if (fieldType instanceof SeaTunnelRowType) { - SeaTunnelRowType rowType = (SeaTunnelRowType) fieldType; - Map collect = (Map) fieldValue; - Object[] seaTunnelFields = new Object[rowType.getTotalFields()]; - for (int i = 0; i < rowType.getTotalFields(); i++) { - String fieldName = rowType.getFieldName(i); - SeaTunnelDataType fieldDataType = rowType.getFieldType(i); - Object value = collect.get(fieldName); - if (value != null) { - seaTunnelFields[i] = convertValue(fieldDataType, value); - } - } - return new SeaTunnelRow(seaTunnelFields); - } else if (fieldType instanceof PrimitiveByteArrayType) { - return Base64.getDecoder().decode(fieldValue.toString()); - } else if (VOID_TYPE.equals(fieldType) || fieldType == null) { - return null; - } else { - throw new TypesenseConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_DATA_TYPE, - "Unexpected value: " + fieldType); - } - } - } - - private LocalDateTime parseDate(String fieldValue) { - // handle strings of timestamp type - try { - long ts = Long.parseLong(fieldValue); - return LocalDateTime.ofInstant(Instant.ofEpochMilli(ts), ZoneId.systemDefault()); - } catch (NumberFormatException e) { - // no op - } - String formatDate = fieldValue.replace("T", " ").replace("Z", ""); - if (fieldValue.length() == "yyyyMMdd".length() - || fieldValue.length() == "yyyy-MM-dd".length()) { - formatDate = fieldValue + " 00:00:00"; - } - DateTimeFormatter dateTimeFormatter = dateTimeFormatterMap.get(formatDate.length()); - if (dateTimeFormatter == null) { - throw new TypesenseConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, "unsupported date format"); - } - return LocalDateTime.parse(formatDate, dateTimeFormatter); - } - - Object recursiveGet(Map collect, String keyWithRecursive) { - Object value = null; - boolean isFirst = true; - for (String key : keyWithRecursive.split("\\.")) { - if (isFirst) { - value = collect.get(key); - isFirst = false; - } else if (value instanceof ObjectNode) { - value = ((ObjectNode) value).get(key); - } - } - return value; - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java deleted file mode 100644 index a5864e596bc..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/SeaTunnelRowDeserializer.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -public interface SeaTunnelRowDeserializer { - - SeaTunnelRow deserialize(TypesenseRecord rowRecord); -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java deleted file mode 100644 index 154b94ddc65..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/serialize/source/TypesenseRecord.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source; - -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.ToString; - -import java.util.Map; - -@Getter -@ToString -@AllArgsConstructor -public class TypesenseRecord { - private Map doc; -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java deleted file mode 100644 index e52638f83ef..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSink.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.sink; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.sink.DataSaveMode; -import org.apache.seatunnel.api.sink.DefaultSaveModeHandler; -import org.apache.seatunnel.api.sink.SaveModeHandler; -import org.apache.seatunnel.api.sink.SchemaSaveMode; -import org.apache.seatunnel.api.sink.SeaTunnelSink; -import org.apache.seatunnel.api.sink.SinkWriter; -import org.apache.seatunnel.api.sink.SupportMultiTableSink; -import org.apache.seatunnel.api.sink.SupportSaveMode; -import org.apache.seatunnel.api.table.catalog.Catalog; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.factory.CatalogFactory; -import org.apache.seatunnel.api.table.factory.FactoryUtil; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; -import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseAggregatedCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseSinkState; - -import java.util.Optional; - -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.MAX_BATCH_SIZE; -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.MAX_RETRY_COUNT; - -public class TypesenseSink - implements SeaTunnelSink< - SeaTunnelRow, - TypesenseSinkState, - TypesenseCommitInfo, - TypesenseAggregatedCommitInfo>, - SupportMultiTableSink, - SupportSaveMode { - - private ReadonlyConfig config; - private CatalogTable catalogTable; - private final int maxBatchSize; - private final int maxRetryCount; - - public TypesenseSink(ReadonlyConfig config, CatalogTable catalogTable) { - this.config = config; - this.catalogTable = catalogTable; - maxBatchSize = config.get(MAX_BATCH_SIZE); - maxRetryCount = config.get(MAX_RETRY_COUNT); - } - - @Override - public String getPluginName() { - return "Typesense"; - } - - @Override - public TypesenseSinkWriter createWriter(SinkWriter.Context context) { - return new TypesenseSinkWriter(context, catalogTable, config, maxBatchSize, maxRetryCount); - } - - @Override - public Optional getSaveModeHandler() { - CatalogFactory catalogFactory = - FactoryUtil.discoverFactory( - Thread.currentThread().getContextClassLoader(), - CatalogFactory.class, - getPluginName()); - if (catalogFactory == null) { - return Optional.empty(); - } - Catalog catalog = catalogFactory.createCatalog(catalogFactory.factoryIdentifier(), config); - SchemaSaveMode schemaSaveMode = config.get(SinkConfig.SCHEMA_SAVE_MODE); - DataSaveMode dataSaveMode = config.get(SinkConfig.DATA_SAVE_MODE); - - TablePath tablePath = TablePath.of("", catalogTable.getTableId().getTableName()); - catalog.open(); - return Optional.of( - new DefaultSaveModeHandler( - schemaSaveMode, dataSaveMode, catalog, tablePath, null, null)); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java deleted file mode 100644 index 4c937609406..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkFactory.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.sink; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; -import org.apache.seatunnel.api.table.connector.TableSink; -import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableSinkFactory; -import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; -import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; - -import com.google.auto.service.AutoService; - -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.COLLECTION; -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.KEY_DELIMITER; -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig.PRIMARY_KEYS; -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.APIKEY; -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.HOSTS; - -@AutoService(Factory.class) -public class TypesenseSinkFactory implements TableSinkFactory { - - @Override - public String factoryIdentifier() { - return "Typesense"; - } - - @Override - public OptionRule optionRule() { - return OptionRule.builder() - .required( - HOSTS, - COLLECTION, - APIKEY, - SinkConfig.SCHEMA_SAVE_MODE, - SinkConfig.DATA_SAVE_MODE) - .optional(PRIMARY_KEYS, KEY_DELIMITER) - .build(); - } - - @Override - public TableSink createSink(TableSinkFactoryContext context) { - ReadonlyConfig readonlyConfig = context.getOptions(); - String original = readonlyConfig.get(COLLECTION); - CatalogTable newTable = - CatalogTable.of( - TableIdentifier.of( - context.getCatalogTable().getCatalogName(), - context.getCatalogTable().getTablePath().getDatabaseName(), - original), - context.getCatalogTable()); - return () -> new TypesenseSink(readonlyConfig, newTable); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java deleted file mode 100644 index 9fdaf68a242..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseSinkWriter.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.sink; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.sink.SinkWriter; -import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.type.RowKind; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.common.utils.RetryUtils; -import org.apache.seatunnel.common.utils.RetryUtils.RetryMaterial; -import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; -import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; -import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.SeaTunnelRowSerializer; -import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.TypesenseRowSerializer; -import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.state.TypesenseSinkState; - -import lombok.extern.slf4j.Slf4j; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; - -import static org.apache.seatunnel.api.table.type.RowKind.INSERT; -import static org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode.INSERT_DOC_ERROR; - -@Slf4j -public class TypesenseSinkWriter - implements SinkWriter, - SupportMultiTableSinkWriter { - - private final Context context; - private final int maxBatchSize; - private final SeaTunnelRowSerializer seaTunnelRowSerializer; - - private final List requestEsList; - - private final String collection; - private TypesenseClient typesenseClient; - private RetryMaterial retryMaterial; - private static final long DEFAULT_SLEEP_TIME_MS = 200L; - - public TypesenseSinkWriter( - Context context, - CatalogTable catalogTable, - ReadonlyConfig config, - int maxBatchSize, - int maxRetryCount) { - this.context = context; - this.maxBatchSize = maxBatchSize; - - collection = catalogTable.getTableId().getTableName(); - CollectionInfo collectionInfo = - new CollectionInfo(catalogTable.getTableId().getTableName(), config); - typesenseClient = TypesenseClient.createInstance(config); - this.seaTunnelRowSerializer = - new TypesenseRowSerializer(collectionInfo, catalogTable.getSeaTunnelRowType()); - - this.requestEsList = new ArrayList<>(maxBatchSize); - this.retryMaterial = - new RetryMaterial(maxRetryCount, true, exception -> true, DEFAULT_SLEEP_TIME_MS); - } - - @Override - public void write(SeaTunnelRow element) { - if (RowKind.UPDATE_BEFORE.equals(element.getRowKind())) { - return; - } - - switch (element.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - String indexRequestRow = seaTunnelRowSerializer.serializeRow(element); - requestEsList.add(indexRequestRow); - if (requestEsList.size() >= maxBatchSize) { - insert(collection, requestEsList); - } - break; - case UPDATE_BEFORE: - case DELETE: - String id = seaTunnelRowSerializer.serializeRowForDelete(element); - typesenseClient.deleteCollectionData(collection, id); - break; - default: - throw new TypesenseConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "Unsupported write row kind: " + element.getRowKind()); - } - } - - @Override - public Optional prepareCommit() { - insert(this.collection, this.requestEsList); - return Optional.empty(); - } - - private void insert(String collection, List requestEsList) { - try { - RetryUtils.retryWithException( - () -> { - typesenseClient.insert(collection, requestEsList); - return null; - }, - retryMaterial); - requestEsList.clear(); - } catch (Exception e) { - log.error(INSERT_DOC_ERROR.getDescription()); - throw new TypesenseConnectorException( - INSERT_DOC_ERROR, INSERT_DOC_ERROR.getDescription()); - } - } - - @Override - public void abortPrepare() {} - - @Override - public void close() { - insert(collection, requestEsList); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java deleted file mode 100644 index a3860a0495a..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSource.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.source; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.source.Boundedness; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.source.SupportColumnProjection; -import org.apache.seatunnel.api.source.SupportParallelism; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; -import org.apache.seatunnel.api.table.catalog.schema.TableSchemaOptions; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -import lombok.extern.slf4j.Slf4j; - -import java.util.Collections; -import java.util.List; - -@Slf4j -public class TypesenseSource - implements SeaTunnelSource, - SupportParallelism, - SupportColumnProjection { - - private final ReadonlyConfig config; - - private CatalogTable catalogTable; - - public TypesenseSource(ReadonlyConfig config) { - this.config = config; - if (config.getOptional(TableSchemaOptions.SCHEMA).isPresent()) { - catalogTable = CatalogTableUtil.buildWithConfig(config); - } - } - - @Override - public String getPluginName() { - return "Typesense"; - } - - @Override - public Boundedness getBoundedness() { - return Boundedness.BOUNDED; - } - - @Override - public List getProducedCatalogTables() { - return Collections.singletonList(catalogTable); - } - - @Override - public SourceReader createReader( - SourceReader.Context readerContext) throws Exception { - return new TypesenseSourceReader(readerContext, config, catalogTable.getSeaTunnelRowType()); - } - - @Override - public SourceSplitEnumerator createEnumerator( - SourceSplitEnumerator.Context enumeratorContext) { - return new TypesenseSourceSplitEnumerator(enumeratorContext, config); - } - - @Override - public SourceSplitEnumerator restoreEnumerator( - SourceSplitEnumerator.Context enumeratorContext, - TypesenseSourceState checkpointState) { - return new TypesenseSourceSplitEnumerator(enumeratorContext, config); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java deleted file mode 100644 index 254763ebbb1..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceFactory.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.source; - -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.table.connector.TableSource; -import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableSourceFactory; -import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; - -import com.google.auto.service.AutoService; - -import java.io.Serializable; - -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig.COLLECTION; -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.APIKEY; -import static org.apache.seatunnel.connectors.seatunnel.typesense.config.TypesenseConnectionConfig.HOSTS; - -@AutoService(Factory.class) -public class TypesenseSourceFactory implements TableSourceFactory { - - @Override - public String factoryIdentifier() { - return "Typesense"; - } - - @Override - public OptionRule optionRule() { - return OptionRule.builder().required(HOSTS, APIKEY).optional(COLLECTION).build(); - } - - @Override - public - TableSource createSource(TableSourceFactoryContext context) { - return () -> (SeaTunnelSource) new TypesenseSource(context.getOptions()); - } - - @Override - public Class getSourceClass() { - return TypesenseSource.class; - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java deleted file mode 100644 index 16946519968..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceReader.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.source; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; -import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.DefaultSeaTunnelRowDeserializer; -import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.SeaTunnelRowDeserializer; -import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.source.TypesenseRecord; - -import org.typesense.model.SearchResult; -import org.typesense.model.SearchResultHit; - -import lombok.extern.slf4j.Slf4j; - -import java.util.ArrayList; -import java.util.Deque; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -@Slf4j -public class TypesenseSourceReader implements SourceReader { - - SourceReader.Context context; - - private final ReadonlyConfig config; - - private final SeaTunnelRowDeserializer deserializer; - - private TypesenseClient typesenseClient; - - Deque splits = new LinkedList<>(); - - boolean noMoreSplit; - - private final long pollNextWaitTime = 1000L; - - public TypesenseSourceReader( - SourceReader.Context context, ReadonlyConfig config, SeaTunnelRowType rowTypeInfo) { - this.context = context; - this.config = config; - this.deserializer = new DefaultSeaTunnelRowDeserializer(rowTypeInfo); - } - - @Override - public void open() { - typesenseClient = TypesenseClient.createInstance(this.config); - } - - @Override - public void close() { - // Nothing , because typesense does not require - } - - @Override - public List snapshotState(long checkpointId) throws Exception { - return new ArrayList<>(splits); - } - - @Override - public void addSplits(List splits) { - this.splits.addAll(splits); - } - - @Override - public void handleNoMoreSplits() { - noMoreSplit = true; - } - - @Override - public void pollNext(Collector output) throws Exception { - synchronized (output.getCheckpointLock()) { - TypesenseSourceSplit split = splits.poll(); - if (split != null) { - SourceCollectionInfo sourceCollectionInfo = split.getSourceCollectionInfo(); - int pageSize = sourceCollectionInfo.getQueryBatchSize(); - while (true) { - SearchResult searchResult = - typesenseClient.search( - sourceCollectionInfo.getCollection(), - sourceCollectionInfo.getQuery(), - sourceCollectionInfo.getOffset(), - sourceCollectionInfo.getQueryBatchSize()); - Integer found = searchResult.getFound(); - List hits = searchResult.getHits(); - for (SearchResultHit hit : hits) { - Map document = hit.getDocument(); - SeaTunnelRow seaTunnelRow = - deserializer.deserialize(new TypesenseRecord(document)); - output.collect(seaTunnelRow); - } - if ((double) found / pageSize - 1 - > sourceCollectionInfo.getOffset() / pageSize) { - sourceCollectionInfo.setOffset(sourceCollectionInfo.getOffset() + pageSize); - } else { - break; - } - } - - } else if (noMoreSplit) { - log.info("Closed the bounded Typesense source"); - context.signalNoMoreElement(); - } else { - Thread.sleep(pollNextWaitTime); - } - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception {} -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java deleted file mode 100644 index 39cf8530eec..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplit.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.source; - -import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; - -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.ToString; - -@ToString -@AllArgsConstructor -public class TypesenseSourceSplit implements SourceSplit { - - private static final long serialVersionUID = -1L; - - private String splitId; - - @Getter private SourceCollectionInfo sourceCollectionInfo; - - @Override - public String splitId() { - return splitId; - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java deleted file mode 100644 index adb149ae4f7..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/source/TypesenseSourceSplitEnumerator.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.source; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; -import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; -import org.apache.seatunnel.connectors.seatunnel.typesense.config.SourceConfig; -import org.apache.seatunnel.connectors.seatunnel.typesense.dto.SourceCollectionInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; - -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -@Slf4j -public class TypesenseSourceSplitEnumerator - implements SourceSplitEnumerator { - - private final SourceSplitEnumerator.Context context; - - private final ReadonlyConfig config; - - private TypesenseClient typesenseClient; - - private final Object stateLock = new Object(); - - private Map> pendingSplit; - - private volatile boolean shouldEnumerate; - - public TypesenseSourceSplitEnumerator( - SourceSplitEnumerator.Context context, ReadonlyConfig config) { - this(context, null, config); - } - - public TypesenseSourceSplitEnumerator( - SourceSplitEnumerator.Context context, - TypesenseSourceState sourceState, - ReadonlyConfig config) { - this.context = context; - this.config = config; - this.pendingSplit = new HashMap<>(); - this.shouldEnumerate = sourceState == null; - if (sourceState != null) { - this.shouldEnumerate = sourceState.isShouldEnumerate(); - this.pendingSplit.putAll(sourceState.getPendingSplit()); - } - } - - @Override - public void open() { - // Nothing - } - - @Override - public void run() throws Exception { - Set readers = context.registeredReaders(); - if (shouldEnumerate) { - List newSplits = getTypesenseSplit(); - - synchronized (stateLock) { - addPendingSplit(newSplits); - shouldEnumerate = false; - } - - assignSplit(readers); - } - - log.debug( - "No more splits to assign." + " Sending NoMoreSplitsEvent to reader {}.", readers); - readers.forEach(context::signalNoMoreSplits); - } - - private void addPendingSplit(Collection splits) { - int readerCount = context.currentParallelism(); - for (TypesenseSourceSplit split : splits) { - int ownerReader = getSplitOwner(split.splitId(), readerCount); - log.info("Assigning {} to {} reader.", split, ownerReader); - pendingSplit.computeIfAbsent(ownerReader, r -> new ArrayList<>()).add(split); - } - } - - private void assignSplit(Collection readers) { - log.debug("Assign pendingSplits to readers {}", readers); - - for (int reader : readers) { - List assignmentForReader = pendingSplit.remove(reader); - if (assignmentForReader != null && !assignmentForReader.isEmpty()) { - log.info("Assign splits {} to reader {}", assignmentForReader, reader); - try { - context.assignSplit(reader, assignmentForReader); - } catch (Exception e) { - log.error( - "Failed to assign splits {} to reader {}", - assignmentForReader, - reader, - e); - pendingSplit.put(reader, assignmentForReader); - } - } - } - } - - private static int getSplitOwner(String tp, int numReaders) { - return (tp.hashCode() & Integer.MAX_VALUE) % numReaders; - } - - private List getTypesenseSplit() { - List splits = new ArrayList<>(); - - String collection = config.get(SourceConfig.COLLECTION); - String query = config.get(SourceConfig.QUERY); - int queryBatchSize = config.get(SourceConfig.QUERY_BATCH_SIZE); - splits.add( - new TypesenseSourceSplit( - collection, - new SourceCollectionInfo(collection, query, 0, 0, queryBatchSize))); - return splits; - } - - @Override - public void close() throws IOException { - // Nothing - } - - @Override - public void addSplitsBack(List splits, int subtaskId) { - if (!splits.isEmpty()) { - addPendingSplit(splits); - assignSplit(Collections.singletonList(subtaskId)); - } - } - - @Override - public int currentUnassignedSplitSize() { - return pendingSplit.size(); - } - - @Override - public void handleSplitRequest(int subtaskId) { - throw new TypesenseConnectorException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "Unsupported handleSplitRequest: " + subtaskId); - } - - @Override - public void registerReader(int subtaskId) { - log.debug("Register reader {} to IoTDBSourceSplitEnumerator.", subtaskId); - if (!pendingSplit.isEmpty()) { - assignSplit(Collections.singletonList(subtaskId)); - } - } - - @Override - public TypesenseSourceState snapshotState(long checkpointId) throws Exception { - synchronized (stateLock) { - return new TypesenseSourceState(shouldEnumerate, pendingSplit); - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception {} -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java deleted file mode 100644 index b0f814dd8fd..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseAggregatedCommitInfo.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.state; - -import java.io.Serializable; - -public class TypesenseAggregatedCommitInfo implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java deleted file mode 100644 index 05e0ea83f72..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseCommitInfo.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.state; - -import java.io.Serializable; - -public class TypesenseCommitInfo implements Serializable {} diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java b/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java deleted file mode 100644 index cc6f26cd43f..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverter.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.util; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; - -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorErrorCode; -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - -public class URLParamsConverter { - - public static String convertParamsToJson(String paramsString) { - return Optional.ofNullable(paramsString) - .filter(s -> !s.isEmpty()) - .map(URLParamsConverter::parseParams) - .map( - paramsMap -> { - try { - return new ObjectMapper().writeValueAsString(paramsMap); - } catch (IOException e) { - throw new RuntimeException("Error converting params to JSON", e); - } - }) - .orElseThrow( - () -> - new IllegalArgumentException( - "Parameter string must not be null or empty.")); - } - - private static Map parseParams(String paramsString) { - return Arrays.stream( - Optional.ofNullable(paramsString) - .filter(s -> !s.isEmpty()) - .orElseThrow( - () -> - new IllegalArgumentException( - "Parameter string must not be null or empty.")) - .split("&")) - .map(part -> part.split("=", 2)) - .peek( - keyValue -> { - if (keyValue.length != 2) { - throw new TypesenseConnectorException( - TypesenseConnectorErrorCode.QUERY_PARAM_ERROR, - "Query parameter error: " + Arrays.toString(keyValue)); - } - }) - .collect(Collectors.toMap(keyValue -> keyValue[0], keyValue -> keyValue[1])); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java deleted file mode 100644 index bb604869ff1..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/serializer/TypesenseRowSerializerTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.serializer; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.table.type.RowKind; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.typesense.config.SinkConfig; -import org.apache.seatunnel.connectors.seatunnel.typesense.dto.CollectionInfo; -import org.apache.seatunnel.connectors.seatunnel.typesense.serialize.sink.TypesenseRowSerializer; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; - -public class TypesenseRowSerializerTest { - @Test - public void testSerializeUpsert() { - String collection = "test"; - String primaryKey = "id"; - Map confMap = new HashMap<>(); - confMap.put(SinkConfig.COLLECTION.key(), collection); - confMap.put(SinkConfig.PRIMARY_KEYS.key(), Arrays.asList(primaryKey)); - - ReadonlyConfig pluginConf = ReadonlyConfig.fromMap(confMap); - CollectionInfo collectionInfo = new CollectionInfo(collection, pluginConf); - SeaTunnelRowType schema = - new SeaTunnelRowType( - new String[] {primaryKey, "name"}, - new SeaTunnelDataType[] {STRING_TYPE, STRING_TYPE}); - TypesenseRowSerializer typesenseRowSerializer = - new TypesenseRowSerializer(collectionInfo, schema); - String id = "0001"; - String name = "jack"; - SeaTunnelRow row = new SeaTunnelRow(new Object[] {id, name}); - row.setRowKind(RowKind.UPDATE_AFTER); - Assertions.assertEquals(typesenseRowSerializer.serializeRowForDelete(row), id); - row.setRowKind(RowKind.INSERT); - String data = "{\"name\":\"jack\",\"id\":\"0001\"}"; - Assertions.assertEquals(typesenseRowSerializer.serializeRow(row), data); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java deleted file mode 100644 index 568c96dbeed..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/sink/TypesenseFactoryTest.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.sink; - -import org.apache.seatunnel.connectors.seatunnel.typesense.source.TypesenseSourceFactory; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class TypesenseFactoryTest { - - @Test - void optionRule() { - Assertions.assertNotNull((new TypesenseSourceFactory()).optionRule()); - Assertions.assertNotNull((new TypesenseSinkFactory()).optionRule()); - } -} diff --git a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java b/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java deleted file mode 100644 index 716cb7cc20a..00000000000 --- a/seatunnel-connectors-v2/connector-typesense/src/test/java/org/apache/seatunnel/connectors/seatunnel/typesense/util/URLParamsConverterTest.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.typesense.util; - -import org.apache.seatunnel.connectors.seatunnel.typesense.exception.TypesenseConnectorException; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class URLParamsConverterTest { - - @Test - public void convertParamsToJson() { - String json = URLParamsConverter.convertParamsToJson("q=*&filter_by=num_employees:10"); - Assertions.assertEquals(json, "{\"q\":\"*\",\"filter_by\":\"num_employees:10\"}"); - Assertions.assertThrows( - TypesenseConnectorException.class, - () -> URLParamsConverter.convertParamsToJson("q=*&filter_by=num_employees:10&b")); - } -} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index cf7314e619a..6bd4065d088 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -79,9 +79,7 @@ connector-web3j connector-milvus connector-activemq - connector-qdrant connector-sls - connector-typesense diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd index 89327e8e38f..c4a4db1bc12 100644 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.cmd @@ -45,9 +45,14 @@ for %%I in (%*) do ( set "JAVA_OPTS=%JvmOption%" set "SEATUNNEL_CONFIG=%CONF_DIR%\seatunnel.yaml" +for %%I in (%*) do ( + set "arg=%%I" + if "!arg:~0,10!"=="JvmOption=" ( + set "JAVA_OPTS=%JAVA_OPTS% !arg:~10!" + ) +) set "JAVA_OPTS=%JAVA_OPTS% -Dlog4j2.contextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector" -set "JAVA_OPTS=%JAVA_OPTS% -Dlog4j2.isThreadContextMapInheritable=true" REM Server Debug Config REM Usage instructions: @@ -99,14 +104,6 @@ if "%NODE_ROLE%" == "master" ( exit 1 ) -REM Parse JvmOption from command line, it should be parsed after jvm_options -for %%I in (%*) do ( - set "arg=%%I" - if "!arg:~0,10!"=="JvmOption=" ( - set "JAVA_OPTS=%JAVA_OPTS% !arg:~10!" - ) -) - IF NOT EXIST "%HAZELCAST_CONFIG%" ( echo Error: File %HAZELCAST_CONFIG% does not exist. exit /b 1 diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh index 565644db22b..f9c7a33b406 100755 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel-cluster.sh @@ -68,7 +68,8 @@ fi for i in "$@" do if [[ "${i}" == *"JvmOption"* ]]; then - : + JVM_OPTION="${i}" + JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" elif [[ "${i}" == "-d" || "${i}" == "--daemon" ]]; then DAEMON=true elif [[ "${i}" == "-r" || "${i}" == "--role" ]]; then @@ -83,7 +84,6 @@ done # Log4j2 Config JAVA_OPTS="${JAVA_OPTS} -Dlog4j2.contextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector" -JAVA_OPTS="${JAVA_OPTS} -Dlog4j2.isThreadContextMapInheritable=true" if [ -e "${CONF_DIR}/log4j2.properties" ]; then JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.logging.type=log4j2 -Dlog4j2.configurationFile=${CONF_DIR}/log4j2.properties" JAVA_OPTS="${JAVA_OPTS} -Dseatunnel.logs.path=${APP_DIR}/logs" @@ -139,15 +139,6 @@ JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.config=${HAZELCAST_CONFIG}" # port in your IDE. After that, you can happily debug your code. # JAVA_OPTS="${JAVA_OPTS} -Xdebug -Xrunjdwp:server=y,transport=dt_socket,address=5001,suspend=n" -# Parse JvmOption from command line, it should be parsed after jvm_options -for i in "$@" -do - if [[ "${i}" == *"JvmOption"* ]]; then - JVM_OPTION="${i}" - JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" - fi -done - CLASS_PATH=${APP_DIR}/lib/*:${APP_JAR} echo "start ${NODE_ROLE} node" diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd index cecd3797cda..b7208616714 100644 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.cmd @@ -58,6 +58,16 @@ if defined JvmOption ( set "JAVA_OPTS=%JAVA_OPTS% %JvmOption%" ) +for %%i in (%*) do ( + set "arg=%%i" + if "!arg:~0,9!"=="JvmOption" ( + set "JVM_OPTION=!arg:~9!" + set "JAVA_OPTS=!JAVA_OPTS! !JVM_OPTION!" + goto :break_loop + ) +) +:break_loop + set "JAVA_OPTS=%JAVA_OPTS% -Dhazelcast.client.config=%HAZELCAST_CLIENT_CONFIG%" set "JAVA_OPTS=%JAVA_OPTS% -Dseatunnel.config=%SEATUNNEL_CONFIG%" set "JAVA_OPTS=%JAVA_OPTS% -Dhazelcast.config=%HAZELCAST_CONFIG%" @@ -95,15 +105,4 @@ for /f "usebackq delims=" %%a in ("%APP_DIR%\config\jvm_client_options") do ( ) ) -REM Parse JvmOption from command line, it should be parsed after jvm_client_options -for %%i in (%*) do ( - set "arg=%%i" - if "!arg:~0,9!"=="JvmOption" ( - set "JVM_OPTION=!arg:~9!" - set "JAVA_OPTS=!JAVA_OPTS! !JVM_OPTION!" - goto :break_loop - ) -) -:break_loop - java %JAVA_OPTS% -cp %CLASS_PATH% %APP_MAIN% %args% diff --git a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh index dc4c3f91e56..e7fbee00b36 100755 --- a/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh +++ b/seatunnel-core/seatunnel-starter/src/main/bin/seatunnel.sh @@ -69,6 +69,15 @@ if test ${JvmOption} ;then JAVA_OPTS="${JAVA_OPTS} ${JvmOption}" fi +for i in "$@" +do + if [[ "${i}" == *"JvmOption"* ]]; then + JVM_OPTION="${i}" + JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" + break + fi +done + JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.client.config=${HAZELCAST_CLIENT_CONFIG}" JAVA_OPTS="${JAVA_OPTS} -Dseatunnel.config=${SEATUNNEL_CONFIG}" JAVA_OPTS="${JAVA_OPTS} -Dhazelcast.config=${HAZELCAST_CONFIG}" @@ -99,14 +108,4 @@ while IFS= read -r line || [[ -n "$line" ]]; do fi done < ${APP_DIR}/config/jvm_client_options -# Parse JvmOption from command line, it should be parsed after jvm_client_options -for i in "$@" -do - if [[ "${i}" == *"JvmOption"* ]]; then - JVM_OPTION="${i}" - JAVA_OPTS="${JAVA_OPTS} ${JVM_OPTION#*=}" - break - fi -done - java ${JAVA_OPTS} -cp ${CLASS_PATH} ${APP_MAIN} ${args} diff --git a/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties b/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties index 0bf06d2b773..2dc1b8ca501 100644 --- a/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties +++ b/seatunnel-core/seatunnel-starter/src/main/resources/log4j2.properties @@ -25,7 +25,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -35,7 +35,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 584224c8fd9..fb6935f0894 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -518,12 +518,6 @@ ${project.version} provided - - org.apache.seatunnel - connector-cdc-opengauss - ${project.version} - provided - org.apache.seatunnel connector-tdengine @@ -598,13 +592,6 @@ provided - - org.apache.seatunnel - connector-qdrant - ${project.version} - provided - - com.aliyun.phoenix diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE index 5cc3074d417..f5d9bebbe54 100644 --- a/seatunnel-dist/release-docs/LICENSE +++ b/seatunnel-dist/release-docs/LICENSE @@ -267,7 +267,6 @@ The text of each license is the standard Apache 2.0 license. (Apache-2.0) woodstox-core (com.fasterxml.woodstox:woodstox-core:5.0.3 - https://github.com/FasterXML/woodstox) (Apache-2.0) jcip-annotations (com.github.stephenc.jcip:jcip-annotations:1.0-1 - https://github.com/stephenc/jcip-annotations) (Apache-2.0) gson (com.google.code.gson:gson:2.2.4 - https://github.com/google/gson) - (Apache-2.0) gson (com.google.code.gson:gson:2.8.9 - https://github.com/google/gson) (Apache-2.0) nimbus-jose-jwt (com.nimbusds:nimbus-jose-jwt:7.9 - https://bitbucket.org/connect2id/nimbus-jose-jwt) (Apache-2.0) beanutils (commons-beanutils:commons-beanutils:1.9.4 - https://commons.apache.org/proper/commons-beanutils/) (Apache-2.0) commons-cli (commons-cli:commons-cli:1.2 - https://commons.apache.org/proper/commons-cli/) @@ -319,10 +318,8 @@ The text of each license is the standard Apache 2.0 license. (The Apache Software License, Version 2.0) hazelcast (com.hazelcast:hazelcast:5.1 - https://github.com/hazelcast/hazelcast) (Apache-2.0) disruptor (com.lmax:disruptor:3.4.4 https://lmax-exchange.github.io/disruptor/) (Apache-2.0) error_prone_annotations (com.google.errorprone:error_prone_annotations:2.2.0 https://mvnrepository.com/artifact/com.google.errorprone/error_prone_annotations/2.2.0) - (Apache-2.0) error_prone_annotations (com.google.errorprone:error_prone_annotations:2.18.0 https://mvnrepository.com/artifact/com.google.errorprone/error_prone_annotations/2.18.0) (Apache-2.0) failureaccess (com.google.guava:failureaccess:1.0 https://mvnrepository.com/artifact/com.google.guava/failureaccess/1.0) (Apache-2.0) j2objc-annotations (com.google.j2objc:j2objc-annotations:1.1 https://mvnrepository.com/artifact/com.google.j2objc/j2objc-annotations/1.1) - (Apache-2.0) j2objc-annotations (com.google.j2objc:j2objc-annotations:2.8 https://mvnrepository.com/artifact/com.google.j2objc/j2objc-annotations/2.8) (Apache-2.0) listenablefuture (com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava https://mvnrepository.com/artifact/com.google.guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava) (Apache-2.0) accessors-smart (net.minidev:accessors-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/accessors-smart) (Apache-2.0) json-smart (net.minidev:json-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/json-smart) diff --git a/seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt b/seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt deleted file mode 100644 index ad410e11302..00000000000 --- a/seatunnel-dist/release-docs/licenses/LICENSE-protoc-jar.txt +++ /dev/null @@ -1,201 +0,0 @@ -Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "{}" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright {yyyy} {name of copyright owner} - - Licensed 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. \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml deleted file mode 100644 index f95e5cdb1a0..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/pom.xml +++ /dev/null @@ -1,75 +0,0 @@ - - - - 4.0.0 - - org.apache.seatunnel - seatunnel-connector-v2-e2e - ${revision} - - - connector-cdc-opengauss-e2e - SeaTunnel : E2E : Connector V2 : CDC Opengauss - - - - - org.apache.seatunnel - connector-jdbc - ${project.version} - pom - import - - - - - - - - org.apache.seatunnel - connector-cdc-opengauss - ${project.version} - test - - - - org.apache.seatunnel - connector-jdbc - ${project.version} - test - - - - org.testcontainers - postgresql - ${testcontainer.version} - test - - - - - org.postgresql - postgresql - 42.5.1 - - - - diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java deleted file mode 100644 index dc80a083a76..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/OpengaussCDCIT.java +++ /dev/null @@ -1,727 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres; - -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; -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.lifecycle.Startables; -import org.testcontainers.utility.DockerImageName; - -import com.google.common.collect.Lists; -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.awaitility.Awaitility.await; -import static org.awaitility.Awaitility.given; -import static org.junit.Assert.assertNotNull; - -@Slf4j -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK}, - disabledReason = "Currently SPARK do not support cdc") -public class OpengaussCDCIT extends TestSuiteBase implements TestResource { - private static final int OPENGAUSS_PORT = 5432; - private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); - private static final String USERNAME = "gaussdb"; - private static final String PASSWORD = "openGauss@123"; - private static final String OPENGAUSSQL_DATABASE = "opengauss_cdc"; - private static final String OPENGAUSSQL_DEFAULT_DATABASE = "postgres"; - private static final String OPENGAUSS_SCHEMA = "inventory"; - - private static final String SOURCE_TABLE_1 = "opengauss_cdc_table_1"; - private static final String SOURCE_TABLE_2 = "opengauss_cdc_table_2"; - private static final String SOURCE_TABLE_3 = "opengauss_cdc_table_3"; - private static final String SINK_TABLE_1 = "sink_opengauss_cdc_table_1"; - private static final String SINK_TABLE_2 = "sink_opengauss_cdc_table_2"; - private static final String SINK_TABLE_3 = "sink_opengauss_cdc_table_3"; - - private static final String SOURCE_TABLE_NO_PRIMARY_KEY = "full_types_no_primary_key"; - - private static final String OPENGAUSS_HOST = "opengauss_cdc_e2e"; - - protected static final DockerImageName OPENGAUSS_IMAGE = - DockerImageName.parse("opengauss/opengauss:5.0.0") - .asCompatibleSubstituteFor("postgres"); - - private static final String SOURCE_SQL_TEMPLATE = "select * from %s.%s order by id"; - - public static final GenericContainer OPENGAUSS_CONTAINER = - new GenericContainer<>(OPENGAUSS_IMAGE) - .withNetwork(NETWORK) - .withNetworkAliases(OPENGAUSS_HOST) - .withEnv("GS_PASSWORD", PASSWORD) - .withLogConsumer(new Slf4jLogConsumer(log)); - - private String driverUrl() { - return "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.5.1/postgresql-42.5.1.jar"; - } - - @TestContainerExtension - protected final ContainerExtendedFactory extendedFactory = - container -> { - Container.ExecResult extraCommands = - container.execInContainer( - "bash", - "-c", - "mkdir -p /tmp/seatunnel/plugins/JDBC/lib && cd /tmp/seatunnel/plugins/JDBC/lib && wget " - + driverUrl()); - Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); - }; - - @BeforeAll - @Override - public void startUp() throws Exception { - log.info("The second stage: Starting opengauss containers..."); - OPENGAUSS_CONTAINER.setPortBindings( - Lists.newArrayList(String.format("%s:%s", OPENGAUSS_PORT, OPENGAUSS_PORT))); - Startables.deepStart(Stream.of(OPENGAUSS_CONTAINER)).join(); - log.info("Opengauss Containers are started"); - given().ignoreExceptions() - .await() - .atLeast(100, TimeUnit.MILLISECONDS) - .pollInterval(2, TimeUnit.SECONDS) - .atMost(2, TimeUnit.MINUTES) - .untilAsserted(this::initializeOpengaussSql); - - String[] command1 = { - "/bin/sh", - "-c", - "sed -i 's/^#password_encryption_type = 2/password_encryption_type = 1/' /var/lib/opengauss/data/postgresql.conf" - }; - Container.ExecResult result1 = OPENGAUSS_CONTAINER.execInContainer(command1); - Assertions.assertEquals(0, result1.getExitCode()); - - String[] command2 = { - "/bin/sh", - "-c", - "sed -i 's/host replication gaussdb 0.0.0.0\\/0 md5/host replication gaussdb 0.0.0.0\\/0 sha256/' /var/lib/opengauss/data/pg_hba.conf" - }; - Container.ExecResult result2 = OPENGAUSS_CONTAINER.execInContainer(command2); - Assertions.assertEquals(0, result2.getExitCode()); - String[] command3 = { - "/bin/sh", - "-c", - "echo \"host all dailai 0.0.0.0/0 md5\" >> /var/lib/opengauss/data/pg_hba.conf" - }; - Container.ExecResult result3 = OPENGAUSS_CONTAINER.execInContainer(command3); - Assertions.assertEquals(0, result3.getExitCode()); - - reloadConf(); - - createNewUserForJdbcSink(); - } - - @TestTemplate - public void testOpengaussCdcCheckDataE2e(TestContainer container) { - try { - CompletableFuture.supplyAsync( - () -> { - try { - container.executeJob("/opengausscdc_to_opengauss.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - return null; - }); - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assertions.assertIterableEquals( - query(getQuerySQL(OPENGAUSS_SCHEMA, SOURCE_TABLE_1)), - query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); - }); - - // insert update delete - upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assertions.assertIterableEquals( - query(getQuerySQL(OPENGAUSS_SCHEMA, SOURCE_TABLE_1)), - query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); - }); - } finally { - // Clear related content to ensure that multiple operations are not affected - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); - } - } - - @TestTemplate - @DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support multi table") - public void testOpengaussCdcMultiTableE2e(TestContainer container) { - try { - CompletableFuture.supplyAsync( - () -> { - try { - container.executeJob( - "/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - return null; - }); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> - Assertions.assertAll( - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_1)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_1))), - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_2)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_2))))); - - // insert update delete - upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); - upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> - Assertions.assertAll( - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_1)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_1))), - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_2)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_2))))); - } finally { - // Clear related content to ensure that multiple operations are not affected - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_2); - } - } - - @TestTemplate - @DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support multi table") - public void testMultiTableWithRestore(TestContainer container) - throws IOException, InterruptedException { - try { - CompletableFuture.supplyAsync( - () -> { - try { - return container.executeJob( - "/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - // insert update delete - upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> - Assertions.assertAll( - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_1)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_1))))); - - Pattern jobIdPattern = - Pattern.compile( - ".*Init JobMaster for Job opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf \\(([0-9]*)\\).*", - Pattern.DOTALL); - Matcher matcher = jobIdPattern.matcher(container.getServerLogs()); - String jobId; - if (matcher.matches()) { - jobId = matcher.group(1); - } else { - throw new RuntimeException("Can not find jobId"); - } - - Assertions.assertEquals(0, container.savepointJob(jobId).getExitCode()); - - // Restore job with add a new table - CompletableFuture.supplyAsync( - () -> { - try { - container.restoreJob( - "/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf", - jobId); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - return null; - }); - - upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> - Assertions.assertAll( - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_1)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_1))), - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_2)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_2))))); - - log.info("****************** container logs start ******************"); - String containerLogs = container.getServerLogs(); - log.info(containerLogs); - // pg cdc logs contain ERROR - // Assertions.assertFalse(containerLogs.contains("ERROR")); - log.info("****************** container logs end ******************"); - } finally { - // Clear related content to ensure that multiple operations are not affected - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_1); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_2); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_2); - } - } - - @TestTemplate - @DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support multi table") - public void testAddFiledWithRestore(TestContainer container) - throws IOException, InterruptedException { - try { - CompletableFuture.supplyAsync( - () -> { - try { - return container.executeJob( - "/opengausscdc_to_opengauss_test_add_Filed.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - }); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> - Assertions.assertAll( - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_3)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_3))))); - - Pattern jobIdPattern = - Pattern.compile( - ".*Init JobMaster for Job opengausscdc_to_opengauss_test_add_Filed.conf \\(([0-9]*)\\).*", - Pattern.DOTALL); - Matcher matcher = jobIdPattern.matcher(container.getServerLogs()); - String jobId; - if (matcher.matches()) { - jobId = matcher.group(1); - } else { - throw new RuntimeException("Can not find jobId"); - } - - Assertions.assertEquals(0, container.savepointJob(jobId).getExitCode()); - - // add filed add insert source table data - addFieldsForTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_3); - addFieldsForTable(OPENGAUSS_SCHEMA, SINK_TABLE_3); - insertSourceTableForAddFields(OPENGAUSS_SCHEMA, SOURCE_TABLE_3); - - // Restore job - CompletableFuture.supplyAsync( - () -> { - try { - container.restoreJob( - "/opengausscdc_to_opengauss_test_add_Filed.conf", jobId); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - return null; - }); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> - Assertions.assertAll( - () -> - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_3)), - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SINK_TABLE_3))))); - } finally { - // Clear related content to ensure that multiple operations are not affected - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_3); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_3); - } - } - - @TestTemplate - public void testOpengaussCdcCheckDataWithNoPrimaryKey(TestContainer container) - throws Exception { - - try { - CompletableFuture.supplyAsync( - () -> { - try { - container.executeJob( - "/opengausscdc_to_opengauss_with_no_primary_key.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - return null; - }); - - // snapshot stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_NO_PRIMARY_KEY)), - query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); - }); - - // insert update delete - upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_NO_PRIMARY_KEY)), - query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); - }); - } finally { - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); - } - } - - @TestTemplate - public void testOpengaussCdcCheckDataWithCustomPrimaryKey(TestContainer container) - throws Exception { - - try { - CompletableFuture.supplyAsync( - () -> { - try { - container.executeJob( - "/opengausscdc_to_opengauss_with_custom_primary_key.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - return null; - }); - - // snapshot stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_NO_PRIMARY_KEY)), - query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); - }); - - // insert update delete - upsertDeleteSourceTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); - - // stream stage - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assertions.assertIterableEquals( - query( - getQuerySQL( - OPENGAUSS_SCHEMA, - SOURCE_TABLE_NO_PRIMARY_KEY)), - query(getQuerySQL(OPENGAUSS_SCHEMA, SINK_TABLE_1))); - }); - } finally { - clearTable(OPENGAUSS_SCHEMA, SOURCE_TABLE_NO_PRIMARY_KEY); - clearTable(OPENGAUSS_SCHEMA, SINK_TABLE_1); - } - } - - private void addFieldsForTable(String database, String tableName) { - executeSql("ALTER TABLE " + database + "." + tableName + " ADD COLUMN f_big BIGINT"); - } - - private void insertSourceTableForAddFields(String database, String tableName) { - executeSql( - "INSERT INTO " - + database - + "." - + tableName - + " VALUES (2, '2', 32767, 65535, 2147483647);"); - } - - private void clearTable(String database, String tableName) { - executeSql("truncate table " + database + "." + tableName); - } - - private void upsertDeleteSourceTable(String database, String tableName) { - - executeSql( - "INSERT INTO " - + database - + "." - + tableName - + " VALUES (2, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true,\n" - + " 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456',\n" - + " '2020-07-17', '18:00:22', 500);"); - - executeSql( - "INSERT INTO " - + database - + "." - + tableName - + " VALUES (3, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true,\n" - + " 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456',\n" - + " '2020-07-17', '18:00:22', 500);"); - - executeSql("DELETE FROM " + database + "." + tableName + " where id = 2;"); - - executeSql("UPDATE " + database + "." + tableName + " SET f_big = 10000 where id = 3;"); - } - - private void executeSql(String sql) { - try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); - Statement statement = connection.createStatement()) { - statement.execute("SET search_path TO inventory;"); - statement.execute(sql); - } catch (SQLException e) { - throw new RuntimeException(e); - } - } - - private String getQuerySQL(String database, String tableName) { - return String.format(SOURCE_SQL_TEMPLATE, database, tableName); - } - - private List> query(String sql) { - try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE)) { - ResultSet resultSet = connection.createStatement().executeQuery(sql); - List> result = new ArrayList<>(); - int columnCount = resultSet.getMetaData().getColumnCount(); - while (resultSet.next()) { - ArrayList objects = new ArrayList<>(); - for (int i = 1; i <= columnCount; i++) { - Object object = resultSet.getObject(i); - if (object instanceof byte[]) { - byte[] bytes = (byte[]) object; - object = new String(bytes, StandardCharsets.UTF_8); - } - objects.add(object); - } - log.debug( - String.format( - "Print opengauss-CDC query, sql: %s, data: %s", sql, objects)); - result.add(objects); - } - return result; - } catch (SQLException e) { - throw new RuntimeException(e); - } - } - - protected void createNewUserForJdbcSink() throws Exception { - try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); - Statement stmt = connection.createStatement()) { - // create a user for jdbc sink - stmt.execute("CREATE USER dailai WITH PASSWORD 'openGauss@123';"); - stmt.execute("GRANT ALL PRIVILEGES TO dailai;"); - } - } - - protected void reloadConf() throws Exception { - try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); - Statement stmt = connection.createStatement()) { - stmt.execute("select pg_reload_conf();"); - } - } - - protected void initializeOpengaussSql() throws Exception { - try (Connection connection = getJdbcConnection(OPENGAUSSQL_DEFAULT_DATABASE); - Statement stmt = connection.createStatement()) { - stmt.execute("create database " + OPENGAUSSQL_DATABASE); - } - final String ddlFile = String.format("ddl/%s.sql", "inventory"); - final URL ddlTestFile = OpengaussCDCIT.class.getClassLoader().getResource(ddlFile); - assertNotNull("Cannot locate " + ddlFile, ddlTestFile); - try (Connection connection = getJdbcConnection(OPENGAUSSQL_DATABASE); - Statement statement = connection.createStatement()) { - final List statements = - Arrays.stream( - Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() - .map(String::trim) - .filter(x -> !x.startsWith("--") && !x.isEmpty()) - .map( - x -> { - final Matcher m = - COMMENT_PATTERN.matcher(x); - return m.matches() ? m.group(1) : x; - }) - .collect(Collectors.joining("\n")) - .split(";\n")) - .collect(Collectors.toList()); - for (String stmt : statements) { - statement.execute(stmt); - } - } - } - - private Connection getJdbcConnection(String dbName) throws SQLException { - return DriverManager.getConnection( - "jdbc:postgresql://" - + OPENGAUSS_CONTAINER.getHost() - + ":" - + OPENGAUSS_CONTAINER.getMappedPort(OPENGAUSS_PORT) - + "/" - + dbName, - USERNAME, - PASSWORD); - } - - @AfterAll - @Override - public void tearDown() throws Exception { - if (OPENGAUSS_CONTAINER != null) { - OPENGAUSS_CONTAINER.close(); - } - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql deleted file mode 100644 index 5d4564041b7..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/ddl/inventory.sql +++ /dev/null @@ -1,199 +0,0 @@ --- --- Licensed to the Apache Software Foundation (ASF) under one or more --- contributor license agreements. See the NOTICE file distributed with --- this work for additional information regarding copyright ownership. --- The ASF licenses this file to You under the Apache License, Version 2.0 --- (the "License"); you may not use this file except in compliance with --- the License. You may obtain a copy of the License at --- --- http://www.apache.org/licenses/LICENSE-2.0 --- --- Unless required by applicable law or agreed to in writing, software --- distributed under the License is distributed on an "AS IS" BASIS, --- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. --- See the License for the specific language governing permissions and --- limitations under the License. --- - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: opengauss_cdc --- ---------------------------------------------------------------------------------------------------------------- --- Create and populate our products using a single insert with many rows -DROP SCHEMA IF EXISTS inventory CASCADE; -CREATE SCHEMA inventory; -SET search_path TO inventory; - -CREATE TABLE opengauss_cdc_table_1 -( - id INTEGER NOT NULL, - f_bytea BYTEA, - f_small SMALLINT, - f_int INTEGER, - f_big BIGINT, - f_real REAL, - f_double_precision DOUBLE PRECISION, - f_numeric NUMERIC(10, 5), - f_decimal DECIMAL(10, 1), - f_boolean BOOLEAN, - f_text TEXT, - f_char CHAR, - f_character CHARACTER(3), - f_character_varying CHARACTER VARYING(20), - f_timestamp3 TIMESTAMP(3), - f_timestamp6 TIMESTAMP(6), - f_date DATE, - f_time TIME(0), - f_default_numeric NUMERIC, - PRIMARY KEY (id) -); - -CREATE TABLE opengauss_cdc_table_2 -( - id INTEGER NOT NULL, - f_bytea BYTEA, - f_small SMALLINT, - f_int INTEGER, - f_big BIGINT, - f_real REAL, - f_double_precision DOUBLE PRECISION, - f_numeric NUMERIC(10, 5), - f_decimal DECIMAL(10, 1), - f_boolean BOOLEAN, - f_text TEXT, - f_char CHAR, - f_character CHARACTER(3), - f_character_varying CHARACTER VARYING(20), - f_timestamp3 TIMESTAMP(3), - f_timestamp6 TIMESTAMP(6), - f_date DATE, - f_time TIME(0), - f_default_numeric NUMERIC, - PRIMARY KEY (id) -); - -CREATE TABLE sink_opengauss_cdc_table_1 -( - id INTEGER NOT NULL, - f_bytea BYTEA, - f_small SMALLINT, - f_int INTEGER, - f_big BIGINT, - f_real REAL, - f_double_precision DOUBLE PRECISION, - f_numeric NUMERIC(10, 5), - f_decimal DECIMAL(10, 1), - f_boolean BOOLEAN, - f_text TEXT, - f_char CHAR, - f_character CHARACTER(3), - f_character_varying CHARACTER VARYING(20), - f_timestamp3 TIMESTAMP(3), - f_timestamp6 TIMESTAMP(6), - f_date DATE, - f_time TIME(0), - f_default_numeric NUMERIC, - PRIMARY KEY (id) -); - -CREATE TABLE sink_opengauss_cdc_table_2 -( - id INTEGER NOT NULL, - f_bytea BYTEA, - f_small SMALLINT, - f_int INTEGER, - f_big BIGINT, - f_real REAL, - f_double_precision DOUBLE PRECISION, - f_numeric NUMERIC(10, 5), - f_decimal DECIMAL(10, 1), - f_boolean BOOLEAN, - f_text TEXT, - f_char CHAR, - f_character CHARACTER(3), - f_character_varying CHARACTER VARYING(20), - f_timestamp3 TIMESTAMP(3), - f_timestamp6 TIMESTAMP(6), - f_date DATE, - f_time TIME(0), - f_default_numeric NUMERIC, - PRIMARY KEY (id) -); - -CREATE TABLE full_types_no_primary_key -( - id INTEGER NOT NULL, - f_bytea BYTEA, - f_small SMALLINT, - f_int INTEGER, - f_big BIGINT, - f_real REAL, - f_double_precision DOUBLE PRECISION, - f_numeric NUMERIC(10, 5), - f_decimal DECIMAL(10, 1), - f_boolean BOOLEAN, - f_text TEXT, - f_char CHAR, - f_character CHARACTER(3), - f_character_varying CHARACTER VARYING(20), - f_timestamp3 TIMESTAMP(3), - f_timestamp6 TIMESTAMP(6), - f_date DATE, - f_time TIME(0), - f_default_numeric NUMERIC -); - -CREATE TABLE opengauss_cdc_table_3 -( - id INTEGER NOT NULL, - f_bytea BYTEA, - f_small SMALLINT, - f_int INTEGER, - PRIMARY KEY (id) -); - -CREATE TABLE sink_opengauss_cdc_table_3 -( - id INTEGER NOT NULL, - f_bytea BYTEA, - f_small SMALLINT, - f_int INTEGER, - PRIMARY KEY (id) -); - -ALTER TABLE opengauss_cdc_table_1 - REPLICA IDENTITY FULL; - -ALTER TABLE opengauss_cdc_table_2 - REPLICA IDENTITY FULL; - -ALTER TABLE opengauss_cdc_table_3 - REPLICA IDENTITY FULL; - -ALTER TABLE sink_opengauss_cdc_table_1 - REPLICA IDENTITY FULL; - -ALTER TABLE sink_opengauss_cdc_table_2 - REPLICA IDENTITY FULL; - -ALTER TABLE full_types_no_primary_key - REPLICA IDENTITY FULL; - -INSERT INTO opengauss_cdc_table_1 -VALUES (1, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true, - 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', - '2020-07-17', '18:00:22', 500); - -INSERT INTO opengauss_cdc_table_2 -VALUES (1, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true, - 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', - '2020-07-17', '18:00:22', 500); - -INSERT INTO opengauss_cdc_table_3 -VALUES (1, '2', 32767, 65535); - -INSERT INTO full_types_no_primary_key -VALUES (1, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true, - 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', - '2020-07-17', '18:00:22', 500); - - diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf deleted file mode 100644 index c1227226362..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss.conf +++ /dev/null @@ -1,62 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = opengauss_cdc - table = inventory.sink_opengauss_cdc_table_1 - primary_keys = ["id"] - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf deleted file mode 100644 index 3c925f55311..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_test_add_Filed.conf +++ /dev/null @@ -1,62 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_3"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = opengauss_cdc - table = inventory.sink_opengauss_cdc_table_3 - primary_keys = ["id"] - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf deleted file mode 100644 index d916cb5e4e4..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_custom_primary_key.conf +++ /dev/null @@ -1,69 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - exactly_once = true - table-names-config = [ - { - table = "opengauss_cdc.inventory.full_types_no_primary_key" - primaryKeys = ["id"] - } - ] - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = opengauss_cdc - table = inventory.sink_opengauss_cdc_table_1 - primary_keys = ["id"] - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf deleted file mode 100644 index f75d7dc97d8..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_one_table.conf +++ /dev/null @@ -1,63 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = "opengauss_cdc" - schema = "inventory" - tablePrefix = "sink_" - primary_keys = ["id"] - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf deleted file mode 100644 index d5ac1000012..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_multi_table_mode_two_table.conf +++ /dev/null @@ -1,63 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - table-names = ["opengauss_cdc.inventory.opengauss_cdc_table_1","opengauss_cdc.inventory.opengauss_cdc_table_2"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = "opengauss_cdc" - schema = "inventory" - tablePrefix = "sink_" - primary_keys = ["id"] - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf deleted file mode 100644 index 5decfb10606..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-opengauss-e2e/src/test/resources/opengausscdc_to_opengauss_with_no_primary_key.conf +++ /dev/null @@ -1,63 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "STREAMING" - checkpoint.interval = 5000 - read_limit.bytes_per_second=7000000 - read_limit.rows_per_second=400 -} - -source { - Opengauss-CDC { - result_table_name = "customers_opengauss_cdc" - username = "gaussdb" - password = "openGauss@123" - database-names = ["opengauss_cdc"] - schema-names = ["inventory"] - base-url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - decoding.plugin.name = "pgoutput" - table-names = ["opengauss_cdc.inventory.full_types_no_primary_key"] - exactly_once = false - } -} - -transform { - -} - -sink { - jdbc { - source_table_name = "customers_opengauss_cdc" - url = "jdbc:postgresql://opengauss_cdc_e2e:5432/opengauss_cdc?loggerLevel=OFF" - driver = "org.postgresql.Driver" - user = "dailai" - password = "openGauss@123" - - compatible_mode="postgresLow" - generate_sink_sql = true - # You need to configure both database and table - database = opengauss_cdc - table = inventory.sink_opengauss_cdc_table_1 - primary_keys = ["id"] - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java index 650918e3fc6..e92c3993780 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/java/org/apache/seatunnel/e2e/connector/elasticsearch/ElasticsearchIT.java @@ -20,7 +20,6 @@ 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.shade.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.TablePath; @@ -69,27 +68,17 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.LockSupport; -import java.util.function.Function; -import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @Slf4j public class ElasticsearchIT extends TestSuiteBase implements TestResource { - private static final long INDEX_REFRESH_MILL_DELAY = 5000L; - - private List testDataset1; - - private List testDataset2; + private List testDataset; private ElasticsearchContainer container; @@ -125,8 +114,7 @@ public void startUp() throws Exception { Optional.empty(), Optional.empty(), Optional.empty()); - testDataset1 = generateTestDataSet1(); - testDataset2 = generateTestDataSet2(); + testDataset = generateTestDataSet(); createIndexForResourceNull("st_index"); createIndexDocs(); createIndexWithFullType(); @@ -135,18 +123,15 @@ public void startUp() throws Exception { /** create a index,and bulk some documents */ private void createIndexDocs() { - createIndexDocsByName("st_index"); - } - - private void createIndexDocsByName(String indexName) { - createIndexDocsByName(indexName, testDataset1); - } - - private void createIndexDocsByName(String indexName, List testDataSet) { StringBuilder requestBody = new StringBuilder(); - String indexHeader = String.format("{\"index\":{\"_index\":\"%s\"}\n", indexName); - for (int i = 0; i < testDataSet.size(); i++) { - String row = testDataSet.get(i); + Map indexInner = new HashMap<>(); + indexInner.put("_index", "st"); + + Map> indexParam = new HashMap<>(); + indexParam.put("index", indexInner); + String indexHeader = "{\"index\":{\"_index\":\"st_index\"}\n"; + for (int i = 0; i < testDataset.size(); i++) { + String row = testDataset.get(i); requestBody.append(indexHeader); requestBody.append(row); requestBody.append("\n"); @@ -174,7 +159,7 @@ private void createIndexWithFullType() throws IOException, InterruptedException + "\n"); Assertions.assertFalse(response.isErrors(), response.getResponse()); // waiting index refresh - Thread.sleep(INDEX_REFRESH_MILL_DELAY); + Thread.sleep(2000L); Assertions.assertEquals( 2, esRestClient.getIndexDocsCount("st_index_full_type").get(0).getDocsCount()); } @@ -190,121 +175,16 @@ private void createIndexForResourceNull(String indexName) throws IOException { } @TestTemplate - public void testElasticsearchWithSchema(TestContainer container) + public void testElasticsearch(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = container.executeJob("/elasticsearch/elasticsearch_source_and_sink.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - List sinkData = readSinkDataWithSchema("st_index2"); + List sinkData = readSinkData("st_index2"); // for DSL is: {"range":{"c_int":{"gte":10,"lte":20}}} Assertions.assertIterableEquals(mapTestDatasetForDSL(), sinkData); } - @TestTemplate - @DisabledOnContainer( - value = {}, - type = {EngineType.FLINK}, - disabledReason = "Currently FLINK do not support multiple table read") - public void testElasticsSearchWithMultiSourceByFilter(TestContainer container) - throws InterruptedException, IOException { - // read read_filter_index1,read_filter_index2 - // write into read_filter_index1_copy,read_filter_index2_copy - createIndexDocsByName("read_filter_index1", testDataset1); - createIndexDocsByName("read_filter_index2", testDataset2); - - Container.ExecResult execResult = - container.executeJob( - "/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - - HashMap rangeParam = new HashMap<>(); - rangeParam.put("gte", 10); - rangeParam.put("lte", 20); - HashMap range1 = new HashMap<>(); - range1.put("c_int", rangeParam); - Map query1 = new HashMap<>(); - query1.put("range", range1); - - Map query2 = new HashMap<>(); - HashMap range2 = new HashMap<>(); - range2.put("c_int2", rangeParam); - query2.put("range", range2); - - LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(INDEX_REFRESH_MILL_DELAY)); - Set sinkData1 = - new HashSet<>( - getDocsWithTransformDate( - // read all field - Collections.emptyList(), - // read indexName - "read_filter_index1_copy", - // allowed c_null serialized if null - Lists.newArrayList("c_null"), - // query condition - query1, - // transformDate field:c_date - Lists.newArrayList("c_date"), - // order field - "c_int")); - - List index1Data = - mapTestDatasetForDSL( - // use testDataset1 - testDataset1, - // filter testDataset1 match sinkData1 - doc -> { - if (doc.has("c_int")) { - int cInt = doc.get("c_int").asInt(); - return cInt >= 10 && cInt <= 20; - } - return false; - }, - // mapping document all field to string - JsonNode::toString); - Assertions.assertEquals(sinkData1.size(), index1Data.size()); - index1Data.forEach(sinkData1::remove); - // data is completely consistent, and the size is zero after deletion - Assertions.assertEquals(0, sinkData1.size()); - - List index2Data = - mapTestDatasetForDSL( - testDataset2, - // use customer predicate filter data to match sinkData2 - doc -> { - if (doc.has("c_int2")) { - int cInt = doc.get("c_int2").asInt(); - return cInt >= 10 && cInt <= 20; - } - return false; - }, - // mapping doc to string,keep only three fields - doc -> { - Map map = new HashMap<>(); - map.put("c_int2", doc.get("c_int2")); - map.put("c_null2", doc.get("c_null2")); - map.put("c_date2", doc.get("c_date2")); - return JsonUtils.toJsonString(map); - }); - - LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(INDEX_REFRESH_MILL_DELAY)); - Set sinkData2 = - new HashSet<>( - getDocsWithTransformDate( - // read three fields from index - Lists.newArrayList("c_int2", "c_null2", "c_date2"), - "read_filter_index2_copy", - //// allowed c_null serialized if null - Lists.newArrayList("c_null2"), - query2, - // // transformDate field:c_date2 - Lists.newArrayList("c_date2"), - // order by c_int2 - "c_int2")); - Assertions.assertEquals(sinkData2.size(), index2Data.size()); - index2Data.forEach(sinkData2::remove); - Assertions.assertEquals(0, sinkData2.size()); - } - @DisabledOnContainer( value = {}, type = {EngineType.FLINK}, @@ -354,7 +234,7 @@ public void testElasticsearchWithFullType(TestContainer container) Container.ExecResult execResult = container.executeJob("/elasticsearch/elasticsearch_source_and_sink_full_type.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - Thread.sleep(INDEX_REFRESH_MILL_DELAY); + Thread.sleep(2000L); Assertions.assertEquals( 1, esRestClient.getIndexDocsCount("st_index_full_type_target").get(0).getDocsCount()); @@ -368,12 +248,12 @@ public void testElasticsearchWithoutSchema(TestContainer container) container.executeJob( "/elasticsearch/elasticsearch_source_without_schema_and_sink.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - List sinkData = readSinkDataWithOutSchema("st_index4"); + List sinkData = readSinkDataWithOutSchema(); // for DSL is: {"range":{"c_int":{"gte":10,"lte":20}}} Assertions.assertIterableEquals(mapTestDatasetForDSL(), sinkData); } - private List generateTestDataSet1() throws JsonProcessingException { + private List generateTestDataSet() throws JsonProcessingException { String[] fields = new String[] { "c_map", @@ -424,83 +304,17 @@ private List generateTestDataSet1() throws JsonProcessingException { return documents; } - private List generateTestDataSet2() throws JsonProcessingException { - String[] fields = - new String[] { - "c_map2", - "c_array2", - "c_string2", - "c_boolean2", - "c_tinyint2", - "c_smallint2", - "c_bigint2", - "c_float2", - "c_double2", - "c_decimal2", - "c_bytes2", - "c_int2", - "c_date2", - "c_timestamp2", - "c_null2" - }; - - List documents = new ArrayList<>(); - ObjectMapper objectMapper = new ObjectMapper(); - for (int i = 0; i < 100; i++) { - Map doc = new HashMap<>(); - Object[] values = - new Object[] { - Collections.singletonMap("key2", Short.parseShort(String.valueOf(i))), - new Byte[] { - Byte.parseByte("11"), Byte.parseByte("22"), Byte.parseByte("33") - }, - "string2", - Boolean.FALSE, - Byte.parseByte("2"), - Short.parseShort("2"), - Long.parseLong("2"), - Float.parseFloat("2.2"), - Double.parseDouble("2.2"), - BigDecimal.valueOf(22, 1), - "test2".getBytes(), - i, - LocalDate.now().toString(), - System.currentTimeMillis(), - // Null values are also a basic use case for testing - null - }; - for (int j = 0; j < fields.length; j++) { - doc.put(fields[j], values[j]); - } - documents.add(objectMapper.writeValueAsString(doc)); - } - return documents; - } - - private List readSinkDataWithOutSchema(String indexName) throws InterruptedException { + private List readSinkDataWithOutSchema() throws InterruptedException { Map> esFieldType = - esRestClient.getFieldTypeMapping(indexName, Lists.newArrayList()); - Thread.sleep(INDEX_REFRESH_MILL_DELAY); + esRestClient.getFieldTypeMapping("st_index4", Lists.newArrayList()); + Thread.sleep(2000); List source = new ArrayList<>(esFieldType.keySet()); - return getDocsWithTransformDate(source, indexName); + return getDocsWithTransformDate(source, "st_index4"); } - // Null values are also a basic use case for testing - // To ensure consistency in comparisons, we need to explicitly serialize null values. - private List readSinkDataWithOutSchema(String indexName, List nullAllowedFields) - throws InterruptedException { - Map> esFieldType = - esRestClient.getFieldTypeMapping(indexName, Lists.newArrayList()); - Thread.sleep(INDEX_REFRESH_MILL_DELAY); - List source = new ArrayList<>(esFieldType.keySet()); - return getDocsWithTransformDate(source, indexName, nullAllowedFields); - } - - // The timestamp type in Elasticsearch is incompatible with that in Seatunnel, - // and we need to handle the conversion here. - private List readSinkDataWithSchema(String index) throws InterruptedException { + private List readSinkData(String index) throws InterruptedException { // wait for index refresh - Thread.sleep(INDEX_REFRESH_MILL_DELAY); + Thread.sleep(2000); List source = Lists.newArrayList( "c_map", @@ -524,7 +338,7 @@ private List readSinkDataWithSchema(String index) throws InterruptedExce private List readMultiSinkData(String index, List source) throws InterruptedException { // wait for index refresh - Thread.sleep(INDEX_REFRESH_MILL_DELAY); + Thread.sleep(2000); Map query = new HashMap<>(); query.put("match_all", Maps.newHashMap()); @@ -580,19 +394,6 @@ private List getDocsWithTransformTimestamp(List source, String i } private List getDocsWithTransformDate(List source, String index) { - return getDocsWithTransformDate(source, index, Collections.emptyList()); - } - - /** - * use default query: c_int >= 10 and c_int <=20 - * - * @param source The field to be read - * @param index indexName - * @param nullAllowedFields If the value of the field is null, it will be serialized to 'null' - * @return serialized data as jsonString - */ - private List getDocsWithTransformDate( - List source, String index, List nullAllowedFields) { HashMap rangeParam = new HashMap<>(); rangeParam.put("gte", 10); rangeParam.put("lte", 20); @@ -608,11 +409,6 @@ private List getDocsWithTransformDate( x.remove("_index"); x.remove("_type"); x.remove("_id"); - for (String field : nullAllowedFields) { - if (!x.containsKey(field)) { - x.put(field, null); - } - } x.replace( "c_date", LocalDate.parse( @@ -631,75 +427,7 @@ private List getDocsWithTransformDate( return docs; } - /** - * use customer query read data - * - * @param source The field to be read - * @param index read index - * @param nullAllowedFields If the value of the field is null, it will be serialized to 'null' - * @param query dls query - * @param dateFields dateField will format with yyyy-MM-dd'T'HH:mm - * @param orderField how to oder data - * @return serialized data as jsonString - */ - private List getDocsWithTransformDate( - List source, - String index, - List nullAllowedFields, - Map query, - List dateFields, - String orderField) { - ScrollResult scrollResult = esRestClient.searchByScroll(index, source, query, "1m", 1000); - scrollResult - .getDocs() - .forEach( - x -> { - x.remove("_index"); - x.remove("_type"); - x.remove("_id"); - for (String field : nullAllowedFields) { - if (!x.containsKey(field)) { - x.put(field, null); - } - } - for (String dateField : dateFields) { - if (x.containsKey(dateField)) { - x.replace( - dateField, - LocalDate.parse( - x.get(dateField).toString(), - DateTimeFormatter.ofPattern( - "yyyy-MM-dd'T'HH:mm")) - .toString()); - } - } - }); - List docs = - scrollResult.getDocs().stream() - .sorted( - Comparator.comparingInt( - o -> Integer.parseInt(o.get(orderField).toString()))) - .map(JsonUtils::toJsonString) - .collect(Collectors.toList()); - return docs; - } - - /** - * default testDataset1 - * - * @return testDataset1 as jsonString array - */ private List mapTestDatasetForDSL() { - return mapTestDatasetForDSL(testDataset1); - } - - /** - * default query filter,c_int >=10 and c_int <= 20 - * - * @param testDataset testDataset - * @return c_int >=10 and c_int <= 20 filtered data - */ - private List mapTestDatasetForDSL(List testDataset) { return testDataset.stream() .map(JsonUtils::parseObject) .filter( @@ -714,25 +442,6 @@ private List mapTestDatasetForDSL(List testDataset) { .collect(Collectors.toList()); } - /** - * Use custom filtering criteria to query data - * - * @param testDataset testDataset - * @param predicate customer query filter - * @param mapStrFunc mapping doc to string - * @return filtered data - */ - private List mapTestDatasetForDSL( - List testDataset, - Predicate predicate, - Function mapStrFunc) { - return testDataset.stream() - .map(JsonUtils::parseObject) - .filter(predicate) - .map(mapStrFunc) - .collect(Collectors.toList()); - } - @AfterEach @Override public void tearDown() { @@ -779,7 +488,7 @@ public void testCatalog() throws InterruptedException, JsonProcessingException { requestBody.append("\n"); } esRestClient.bulk(requestBody.toString()); - Thread.sleep(INDEX_REFRESH_MILL_DELAY); // Wait for data to be indexed + Thread.sleep(2000); // Wait for data to be indexed // Verify data exists List sourceFields = Arrays.asList("field1", "field2"); @@ -791,7 +500,7 @@ public void testCatalog() throws InterruptedException, JsonProcessingException { // Truncate the table elasticSearchCatalog.truncateTable(tablePath, false); - Thread.sleep(INDEX_REFRESH_MILL_DELAY); // Wait for data to be indexed + Thread.sleep(2000); // Wait for data to be indexed // Verify data is deleted scrollResult = esRestClient.searchByScroll("st_index3", sourceFields, query, "1m", 100); @@ -817,18 +526,4 @@ private List generateTestData() throws JsonProcessingException { } return data; } - - /** - * elastic query all dsl - * - * @return elastic query all dsl - */ - private Map queryAll() { - // "query": { - // "match_all": {} - // } - Map matchAll = new HashMap<>(); - matchAll.put("match_all", new HashMap<>()); - return matchAll; - } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf deleted file mode 100644 index 9ac7d3743aa..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-elasticsearch-e2e/src/test/resources/elasticsearch/elasticsearch_multi_source_and_sink_by_filter.conf +++ /dev/null @@ -1,92 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - parallelism = 1 - job.mode = "BATCH" - #checkpoint.interval = 10000 -} - -source { - Elasticsearch { - hosts = ["https://elasticsearch:9200"] - username = "elastic" - password = "elasticsearch" - tls_verify_certificate = false - tls_verify_hostname = false - index_list = [ - { - index = "read_filter_index1" - query = {"range": {"c_int": {"gte": 10, "lte": 20}}} - source = [ - c_map, - c_array, - c_string, - c_boolean, - c_tinyint, - c_smallint, - c_bigint, - c_float, - c_double, - c_decimal, - c_bytes, - c_int, - c_date, - c_timestamp, - c_null - ] - array_column = { - c_array = "array" - } - } - { - index = "read_filter_index2" - query = {"range": {"c_int2": {"gte": 10, "lte": 20}}} - source = [ - c_int2, - c_null2, - c_date2 - ] - - } - - ] - - } -} - -transform { -} - -sink { - Elasticsearch { - hosts = ["https://elasticsearch:9200"] - username = "elastic" - password = "elasticsearch" - tls_verify_certificate = false - tls_verify_hostname = false - - index = "${table_name}_copy" - index_type = "st" - "schema_save_mode"="CREATE_SCHEMA_WHEN_NOT_EXIST" - "data_save_mode"="APPEND_DATA" - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java index 6c3f449d785..63789b0d281 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/s3/S3Utils.java @@ -81,4 +81,18 @@ public void close() { s3Client.shutdown(); } } + + public static void main(String[] args) { + S3Utils s3Utils = new S3Utils(); + s3Utils.uploadTestFiles( + "/Users/gaojun/workspace/seatunnel/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx", + "test/test/seatunnel/e2e.xlsx", + false); + + s3Utils.createDir("test/test1"); + s3Utils.uploadTestFiles( + "/Users/gaojun/workspace/seatunnel/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-s3-e2e/src/test/resources/excel/e2e.xlsx", + "test/test1/seatunnel/e2e.xlsx", + false); + } } 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 1957e1bd08e..50075629249 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 @@ -59,13 +59,8 @@ public class HbaseIT extends TestSuiteBase implements TestResource { private static final String TABLE_NAME = "seatunnel_test"; - private static final String ASSIGN_CF_TABLE_NAME = "assign_cf_table"; - private static final String MULTI_TABLE_ONE_NAME = "hbase_sink_1"; - - private static final String MULTI_TABLE_TWO_NAME = "hbase_sink_2"; - private static final String FAMILY_NAME = "info"; private Connection hbaseConnection; @@ -86,14 +81,9 @@ public void startUp() throws Exception { // Create table for hbase sink test log.info("initial"); hbaseCluster.createTable(TABLE_NAME, Arrays.asList(FAMILY_NAME)); - // Create table for hbase assign cf table sink test hbaseCluster.createTable(ASSIGN_CF_TABLE_NAME, Arrays.asList("cf1", "cf2")); table = TableName.valueOf(TABLE_NAME); tableAssign = TableName.valueOf(ASSIGN_CF_TABLE_NAME); - - // Create table for hbase multi-table sink test - hbaseCluster.createTable(MULTI_TABLE_ONE_NAME, Arrays.asList(FAMILY_NAME)); - hbaseCluster.createTable(MULTI_TABLE_TWO_NAME, Arrays.asList(FAMILY_NAME)); } @AfterAll @@ -107,11 +97,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()); - ArrayList results = readData(table); - Assertions.assertEquals(results.size(), 5); + fakeToHbase(container); Container.ExecResult sourceExecResult = container.executeJob("/hbase-to-assert.conf"); Assertions.assertEquals(0, sourceExecResult.getExitCode()); } @@ -265,25 +251,6 @@ public void testHbaseSinkAssignCfSink(TestContainer container) Assertions.assertEquals(cf2Count, 5); } - @DisabledOnContainer( - value = {}, - type = {EngineType.FLINK}, - disabledReason = "Currently FLINK does not support multiple table write") - public void testHbaseMultiTableSink(TestContainer container) - throws IOException, InterruptedException { - TableName multiTable1 = TableName.valueOf(MULTI_TABLE_ONE_NAME); - TableName multiTable2 = TableName.valueOf(MULTI_TABLE_TWO_NAME); - deleteData(multiTable1); - deleteData(multiTable2); - Container.ExecResult sinkExecResult = - container.executeJob("/fake-to-hbase-with-multipletable.conf"); - Assertions.assertEquals(0, sinkExecResult.getExitCode()); - ArrayList results = readData(multiTable1); - Assertions.assertEquals(results.size(), 1); - results = readData(multiTable2); - Assertions.assertEquals(results.size(), 1); - } - @TestTemplate public void testHbaseSourceWithBatchQuery(TestContainer container) throws IOException, InterruptedException { @@ -351,16 +318,4 @@ private int countData(TableName table) throws IOException { scanner.close(); return count; } - - public ArrayList readData(TableName table) throws IOException { - 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); - } - scanner.close(); - return results; - } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.conf deleted file mode 100644 index 8972bf13249..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/fake-to-hbase-with-multipletable.conf +++ /dev/null @@ -1,86 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - tables_configs = [ - { - schema = { - table = "hbase_sink_1" - fields { - name = STRING - c_string = STRING - c_double = DOUBLE - c_bigint = BIGINT - c_float = FLOAT - c_int = INT - c_smallint = SMALLINT - c_boolean = BOOLEAN - time = BIGINT - } - } - rows = [ - { - kind = INSERT - fields = ["label_1", "sink_1", 4.3, 200, 2.5, 2, 5, true, 1627529632356] - } - ] - }, - { - schema = { - table = "hbase_sink_2" - fields { - name = STRING - c_string = STRING - c_double = DOUBLE - c_bigint = BIGINT - c_float = FLOAT - c_int = INT - c_smallint = SMALLINT - c_boolean = BOOLEAN - } - } - rows = [ - { - kind = INSERT - fields = ["label_2", "sink_2", 4.3, 200, 2.5, 2, 5, true] - } - ] - } - ] - } -} - -sink { - Hbase { - zookeeper_quorum = "hbase_e2e:2181" - table = "${table_name}" - rowkey_column = ["name"] - family_name { - all_columns = info - } - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf deleted file mode 100644 index 6dc7530b4bd..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-hbase-e2e/src/test/resources/hbase-to-assert-with-multipletable.conf +++ /dev/null @@ -1,129 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -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"] - 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 = 11 - }, - { - rule_type = MIN_ROW - rule_value = 11 - } - ], - 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 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index 920b26e1793..7cd6be4fd95 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -31,10 +31,6 @@ import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.iris.IrisCatalog; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; -import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; -import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceTableConfig; -import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; -import org.apache.seatunnel.connectors.seatunnel.jdbc.utils.JdbcCatalogUtils; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; @@ -76,9 +72,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.Properties; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -99,8 +93,7 @@ public abstract class AbstractJdbcIT extends TestSuiteBase implements TestResour "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl() - + " --no-check-certificate"); + + driverUrl()); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; @@ -123,7 +116,7 @@ public abstract class AbstractJdbcIT extends TestSuiteBase implements TestResour protected URLClassLoader getUrlClassLoader() throws MalformedURLException { if (urlClassLoader == null) { urlClassLoader = - new InsecureURLClassLoader( + new URLClassLoader( new URL[] {new URL(driverUrl())}, AbstractJdbcIT.class.getClassLoader()); Thread.currentThread().setContextClassLoader(urlClassLoader); @@ -461,43 +454,6 @@ public void testCatalog() { } } - @Test - public void testCatalogWithCatalogUtils() throws SQLException, ClassNotFoundException { - if (StringUtils.isBlank(jdbcCase.getTablePathFullName())) { - return; - } - - List tablesConfig = new ArrayList<>(); - JdbcSourceTableConfig tableConfig = - JdbcSourceTableConfig.builder() - .query("SELECT * FROM " + jdbcCase.getSourceTable()) - .useSelectCount(false) - .build(); - tablesConfig.add(tableConfig); - Map tables = - JdbcCatalogUtils.getTables( - JdbcConnectionConfig.builder() - .url(jdbcCase.getJdbcUrl().replace(HOST, dbServer.getHost())) - .driverName(jdbcCase.getDriverClass()) - .username(jdbcCase.getUserName()) - .password(jdbcCase.getPassword()) - .build(), - tablesConfig); - Set tablePaths = tables.keySet(); - - tablePaths.forEach( - tablePath -> { - log.info( - "Expected: {} Actual: {}", - tablePath.getFullName(), - jdbcCase.getTablePathFullName()); - Assertions.assertTrue( - tablePath - .getFullName() - .equalsIgnoreCase(jdbcCase.getTablePathFullName())); - }); - } - protected Object[] toArrayResult(ResultSet resultSet, String[] fieldNames) throws SQLException, IOException { List result = new ArrayList<>(0); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java deleted file mode 100644 index fc8a169abd4..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/InsecureURLClassLoader.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.jdbc; - -import javax.net.ssl.HttpsURLConnection; -import javax.net.ssl.SSLContext; -import javax.net.ssl.TrustManager; -import javax.net.ssl.X509TrustManager; - -import java.net.MalformedURLException; -import java.net.URL; -import java.net.URLClassLoader; -import java.security.SecureRandom; -import java.security.cert.X509Certificate; - -public class InsecureURLClassLoader extends URLClassLoader { - public InsecureURLClassLoader(URL[] urls, ClassLoader parent) throws MalformedURLException { - super(urls, parent); - disableCertificateValidation(); - } - - private static void disableCertificateValidation() { - TrustManager[] trustAllCerts = - new TrustManager[] { - new X509TrustManager() { - public X509Certificate[] getAcceptedIssuers() { - return null; - } - - public void checkClientTrusted(X509Certificate[] certs, String authType) {} - - public void checkServerTrusted(X509Certificate[] certs, String authType) {} - } - }; - - try { - SSLContext sc = SSLContext.getInstance("SSL"); - sc.init(null, trustAllCerts, new SecureRandom()); - HttpsURLConnection.setDefaultSSLSocketFactory(sc.getSocketFactory()); - } catch (Exception e) { - e.printStackTrace(); - } - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java index 3dd7b64b95d..006d3d5f351 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcCase.java @@ -57,8 +57,4 @@ public class JdbcCase { private String catalogDatabase; private String catalogSchema; private String catalogTable; - - // The full path of the table created when initializing data - // According to whether jdbc api supports setting - private String tablePathFullName; } 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/JdbcMysqlIT.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/JdbcMysqlIT.java index 1b148ce494d..6bd97863b25 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/JdbcMysqlIT.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/JdbcMysqlIT.java @@ -180,7 +180,6 @@ JdbcCase getJdbcCase() { .testData(testDataSet) .catalogDatabase(CATALOG_DATABASE) .catalogTable(MYSQL_SINK) - .tablePathFullName(MYSQL_DATABASE + "." + MYSQL_SOURCE) .build(); } 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 19f7f118f28..b7c4a54b59b 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 @@ -25,15 +25,11 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.oracle.OracleDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; -import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.commons.lang3.tuple.Pair; -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.OracleContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -81,9 +77,6 @@ public class JdbcOracleIT extends AbstractJdbcIT { + " VARCHAR_10_COL varchar2(10),\n" + " CHAR_10_COL char(10),\n" + " CLOB_COL clob,\n" - + " NUMBER_1 number(1),\n" - + " NUMBER_6 number(6),\n" - + " NUMBER_10 number(10),\n" + " NUMBER_3_SF_2_DP number(3, 2),\n" + " NUMBER_7_SF_N2_DP number(7, -2),\n" + " INTEGER_COL integer,\n" @@ -104,9 +97,6 @@ public class JdbcOracleIT extends AbstractJdbcIT { + " VARCHAR_10_COL varchar2(10),\n" + " CHAR_10_COL char(10),\n" + " CLOB_COL clob,\n" - + " NUMBER_1 number(1),\n" - + " NUMBER_6 number(6),\n" - + " NUMBER_10 number(10),\n" + " NUMBER_3_SF_2_DP number(3, 2),\n" + " NUMBER_7_SF_N2_DP number(7, -2),\n" + " INTEGER_COL integer,\n" @@ -125,9 +115,6 @@ public class JdbcOracleIT extends AbstractJdbcIT { "VARCHAR_10_COL", "CHAR_10_COL", "CLOB_COL", - "NUMBER_1", - "NUMBER_6", - "NUMBER_10", "NUMBER_3_SF_2_DP", "NUMBER_7_SF_N2_DP", "INTEGER_COL", @@ -161,14 +148,6 @@ public void testSampleDataFromColumnSuccess() throws Exception { dialect.sampleDataFromColumn(connection, table, "INTEGER_COL", 1, 1024); } - @TestTemplate - public void testOracleWithoutDecimalTypeNarrowing(TestContainer container) throws Exception { - Container.ExecResult execResult = - container.executeJob( - "/jdbc_oracle_source_to_sink_without_decimal_type_narrowing.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } - @Override JdbcCase getJdbcCase() { Map containerEnv = new HashMap<>(); @@ -205,8 +184,6 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) - // oracle jdbc not support getTables/getCatalog/getSchema , is empty - .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } @@ -230,9 +207,6 @@ Pair> initTestData() { String.format("f%s", i), String.format("f%s", i), String.format("f%s", i), - 1, - i * 10, - i * 1000, BigDecimal.valueOf(1.1), BigDecimal.valueOf(2400), i, 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/JdbcOracleMultipleTablesIT.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/JdbcOracleMultipleTablesIT.java deleted file mode 100644 index 2b0bd05d5aa..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcOracleMultipleTablesIT.java +++ /dev/null @@ -1,307 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.jdbc; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -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.apache.commons.lang3.tuple.Pair; - -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.junit.jupiter.api.function.Executable; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.OracleContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.images.PullPolicy; -import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.DockerImageName; -import org.testcontainers.utility.DockerLoggerFactory; -import org.testcontainers.utility.MountableFile; - -import com.google.common.collect.Lists; -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.math.BigDecimal; -import java.sql.Connection; -import java.sql.Date; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -@Slf4j -public class JdbcOracleMultipleTablesIT extends TestSuiteBase implements TestResource { - private static final String ORACLE_IMAGE = "gvenzl/oracle-xe:21-slim-faststart"; - private static final String ORACLE_NETWORK_ALIASES = "e2e_oracleDb"; - private static final int ORACLE_PORT = 1521; - private static final String USERNAME = "TESTUSER"; - private static final String PASSWORD = "testPassword"; - private static final String DATABASE = "XE"; - private static final String SCHEMA = USERNAME; - private static final Pair> TEST_DATASET = generateTestDataset(); - private static final List TABLES = Arrays.asList("TABLE1", "TABLE2"); - private static final List SOURCE_TABLES = - TABLES.stream().map(table -> SCHEMA + "." + table).collect(Collectors.toList()); - - private static final List SINK_TABLES = - TABLES.stream() - .map(table -> SCHEMA + "." + "SINK_" + table) - .collect(Collectors.toList()); - private static final String CREATE_TABLE_SQL = - "create table %s\n" - + "(\n" - + " VARCHAR_10_COL varchar2(10),\n" - + " CHAR_10_COL char(10),\n" - + " CLOB_COL clob,\n" - + " NUMBER_1 number(1),\n" - + " NUMBER_6 number(6),\n" - + " NUMBER_10 number(10),\n" - + " NUMBER_3_SF_2_DP number(3, 2),\n" - + " NUMBER_7_SF_N2_DP number(7, -2),\n" - + " INTEGER_COL integer,\n" - + " FLOAT_COL float(10),\n" - + " REAL_COL real,\n" - + " BINARY_FLOAT_COL binary_float,\n" - + " BINARY_DOUBLE_COL binary_double,\n" - + " DATE_COL date,\n" - + " TIMESTAMP_WITH_3_FRAC_SEC_COL timestamp(3),\n" - + " TIMESTAMP_WITH_LOCAL_TZ timestamp with local time zone,\n" - + " XML_TYPE_COL \"SYS\".\"XMLTYPE\"" - + ")"; - - private OracleContainer oracleContainer; - private Connection connection; - - @TestContainerExtension - protected final ContainerExtendedFactory extendedFactory = - container -> { - Container.ExecResult extraCommands = - container.execInContainer( - "bash", - "-c", - "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + "https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/12.2.0.1/ojdbc8-12.2.0.1.jar && wget https://repo1.maven.org/maven2/com/oracle/database/xml/xdb6/12.2.0.1/xdb6-12.2.0.1.jar && wget https://repo1.maven.org/maven2/com/oracle/database/xml/xmlparserv2/12.2.0.1/xmlparserv2-12.2.0.1.jar"); - Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); - }; - - @BeforeAll - @Override - public void startUp() throws Exception { - DockerImageName imageName = DockerImageName.parse(ORACLE_IMAGE); - oracleContainer = - new OracleContainer(imageName) - .withUsername(USERNAME) - .withPassword(PASSWORD) - .withDatabaseName(SCHEMA) - .withCopyFileToContainer( - MountableFile.forClasspathResource("sql/oracle_init.sql"), - "/container-entrypoint-startdb.d/init.sql") - .withNetwork(NETWORK) - .withNetworkAliases(ORACLE_NETWORK_ALIASES) - .withExposedPorts(ORACLE_PORT) - .withImagePullPolicy((PullPolicy.alwaysPull())) - .withLogConsumer( - new Slf4jLogConsumer(DockerLoggerFactory.getLogger(ORACLE_IMAGE))); - - oracleContainer.setPortBindings( - Lists.newArrayList(String.format("%s:%s", ORACLE_PORT, ORACLE_PORT))); - - Startables.deepStart(Stream.of(oracleContainer)).join(); - - connection = oracleContainer.createConnection(""); - createTables(SOURCE_TABLES); - createTables(SINK_TABLES); - initSourceTablesData(); - } - - @DisabledOnContainer( - value = {}, - type = {EngineType.FLINK}, - disabledReason = "Currently FLINK do not support multiple tables") - @TestTemplate - public void testMysqlJdbcMultipleTableE2e(TestContainer container) - throws IOException, InterruptedException, SQLException { - clearSinkTables(); - - Container.ExecResult execResult = - container.executeJob("/jdbc_oracle_source_with_multiple_tables_to_sink.conf"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - - List asserts = - TABLES.stream() - .map( - (Function) - table -> - () -> - Assertions.assertIterableEquals( - query( - String.format( - "SELECT * FROM %s.%s order by INTEGER_COL asc", - SCHEMA, table)), - query( - String.format( - "SELECT * FROM %s.%s order by INTEGER_COL asc", - SCHEMA, - "SINK_" + table)))) - .collect(Collectors.toList()); - Assertions.assertAll(asserts); - } - - @AfterAll - @Override - public void tearDown() throws Exception { - if (connection != null) { - connection.close(); - } - if (oracleContainer != null) { - oracleContainer.close(); - } - } - - private void createTables(List tables) throws SQLException { - try (Statement statement = connection.createStatement()) { - tables.forEach( - tableName -> { - try { - statement.execute(String.format(CREATE_TABLE_SQL, tableName)); - } catch (SQLException e) { - throw new RuntimeException(e); - } - }); - } - } - - private void initSourceTablesData() throws SQLException { - String columns = Arrays.stream(TEST_DATASET.getLeft()).collect(Collectors.joining(", ")); - String placeholders = - Arrays.stream(TEST_DATASET.getLeft()) - .map(f -> "?") - .collect(Collectors.joining(", ")); - for (String table : SOURCE_TABLES) { - String sql = - "INSERT INTO " + table + " (" + columns + " ) VALUES (" + placeholders + ")"; - try (PreparedStatement statement = connection.prepareStatement(sql)) { - for (SeaTunnelRow row : TEST_DATASET.getRight()) { - for (int i = 0; i < row.getArity(); i++) { - statement.setObject(i + 1, row.getField(i)); - } - statement.addBatch(); - } - statement.executeBatch(); - } - } - } - - private List> query(String sql) { - try (Statement statement = connection.createStatement(); - ResultSet resultSet = statement.executeQuery(sql)) { - List> result = new ArrayList<>(); - int columnCount = resultSet.getMetaData().getColumnCount(); - while (resultSet.next()) { - ArrayList objects = new ArrayList<>(); - for (int i = 1; i <= columnCount; i++) { - objects.add(resultSet.getString(i)); - } - result.add(objects); - log.debug(String.format("Print query, sql: %s, data: %s", sql, objects)); - } - return result; - } catch (SQLException e) { - throw new RuntimeException(e); - } - } - - private void clearSinkTables() throws SQLException { - for (String table : SINK_TABLES) { - String sql = "truncate table " + table; - try (Statement statement = connection.createStatement()) { - statement.execute(sql); - } - } - } - - private static Pair> generateTestDataset() { - String[] fieldNames = - new String[] { - "VARCHAR_10_COL", - "CHAR_10_COL", - "CLOB_COL", - "NUMBER_1", - "NUMBER_6", - "NUMBER_10", - "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" - }; - List rows = new ArrayList<>(); - for (int i = 0; i < 2000; i++) { - SeaTunnelRow row = - new SeaTunnelRow( - new Object[] { - String.format("f%s", i), - String.format("f%s", i), - String.format("f%s", i), - 1, - i * 10, - i * 1000, - BigDecimal.valueOf(1.1), - BigDecimal.valueOf(2400), - i, - Float.parseFloat("2.2"), - Float.parseFloat("2.2"), - Float.parseFloat("22.2"), - Double.parseDouble("2.2"), - Date.valueOf(LocalDate.now()), - Timestamp.valueOf(LocalDateTime.now()), - Timestamp.valueOf(LocalDateTime.now()), - "SeaTunnel : E2E : Connector V2 : Oracle XMLType" - }); - rows.add(row); - } - - return Pair.of(fieldNames, rows); - } -} 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.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink.conf index 4df8c7b9934..d956894c340 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink.conf @@ -30,7 +30,7 @@ source { url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" user = testUser password = testPassword - query = "SELECT VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" + 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" } @@ -46,7 +46,7 @@ sink { 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_1,NUMBER_6,NUMBER_10,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(?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + 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" } 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 index 1988b488721..8a0c8310443 100644 --- 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 @@ -31,7 +31,7 @@ source { user = testUser password = testPassword use_select_count = true - query = "SELECT VARCHAR_10_COL,CHAR_10_COL,CLOB_COL,NUMBER_1,NUMBER_6,NUMBER_10,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" + 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" } @@ -47,7 +47,7 @@ sink { 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_1,NUMBER_6,NUMBER_10,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(?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + 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" } 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 index 4d01da5c72a..ebebdb55051 100644 --- 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 @@ -47,7 +47,7 @@ sink { 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_1,NUMBER_6,NUMBER_10,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(?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + 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" } 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 index 94a850fdd01..d00ce9b6434 100644 --- 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 @@ -48,7 +48,7 @@ sink { 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_1,NUMBER_6,NUMBER_10,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(?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?)" + 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" } 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_without_decimal_type_narrowing.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_without_decimal_type_narrowing.conf deleted file mode 100644 index 58e98f5deff..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_to_sink_without_decimal_type_narrowing.conf +++ /dev/null @@ -1,82 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - Jdbc { - driver = oracle.jdbc.driver.OracleDriver - url = "jdbc:oracle:thin:@e2e_oracleDb:1521/TESTUSER" - user = testUser - password = testPassword - decimal_type_narrowing = false - query = "SELECT NUMBER_1,NUMBER_6,NUMBER_10 FROM E2E_TABLE_SOURCE" - properties { - database.oracle.jdbc.timezoneAsRegion = "false" - } - } -} - -sink { - Assert { - rules { - row_rules = [ - { - rule_type = MAX_ROW - rule_value = 20000 - }, - { - rule_type = MIN_ROW - rule_value = 20000 - } - ], - field_rules = [ - { - field_name = NUMBER_1 - field_type = "decimal(1, 0)" - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = NUMBER_6 - field_type = "decimal(6, 0)" - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = NUMBER_10 - field_type = "decimal(10, 0)" - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } - -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf deleted file mode 100644 index 22c66493134..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-1/src/test/resources/jdbc_oracle_source_with_multiple_tables_to_sink.conf +++ /dev/null @@ -1,65 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### 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_list = [ - { - table_path = "TESTUSER.TABLE1" - }, - { - table_path = "TESTUSER.TABLE2" - } - ] - 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 - database = XE - table = "TESTUSER.SINK_${table_name}" - generate_sink_sql = true - properties { - database.oracle.jdbc.timezoneAsRegion = "false" - } - } - -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java index e34557b7394..1d41c480c34 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcStarRocksdbIT.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.commons.lang3.tuple.Pair; @@ -106,7 +105,6 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) - .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index dfa128f8134..f183c0c1932 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.ExceptionUtils; @@ -44,7 +43,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { private static final String HIVE_DATABASE = "default"; - private static final String HIVE_SOURCE = "hive_e2e_source_table"; + private static final String HIVE_SOURCE = "e2e_table_source"; private static final String HIVE_USERNAME = "root"; private static final String HIVE_PASSWORD = null; private static final int HIVE_PORT = 10000; @@ -95,7 +94,6 @@ JdbcCase getJdbcCase() { .sourceTable(HIVE_SOURCE) .createSql(CREATE_SQL) .configFile(CONFIG_FILE) - .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java index d36b864215c..de5621899a4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerIT.java @@ -180,7 +180,6 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) - .tablePathFullName(TablePath.DEFAULT.getFullName()) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java index cc6af23d6ef..f2b9097ffa7 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmIT.java @@ -127,7 +127,6 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) - .tablePathFullName(String.format("%s.%s", DM_DATABASE, DM_SOURCE)) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java index c90d91693f4..388cf67ae9c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGBase8aIT.java @@ -114,7 +114,7 @@ void compareResult(String executeKey) {} @Override String driverUrl() { - return "https://cdn.gbase.cn/products/30/p5CiVwXBKQYIUGN8ecHvk/gbase-connector-java-9.5.0.7-build1-bin.jar"; + return "https://www.gbase8.cn/wp-content/uploads/2020/10/gbase-connector-java-8.3.81.53-build55.5.7-bin_min_mix.jar"; } @Override diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java index 1ea4e2b693f..9c98c29a7a7 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-5/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcGreenplumIT.java @@ -84,7 +84,6 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) - .tablePathFullName(GREENPLUM_SOURCE) .build(); } 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/JdbcXuguIT.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/JdbcXuguIT.java index 186e23530a0..5fdae0ad939 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/JdbcXuguIT.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/JdbcXuguIT.java @@ -152,7 +152,6 @@ JdbcCase getJdbcCase() { .configFile(CONFIG_FILE) .insertSql(insertSql) .testData(testDataSet) - .tablePathFullName(XUGU_DATABASE + "." + XUGU_SCHEMA + "." + XUGU_SOURCE) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml index 668747b9db1..fa2e1930cce 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/pom.xml @@ -43,12 +43,6 @@ connector-cdc-mysql ${project.version} test - - - org.apache.kafka - kafka-clients - - org.apache.seatunnel 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 ffc97f4dd33..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 @@ -19,15 +19,9 @@ 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.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.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.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; @@ -46,7 +40,6 @@ 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.protobuf.ProtobufDeserializationSchema; import org.apache.seatunnel.format.text.TextSerializationSchema; import org.apache.kafka.clients.admin.AdminClient; @@ -78,13 +71,8 @@ import lombok.extern.slf4j.Slf4j; -import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.math.BigDecimal; -import java.net.URISyntaxException; -import java.net.URL; -import java.nio.file.Paths; import java.time.Duration; import java.time.LocalDate; import java.time.LocalDateTime; @@ -97,7 +85,6 @@ import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.stream.IntStream; import java.util.stream.Stream; @Slf4j @@ -139,8 +126,7 @@ public void startUp() throws Exception { "test_topic_source", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER, - null); + DEFAULT_FIELD_DELIMITER); generateTestData(serializer::serializeRow, 0, 100); } @@ -273,8 +259,7 @@ public void testSourceKafkaJsonToConsole(TestContainer container) "test_topic_json", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER, - null); + DEFAULT_FIELD_DELIMITER); generateTestData(row -> serializer.serializeRow(row), 0, 100); Container.ExecResult execResult = container.executeJob("/jsonFormatIT/kafka_source_json_to_console.conf"); @@ -289,8 +274,7 @@ public void testSourceKafkaJsonFormatErrorHandleWaySkipToConsole(TestContainer c "test_topic_error_message", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER, - null); + DEFAULT_FIELD_DELIMITER); generateTestData(row -> serializer.serializeRow(row), 0, 100); Container.ExecResult execResult = container.executeJob( @@ -306,8 +290,7 @@ public void testSourceKafkaJsonFormatErrorHandleWayFailToConsole(TestContainer c "test_topic_error_message", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER, - null); + DEFAULT_FIELD_DELIMITER); generateTestData(row -> serializer.serializeRow(row), 0, 100); Container.ExecResult execResult = container.executeJob( @@ -331,8 +314,7 @@ public void testSourceKafkaStartConfig(TestContainer container) "test_topic_group", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER, - null); + DEFAULT_FIELD_DELIMITER); generateTestData(row -> serializer.serializeRow(row), 100, 150); testKafkaGroupOffsetsToConsole(container); } @@ -349,8 +331,7 @@ public void testSourceKafkaStartConfigWithCommitOffset(TestContainer container) "test_topic_group_with_commit_offset", SEATUNNEL_ROW_TYPE, DEFAULT_FORMAT, - DEFAULT_FIELD_DELIMITER, - null); + DEFAULT_FIELD_DELIMITER); generateTestData(row -> serializer.serializeRow(row), 0, 100); testKafkaGroupOffsetsToConsoleWithCommitOffset(container); } @@ -475,8 +456,7 @@ public void testKafkaAvroToAssert(TestContainer container) "test_avro_topic", SEATUNNEL_ROW_TYPE, MessageFormat.AVRO, - DEFAULT_FIELD_DELIMITER, - null); + DEFAULT_FIELD_DELIMITER); int start = 0; int end = 100; generateTestData(row -> serializer.serializeRow(row), start, end); @@ -524,260 +504,6 @@ public void testKafkaAvroToAssert(TestContainer container) }); } - @TestTemplate - public void testFakeSourceToKafkaProtobufFormat(TestContainer container) - throws IOException, InterruptedException, URISyntaxException { - - // Execute the job and verify the exit code - Container.ExecResult execResult = - container.executeJob("/protobuf/fake_to_kafka_protobuf.conf"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - - // Define the SeaTunnelRowType for the address field - SeaTunnelRowType addressType = - new SeaTunnelRowType( - new String[] {"city", "state", "street"}, - new SeaTunnelDataType[] { - BasicType.STRING_TYPE, BasicType.STRING_TYPE, BasicType.STRING_TYPE - }); - - // Define the SeaTunnelRowType for the main schema - SeaTunnelRowType seaTunnelRowType = - new SeaTunnelRowType( - new String[] { - "c_int32", - "c_int64", - "c_float", - "c_double", - "c_bool", - "c_string", - "c_bytes", - "Address", - "attributes", - "phone_numbers" - }, - new SeaTunnelDataType[] { - BasicType.INT_TYPE, - BasicType.LONG_TYPE, - BasicType.FLOAT_TYPE, - BasicType.DOUBLE_TYPE, - BasicType.BOOLEAN_TYPE, - BasicType.STRING_TYPE, - PrimitiveByteArrayType.INSTANCE, - addressType, - new MapType<>(BasicType.STRING_TYPE, BasicType.FLOAT_TYPE), - ArrayType.STRING_ARRAY_TYPE - }); - - // Parse the configuration file - String path = getTestConfigFile("/protobuf/fake_to_kafka_protobuf.conf"); - Config config = ConfigFactory.parseFile(new File(path)); - Config sinkConfig = config.getConfigList("sink").get(0); - - // Prepare the schema properties - Map schemaProperties = new HashMap<>(); - schemaProperties.put( - "protobuf_message_name", sinkConfig.getString("protobuf_message_name")); - schemaProperties.put("protobuf_schema", sinkConfig.getString("protobuf_schema")); - - // Build the table schema based on SeaTunnelRowType - TableSchema schema = - TableSchema.builder() - .columns( - Arrays.asList( - IntStream.range(0, seaTunnelRowType.getTotalFields()) - .mapToObj( - i -> - PhysicalColumn.of( - seaTunnelRowType - .getFieldName(i), - seaTunnelRowType - .getFieldType(i), - 0, - true, - null, - null)) - .toArray(PhysicalColumn[]::new))) - .build(); - - // Create the catalog table - CatalogTable catalogTable = - CatalogTable.of( - TableIdentifier.of("", "", "", "test"), - schema, - schemaProperties, - Collections.emptyList(), - "It is converted from RowType and only has column information."); - - // Initialize the Protobuf deserialization schema - ProtobufDeserializationSchema deserializationSchema = - new ProtobufDeserializationSchema(catalogTable); - - // Retrieve and verify Kafka rows - List kafkaRows = - getKafkaSTRow( - "test_protobuf_topic_fake_source", - value -> { - try { - return deserializationSchema.deserialize(value); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - - Assertions.assertEquals(16, kafkaRows.size()); - - // Validate the contents of each row - kafkaRows.forEach( - row -> { - Assertions.assertInstanceOf(Integer.class, row.getField(0)); - Assertions.assertInstanceOf(Long.class, row.getField(1)); - Assertions.assertInstanceOf(Float.class, row.getField(2)); - Assertions.assertInstanceOf(Double.class, row.getField(3)); - Assertions.assertInstanceOf(Boolean.class, row.getField(4)); - Assertions.assertInstanceOf(String.class, row.getField(5)); - Assertions.assertInstanceOf(byte[].class, row.getField(6)); - Assertions.assertInstanceOf(SeaTunnelRow.class, row.getField(7)); - Assertions.assertInstanceOf(Map.class, row.getField(8)); - Assertions.assertInstanceOf(String[].class, row.getField(9)); - }); - } - - @TestTemplate - public void testKafkaProtobufToAssert(TestContainer container) - throws IOException, InterruptedException, URISyntaxException { - - String confFile = "/protobuf/kafka_protobuf_to_assert.conf"; - String path = getTestConfigFile(confFile); - Config config = ConfigFactory.parseFile(new File(path)); - Config sinkConfig = config.getConfigList("source").get(0); - ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(sinkConfig); - SeaTunnelRowType seaTunnelRowType = buildSeaTunnelRowType(); - - // Prepare schema properties - Map schemaProperties = new HashMap<>(); - schemaProperties.put( - "protobuf_message_name", sinkConfig.getString("protobuf_message_name")); - schemaProperties.put("protobuf_schema", sinkConfig.getString("protobuf_schema")); - - // Build the table schema - TableSchema schema = - TableSchema.builder() - .columns( - Arrays.asList( - IntStream.range(0, seaTunnelRowType.getTotalFields()) - .mapToObj( - i -> - PhysicalColumn.of( - seaTunnelRowType - .getFieldName(i), - seaTunnelRowType - .getFieldType(i), - 0, - true, - null, - null)) - .toArray(PhysicalColumn[]::new))) - .build(); - - // Create catalog table - CatalogTable catalogTable = - CatalogTable.of( - TableIdentifier.of("", "", "", "test"), - schema, - schemaProperties, - Collections.emptyList(), - "It is converted from RowType and only has column information."); - - // Initialize the Protobuf deserialization schema - ProtobufDeserializationSchema deserializationSchema = - new ProtobufDeserializationSchema(catalogTable); - - // Create serializer - DefaultSeaTunnelRowSerializer serializer = - DefaultSeaTunnelRowSerializer.create( - "test_protobuf_topic_fake_source", - seaTunnelRowType, - MessageFormat.PROTOBUF, - DEFAULT_FIELD_DELIMITER, - readonlyConfig); - - // Produce records to Kafka - IntStream.range(0, 20) - .forEach( - i -> { - try { - SeaTunnelRow originalRow = buildSeaTunnelRow(); - ProducerRecord producerRecord = - serializer.serializeRow(originalRow); - producer.send(producerRecord).get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Error sending Kafka message", e); - } - }); - - producer.flush(); - - // Execute the job and validate - Container.ExecResult execResult = container.executeJob(confFile); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); - - // Retrieve and verify Kafka rows - List kafkaSTRow = - getKafkaSTRow( - "test_protobuf_topic_fake_source", - value -> { - try { - return deserializationSchema.deserialize(value); - } catch (IOException e) { - throw new RuntimeException("Error deserializing Kafka message", e); - } - }); - - // Prepare expected values for assertions - SeaTunnelRow expectedAddress = new SeaTunnelRow(3); - expectedAddress.setField(0, "city_value"); - expectedAddress.setField(1, "state_value"); - expectedAddress.setField(2, "street_value"); - - Map expectedAttributesMap = new HashMap<>(); - expectedAttributesMap.put("k1", 0.1F); - expectedAttributesMap.put("k2", 2.3F); - - String[] expectedPhoneNumbers = {"1", "2"}; - - // Assertions - Assertions.assertEquals(20, kafkaSTRow.size()); - kafkaSTRow.forEach( - row -> { - Assertions.assertAll( - "Verify row fields", - () -> Assertions.assertEquals(123, (int) row.getField(0)), - () -> Assertions.assertEquals(123123123123L, (long) row.getField(1)), - () -> Assertions.assertEquals(0.123f, (float) row.getField(2)), - () -> Assertions.assertEquals(0.123d, (double) row.getField(3)), - () -> Assertions.assertFalse((boolean) row.getField(4)), - () -> Assertions.assertEquals("test data", row.getField(5).toString()), - () -> - Assertions.assertArrayEquals( - new byte[] {1, 2, 3}, (byte[]) row.getField(6)), - () -> Assertions.assertEquals(expectedAddress, row.getField(7)), - () -> Assertions.assertEquals(expectedAttributesMap, row.getField(8)), - () -> - Assertions.assertArrayEquals( - expectedPhoneNumbers, (String[]) row.getField(9))); - }); - } - - public static String getTestConfigFile(String configFile) - throws FileNotFoundException, URISyntaxException { - URL resource = KafkaIT.class.getResource(configFile); - if (resource == null) { - throw new FileNotFoundException("Can't find config file: " + configFile); - } - return Paths.get(resource.toURI()).toString(); - } - public void testKafkaLatestToConsole(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = @@ -1033,68 +759,4 @@ interface ProducerRecordConverter { interface ConsumerRecordConverter { SeaTunnelRow convert(byte[] value); } - - private SeaTunnelRow buildSeaTunnelRow() { - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(10); - - Map attributesMap = new HashMap<>(); - attributesMap.put("k1", 0.1F); - attributesMap.put("k2", 2.3F); - - String[] phoneNumbers = {"1", "2"}; - byte[] byteVal = {1, 2, 3}; - - SeaTunnelRow address = new SeaTunnelRow(3); - address.setField(0, "city_value"); - address.setField(1, "state_value"); - address.setField(2, "street_value"); - - seaTunnelRow.setField(0, 123); - seaTunnelRow.setField(1, 123123123123L); - seaTunnelRow.setField(2, 0.123f); - seaTunnelRow.setField(3, 0.123d); - seaTunnelRow.setField(4, false); - seaTunnelRow.setField(5, "test data"); - seaTunnelRow.setField(6, byteVal); - seaTunnelRow.setField(7, address); - seaTunnelRow.setField(8, attributesMap); - seaTunnelRow.setField(9, phoneNumbers); - - return seaTunnelRow; - } - - private SeaTunnelRowType buildSeaTunnelRowType() { - SeaTunnelRowType addressType = - new SeaTunnelRowType( - new String[] {"city", "state", "street"}, - new SeaTunnelDataType[] { - BasicType.STRING_TYPE, BasicType.STRING_TYPE, BasicType.STRING_TYPE - }); - - return new SeaTunnelRowType( - new String[] { - "c_int32", - "c_int64", - "c_float", - "c_double", - "c_bool", - "c_string", - "c_bytes", - "Address", - "attributes", - "phone_numbers" - }, - new SeaTunnelDataType[] { - BasicType.INT_TYPE, - BasicType.LONG_TYPE, - BasicType.FLOAT_TYPE, - BasicType.DOUBLE_TYPE, - BasicType.BOOLEAN_TYPE, - BasicType.STRING_TYPE, - PrimitiveByteArrayType.INSTANCE, - addressType, - new MapType<>(BasicType.STRING_TYPE, BasicType.FLOAT_TYPE), - ArrayType.STRING_ARRAY_TYPE - }); - } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf index 6ef14eed66c..8bc6d41cd3f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/multiFormatIT/kafka_multi_source_to_pg.conf @@ -26,11 +26,11 @@ env { source { Kafka { bootstrap.servers = "kafka_e2e:9092" - consumer.group = "ogg_multi_group" table_list = [ { topic = "^test-ogg-sou.*" pattern = "true" + consumer.group = "ogg_multi_group" start_mode = earliest schema = { fields { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf deleted file mode 100644 index 29a9985629b..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/fake_to_kafka_protobuf.conf +++ /dev/null @@ -1,100 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" - - # spark config - spark.executor.instances = 1 - spark.executor.cores = 1 - spark.executor.memory = "1g" - spark.master = local - -} -source { - FakeSource { - parallelism = 1 - result_table_name = "fake" - row.num = 16 - schema = { - fields { - c_int32 = int - c_int64 = long - c_float = float - c_double = double - c_bool = boolean - c_string = string - c_bytes = bytes - - Address { - city = string - state = string - street = string - } - attributes = "map" - phone_numbers = "array" - } - } - } -} - -sink { - kafka { - topic = "test_protobuf_topic_fake_source" - bootstrap.servers = "kafkaCluster:9092" - format = protobuf - kafka.request.timeout.ms = 60000 -# semantics = EXACTLY_ONCE - kafka.config = { - acks = "all" - request.timeout.ms = 60000 - buffer.memory = 33554432 - } - protobuf_message_name = Person - protobuf_schema = """ - syntax = "proto3"; - - package org.apache.seatunnel.format.protobuf; - - option java_outer_classname = "ProtobufE2E"; - - message Person { - int32 c_int32 = 1; - int64 c_int64 = 2; - float c_float = 3; - double c_double = 4; - bool c_bool = 5; - string c_string = 6; - bytes c_bytes = 7; - - message Address { - string street = 1; - string city = 2; - string state = 3; - string zip = 4; - } - - Address address = 8; - - map attributes = 9; - - repeated string phone_numbers = 10; - } - """ - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf deleted file mode 100644 index 3375cd11d77..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/protobuf/kafka_protobuf_to_assert.conf +++ /dev/null @@ -1,177 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" - spark.app.name = "SeaTunnel" - spark.executor.instances = 1 - spark.executor.cores = 1 - spark.executor.memory = "1g" - spark.master = local -} - -source { - Kafka { - topic = "test_protobuf_topic_fake_source" - format = protobuf - protobuf_message_name = Person - protobuf_schema = """ - syntax = "proto3"; - - package org.apache.seatunnel.format.protobuf; - - option java_outer_classname = "ProtobufE2E"; - - message Person { - int32 c_int32 = 1; - int64 c_int64 = 2; - float c_float = 3; - double c_double = 4; - bool c_bool = 5; - string c_string = 6; - bytes c_bytes = 7; - - message Address { - string street = 1; - string city = 2; - string state = 3; - string zip = 4; - } - - Address address = 8; - - map attributes = 9; - - repeated string phone_numbers = 10; - } - """ - schema = { - fields { - c_int32 = int - c_int64 = long - c_float = float - c_double = double - c_bool = boolean - c_string = string - c_bytes = bytes - - Address { - city = string - state = string - street = string - } - attributes = "map" - phone_numbers = "array" - } - } - bootstrap.servers = "kafkaCluster:9092" - start_mode = "earliest" - result_table_name = "kafka_table" - } -} - -sink { - Assert { - source_table_name = "kafka_table" - rules = { - field_rules = [ - { - field_name = c_int32 - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = c_int64 - field_type = long - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = c_float - field_type = float - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = c_double - field_type = double - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = c_bool - field_type = boolean - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = c_string - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = c_bytes - field_type = bytes - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = attributes - field_type = "map" - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = phone_numbers - field_type = array - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java index 700bf25f510..13dcd3d675e 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonRecord.java @@ -32,7 +32,6 @@ public class PaimonRecord { public Long pkId; public String name; - public Integer score; public String dt; public Timestamp oneTime; public Timestamp twoTime; 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 0168cc8f534..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 @@ -508,43 +508,6 @@ public void testFakeSinkPaimonWithFullTypeAndReadWithFilter(TestContainer contai Assertions.assertEquals(0, readResult4.getExitCode()); } - @TestTemplate - public void testSinkPaimonTruncateTable(TestContainer container) throws Exception { - Container.ExecResult writeResult = - container.executeJob("/fake_sink_paimon_truncate_with_local_case1.conf"); - Assertions.assertEquals(0, writeResult.getExitCode()); - Container.ExecResult readResult = - container.executeJob("/fake_sink_paimon_truncate_with_local_case2.conf"); - Assertions.assertEquals(0, readResult.getExitCode()); - given().ignoreExceptions() - .await() - .atLeast(100L, TimeUnit.MILLISECONDS) - .atMost(30L, TimeUnit.SECONDS) - .untilAsserted( - () -> { - // copy paimon to local - container.executeExtraCommands(containerExtendedFactory); - List paimonRecords = - loadPaimonData("seatunnel_namespace10", TARGET_TABLE); - Assertions.assertEquals(2, paimonRecords.size()); - paimonRecords.forEach( - paimonRecord -> { - if (paimonRecord.getPkId() == 1) { - Assertions.assertEquals("Aa", paimonRecord.getName()); - } - if (paimonRecord.getPkId() == 2) { - Assertions.assertEquals("Bb", paimonRecord.getName()); - } - Assertions.assertEquals(200, paimonRecord.getScore()); - }); - List ids = - paimonRecords.stream() - .map(PaimonRecord::getPkId) - .collect(Collectors.toList()); - Assertions.assertFalse(ids.contains(3L)); - }); - } - protected final ContainerExtendedFactory containerExtendedFactory = container -> { if (isWindows) { @@ -605,7 +568,7 @@ private void extractFilesWin() { } private List loadPaimonData(String dbName, String tbName) throws Exception { - FileStoreTable table = (FileStoreTable) getTable(dbName, tbName); + Table table = getTable(dbName, tbName); ReadBuilder readBuilder = table.newReadBuilder(); TableScan.Plan plan = readBuilder.newScan().plan(); TableRead tableRead = readBuilder.newRead(); @@ -619,12 +582,7 @@ private List loadPaimonData(String dbName, String tbName) throws E try (RecordReader reader = tableRead.createReader(plan)) { reader.forEachRemaining( row -> { - PaimonRecord paimonRecord = - new PaimonRecord(row.getLong(0), row.getString(1).toString()); - if (table.schema().fieldNames().contains("score")) { - paimonRecord.setScore(row.getInt(2)); - } - result.add(paimonRecord); + result.add(new PaimonRecord(row.getLong(0), row.getString(1).toString())); log.info("key_id:" + row.getLong(0) + ", name:" + row.getString(1)); }); } @@ -653,7 +611,7 @@ private Identifier getIdentifier(String dbName, String tbName) { private Catalog getCatalog() { Options options = new Options(); if (isWindows) { - options.set("warehouse", CATALOG_DIR_WIN); + options.set("warehouse", "file://" + CATALOG_DIR_WIN); } else { options.set("warehouse", "file://" + CATALOG_DIR); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java index e93f45d9139..259bc0128a5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkHdfsIT.java @@ -34,7 +34,6 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.InternalRow; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.table.source.TableRead; @@ -51,7 +50,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; import static org.awaitility.Awaitility.given; @@ -206,131 +204,4 @@ public void testFakeCDCSinkPaimonWithHiveCatalogAndRead(TestContainer container) container.executeJob("/paimon_to_assert_with_hivecatalog.conf"); Assertions.assertEquals(0, readResult.getExitCode()); } - - @TestTemplate - public void testSinkPaimonHdfsTruncateTable(TestContainer container) throws Exception { - Container.ExecResult writeResult = - container.executeJob("/fake_sink_paimon_truncate_with_hdfs_case1.conf"); - Assertions.assertEquals(0, writeResult.getExitCode()); - Container.ExecResult readResult = - container.executeJob("/fake_sink_paimon_truncate_with_hdfs_case2.conf"); - Assertions.assertEquals(0, readResult.getExitCode()); - given().ignoreExceptions() - .await() - .atLeast(100L, TimeUnit.MILLISECONDS) - .atMost(180L, TimeUnit.SECONDS) - .untilAsserted( - () -> { - PaimonSinkConfig paimonSinkConfig = - new PaimonSinkConfig( - ReadonlyConfig.fromMap(PAIMON_SINK_PROPERTIES)); - PaimonCatalogLoader paimonCatalogLoader = - new PaimonCatalogLoader(paimonSinkConfig); - Catalog catalog = paimonCatalogLoader.loadCatalog(); - List paimonRecords = - loadPaimonData(catalog, "seatunnel_namespace11", "st_test"); - Assertions.assertEquals(2, paimonRecords.size()); - paimonRecords.forEach( - paimonRecord -> { - if (paimonRecord.getPkId() == 1) { - Assertions.assertEquals("Aa", paimonRecord.getName()); - } - if (paimonRecord.getPkId() == 2) { - Assertions.assertEquals("Bb", paimonRecord.getName()); - } - Assertions.assertEquals(200, paimonRecord.getScore()); - }); - List ids = - paimonRecords.stream() - .map(PaimonRecord::getPkId) - .collect(Collectors.toList()); - Assertions.assertFalse(ids.contains(3L)); - }); - } - - @TestTemplate - public void testSinkPaimonHiveTruncateTable(TestContainer container) throws Exception { - Container.ExecResult writeResult = - container.executeJob("/fake_sink_paimon_truncate_with_hive_case1.conf"); - Assertions.assertEquals(0, writeResult.getExitCode()); - Container.ExecResult readResult = - container.executeJob("/fake_sink_paimon_truncate_with_hive_case2.conf"); - Assertions.assertEquals(0, readResult.getExitCode()); - given().ignoreExceptions() - .await() - .atLeast(100L, TimeUnit.MILLISECONDS) - .atMost(180L, TimeUnit.SECONDS) - .untilAsserted( - () -> { - PaimonSinkConfig paimonSinkConfig = - new PaimonSinkConfig( - ReadonlyConfig.fromMap(PAIMON_SINK_PROPERTIES)); - PaimonCatalogLoader paimonCatalogLoader = - new PaimonCatalogLoader(paimonSinkConfig); - Catalog catalog = paimonCatalogLoader.loadCatalog(); - List paimonRecords = - loadPaimonData(catalog, "seatunnel_namespace12", "st_test"); - Assertions.assertEquals(2, paimonRecords.size()); - paimonRecords.forEach( - paimonRecord -> { - if (paimonRecord.getPkId() == 1) { - Assertions.assertEquals("Aa", paimonRecord.getName()); - } - if (paimonRecord.getPkId() == 2) { - Assertions.assertEquals("Bb", paimonRecord.getName()); - } - Assertions.assertEquals(200, paimonRecord.getScore()); - }); - List ids = - paimonRecords.stream() - .map(PaimonRecord::getPkId) - .collect(Collectors.toList()); - Assertions.assertFalse(ids.contains(3L)); - }); - } - - @TestTemplate - public void testSinkPaimonHiveTruncateTable1(TestContainer container) throws Exception { - PaimonSinkConfig paimonSinkConfig = - new PaimonSinkConfig(ReadonlyConfig.fromMap(PAIMON_SINK_PROPERTIES)); - PaimonCatalogLoader paimonCatalogLoader = new PaimonCatalogLoader(paimonSinkConfig); - Catalog catalog = paimonCatalogLoader.loadCatalog(); - List paimonRecords = - loadPaimonData(catalog, "seatunnel_namespace11", "st_test"); - Assertions.assertEquals(2, paimonRecords.size()); - paimonRecords.forEach( - paimonRecord -> { - if (paimonRecord.getPkId() == 1) { - Assertions.assertEquals("Aa", paimonRecord.getName()); - } - if (paimonRecord.getPkId() == 2) { - Assertions.assertEquals("Bb", paimonRecord.getName()); - } - Assertions.assertEquals(200, paimonRecord.getScore()); - }); - List ids = - paimonRecords.stream().map(PaimonRecord::getPkId).collect(Collectors.toList()); - Assertions.assertFalse(ids.contains(3L)); - } - - private List loadPaimonData(Catalog catalog, String dbName, String tbName) - throws Exception { - FileStoreTable table = (FileStoreTable) catalog.getTable(Identifier.create(dbName, tbName)); - ReadBuilder readBuilder = table.newReadBuilder(); - TableScan.Plan plan = readBuilder.newScan().plan(); - TableRead tableRead = readBuilder.newRead(); - List result = new ArrayList<>(); - try (RecordReader reader = tableRead.createReader(plan)) { - reader.forEachRemaining( - row -> { - PaimonRecord paimonRecord = - new PaimonRecord(row.getLong(0), row.getString(1).toString()); - if (table.schema().fieldNames().contains("score")) { - paimonRecord.setScore(row.getInt(2)); - } - result.add(paimonRecord); - }); - } - return result; - } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf deleted file mode 100644 index 92f6f5c6de2..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case1.conf +++ /dev/null @@ -1,80 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - schema = { - fields { - pk_id = bigint - name = string - score = int - } - primaryKey { - name = "pk_id" - columnNames = [pk_id] - } - } - rows = [ - { - kind = INSERT - fields = [1, "A", 100] - }, - { - kind = INSERT - fields = [2, "B", 100] - }, - { - kind = INSERT - fields = [3, "C", 100] - }, - { - kind = UPDATE_BEFORE - fields = [1, "A", 100] - }, - { - kind = UPDATE_AFTER - fields = [1, "A_1", 100] - }, - { - kind = DELETE - fields = [2, "B", 100] - } - ] - } -} - -sink { - Paimon { - warehouse = "hdfs:///tmp/paimon" - database = "seatunnel_namespace11" - table = "st_test" - paimon.hadoop.conf = { - fs.defaultFS = "hdfs://nameservice1" - dfs.nameservices = "nameservice1" - dfs.ha.namenodes.nameservice1 = "nn1,nn2" - dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" - dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" - dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - dfs.client.use.datanode.hostname = "true" - } - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf deleted file mode 100644 index 1a5eac73229..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hdfs_case2.conf +++ /dev/null @@ -1,65 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - schema = { - fields { - pk_id = bigint - name = string - score = int - } - primaryKey { - name = "pk_id" - columnNames = [pk_id] - } - } - rows = [ - { - kind = INSERT - fields = [1, "Aa", 200] - }, - { - kind = INSERT - fields = [2, "Bb", 200] - } - ] - } -} - -sink { - Paimon { - warehouse = "hdfs:///tmp/paimon" - database = "seatunnel_namespace11" - table = "st_test" - data_save_mode=DROP_DATA - paimon.hadoop.conf = { - fs.defaultFS = "hdfs://nameservice1" - dfs.nameservices = "nameservice1" - dfs.ha.namenodes.nameservice1 = "nn1,nn2" - dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" - dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" - dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - dfs.client.use.datanode.hostname = "true" - } - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf deleted file mode 100644 index 26e95870e3d..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case1.conf +++ /dev/null @@ -1,82 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - schema = { - fields { - pk_id = bigint - name = string - score = int - } - primaryKey { - name = "pk_id" - columnNames = [pk_id] - } - } - rows = [ - { - kind = INSERT - fields = [1, "A", 100] - }, - { - kind = INSERT - fields = [2, "B", 100] - }, - { - kind = INSERT - fields = [3, "C", 100] - }, - { - kind = UPDATE_BEFORE - fields = [1, "A", 100] - }, - { - kind = UPDATE_AFTER - fields = [1, "A_1", 100] - }, - { - kind = DELETE - fields = [2, "B", 100] - } - ] - } -} - -sink { - Paimon { - warehouse = "hdfs:///tmp/paimon" - catalog_type="hive" - catalog_uri="thrift://hadoop04:9083" - database = "seatunnel_namespace12" - table = "st_test" - paimon.hadoop.conf = { - fs.defaultFS = "hdfs://nameservice1" - dfs.nameservices = "nameservice1" - dfs.ha.namenodes.nameservice1 = "nn1,nn2" - dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" - dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" - dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - dfs.client.use.datanode.hostname = "true" - } - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf deleted file mode 100644 index ef1e79b86e2..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_hive_case2.conf +++ /dev/null @@ -1,67 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - schema = { - fields { - pk_id = bigint - name = string - score = int - } - primaryKey { - name = "pk_id" - columnNames = [pk_id] - } - } - rows = [ - { - kind = INSERT - fields = [1, "Aa", 200] - }, - { - kind = INSERT - fields = [2, "Bb", 200] - } - ] - } -} - -sink { - Paimon { - warehouse = "hdfs:///tmp/paimon" - catalog_type="hive" - catalog_uri="thrift://hadoop04:9083" - database = "seatunnel_namespace12" - table = "st_test" - data_save_mode=DROP_DATA - paimon.hadoop.conf = { - fs.defaultFS = "hdfs://nameservice1" - dfs.nameservices = "nameservice1" - dfs.ha.namenodes.nameservice1 = "nn1,nn2" - dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020" - dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020" - dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - dfs.client.use.datanode.hostname = "true" - } - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf deleted file mode 100644 index e22474a06d5..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case1.conf +++ /dev/null @@ -1,71 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - schema = { - fields { - pk_id = bigint - name = string - score = int - } - primaryKey { - name = "pk_id" - columnNames = [pk_id] - } - } - rows = [ - { - kind = INSERT - fields = [1, "A", 100] - }, - { - kind = INSERT - fields = [2, "B", 100] - }, - { - kind = INSERT - fields = [3, "C", 100] - }, - { - kind = UPDATE_BEFORE - fields = [1, "A", 100] - }, - { - kind = UPDATE_AFTER - fields = [1, "A_1", 100] - }, - { - kind = DELETE - fields = [2, "B", 100] - } - ] - } -} - -sink { - Paimon { - warehouse = "file:///tmp/paimon" - database = "seatunnel_namespace10" - table = "st_test" - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf deleted file mode 100644 index 64cb24bc8ef..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/resources/fake_sink_paimon_truncate_with_local_case2.conf +++ /dev/null @@ -1,56 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - schema = { - fields { - pk_id = bigint - name = string - score = int - } - primaryKey { - name = "pk_id" - columnNames = [pk_id] - } - } - rows = [ - { - kind = INSERT - fields = [1, "Aa", 200] - }, - { - kind = INSERT - fields = [2, "Bb", 200] - } - ] - } -} - -sink { - Paimon { - warehouse = "file:///tmp/paimon" - database = "seatunnel_namespace10" - table = "st_test" - data_save_mode=DROP_DATA - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml deleted file mode 100644 index 042e528c5e2..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/pom.xml +++ /dev/null @@ -1,68 +0,0 @@ - - - - 4.0.0 - - org.apache.seatunnel - seatunnel-connector-v2-e2e - ${revision} - - - connector-qdrant-e2e - SeaTunnel : E2E : Connector V2 : Qdrant - - - - org.apache.seatunnel - seatunnel-guava - ${project.version} - optional - - - - io.qdrant - client - 1.11.0 - test - - - org.apache.seatunnel - connector-qdrant - ${project.version} - test - - - org.testcontainers - qdrant - 1.20.1 - test - - - - org.apache.seatunnel - connector-assert - ${project.version} - test - - - org.apache.seatunnel - connector-fake - ${project.version} - test - - - diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java deleted file mode 100644 index 21854a12cad..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/qdrant/QdrantIT.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.e2e.connector.v2.qdrant; - -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.junit.DisabledOnContainer; - -import org.awaitility.Awaitility; -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.lifecycle.Startables; -import org.testcontainers.qdrant.QdrantContainer; -import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; - -import io.qdrant.client.QdrantClient; -import io.qdrant.client.QdrantGrpcClient; -import io.qdrant.client.grpc.Collections; -import io.qdrant.client.grpc.Points; -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.stream.Stream; - -import static io.qdrant.client.PointIdFactory.id; -import static io.qdrant.client.ValueFactory.value; -import static io.qdrant.client.VectorFactory.vector; -import static io.qdrant.client.VectorsFactory.namedVectors; - -@Slf4j -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "SPARK and FLINK do not support vector types yet") -public class QdrantIT extends TestSuiteBase implements TestResource { - - private static final String ALIAS = "qdrante2e"; - private static final String SOURCE_COLLECTION = "source_collection"; - private static final String SINK_COLLECTION = "sink_collection"; - private static final String IMAGE = "qdrant/qdrant:latest"; - private QdrantContainer container; - private QdrantClient qdrantClient; - - @BeforeAll - @Override - public void startUp() throws Exception { - this.container = new QdrantContainer(IMAGE).withNetwork(NETWORK).withNetworkAliases(ALIAS); - Startables.deepStart(Stream.of(this.container)).join(); - Awaitility.given().ignoreExceptions().await().atMost(10L, TimeUnit.SECONDS); - this.initQdrant(); - this.initSourceData(); - } - - private void initQdrant() { - qdrantClient = - new QdrantClient( - QdrantGrpcClient.newBuilder( - container.getHost(), container.getGrpcPort(), false) - .build()); - } - - private void initSourceData() throws Exception { - qdrantClient - .createCollectionAsync( - SOURCE_COLLECTION, - ImmutableMap.of( - "my_vector", - Collections.VectorParams.newBuilder() - .setSize(4) - .setDistance(Collections.Distance.Cosine) - .build())) - .get(); - - qdrantClient - .createCollectionAsync( - SINK_COLLECTION, - ImmutableMap.of( - "my_vector", - Collections.VectorParams.newBuilder() - .setSize(4) - .setDistance(Collections.Distance.Cosine) - .build())) - .get(); - - List points = new ArrayList<>(); - for (int i = 1; i <= 10; i++) { - Points.PointStruct.Builder pointStruct = Points.PointStruct.newBuilder(); - pointStruct.setId(id(i)); - List floats = Arrays.asList((float) i, (float) i, (float) i, (float) i); - pointStruct.setVectors(namedVectors(ImmutableMap.of("my_vector", vector(floats)))); - - pointStruct.putPayload("file_size", value(i)); - pointStruct.putPayload("file_name", value("file-name-" + i)); - - points.add(pointStruct.build()); - } - - qdrantClient - .upsertAsync( - Points.UpsertPoints.newBuilder() - .setCollectionName(SOURCE_COLLECTION) - .addAllPoints(points) - .build()) - .get(); - } - - @AfterAll - @Override - public void tearDown() { - this.qdrantClient.close(); - } - - @TestTemplate - public void testQdrant(TestContainer container) - throws IOException, InterruptedException, ExecutionException { - Container.ExecResult execResult = container.executeJob("/qdrant-to-qdrant.conf"); - Assertions.assertEquals(execResult.getExitCode(), 0); - Assertions.assertEquals(qdrantClient.countAsync(SINK_COLLECTION).get(), 10); - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf deleted file mode 100644 index 8fa4c4f1da0..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-qdrant-e2e/src/test/resources/qdrant-to-qdrant.conf +++ /dev/null @@ -1,51 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - Qdrant { - collection_name = "source_collection" - host = "qdrante2e" - schema = { - columns = [ - { - name = file_name - type = string - } - { - name = file_size - type = int - } - { - name = my_vector - type = float_vector - } - ] - } - } -} - -sink { - Qdrant { - collection_name = "sink_collection" - host = "qdrante2e" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java index a846949d857..7052aa9bef8 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/java/org/apache/seatunnel/e2e/connector/rabbitmq/RabbitmqIT.java @@ -75,9 +75,6 @@ public class RabbitmqIT extends TestSuiteBase implements TestResource { private static final String SINK_QUEUE_NAME = "test1"; private static final String USERNAME = "guest"; private static final String PASSWORD = "guest"; - private static final Boolean DURABLE = true; - private static final Boolean EXCLUSIVE = false; - private static final Boolean AUTO_DELETE = false; private static final Pair> TEST_DATASET = generateTestDataSet(); @@ -188,9 +185,6 @@ private void initRabbitMQ() { config.setVirtualHost("/"); config.setUsername(USERNAME); config.setPassword(PASSWORD); - config.setDurable(DURABLE); - config.setExclusive(EXCLUSIVE); - config.setAutoDelete(AUTO_DELETE); rabbitmqClient = new RabbitmqClient(config); } catch (Exception e) { throw new RuntimeException("init Rabbitmq error", e); @@ -207,9 +201,6 @@ private RabbitmqClient initSinkRabbitMQ() { config.setVirtualHost("/"); config.setUsername(USERNAME); config.setPassword(PASSWORD); - config.setDurable(DURABLE); - config.setExclusive(EXCLUSIVE); - config.setAutoDelete(AUTO_DELETE); return new RabbitmqClient(config); } catch (Exception e) { throw new RuntimeException("init Rabbitmq error", e); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf index 61267a3adce..b3a834bdc2f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-rabbitmq-e2e/src/test/resources/rabbitmq-to-rabbitmq.conf @@ -28,9 +28,6 @@ source { username = "guest" password = "guest" queue_name = "test" - durable = "true" - exclusive = "false" - auto_delete = "false" for_e2e_testing = true schema = { fields { @@ -64,9 +61,6 @@ sink { virtual_host = "/" username = "guest" password = "guest" - durable = "true" - exclusive = "false" - auto_delete = "false" queue_name = "test1" } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml deleted file mode 100644 index 0a7243ed6a3..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/pom.xml +++ /dev/null @@ -1,51 +0,0 @@ - - - - 4.0.0 - - org.apache.seatunnel - seatunnel-connector-v2-e2e - ${revision} - - - connector-typesense-e2e - SeaTunnel : E2E : Connector V2 : Typesense - - - 8 - 8 - UTF-8 - - - - - - org.apache.seatunnel - connector-fake - ${project.version} - test - - - org.apache.seatunnel - connector-typesense - ${project.version} - test - - - diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java deleted file mode 100644 index afff4972e4b..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/java/org/apache/seatunnel/e2e/connector/typesense/TypesenseIT.java +++ /dev/null @@ -1,265 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.e2e.connector.typesense; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; - -import org.apache.seatunnel.connectors.seatunnel.typesense.client.TypesenseClient; -import org.apache.seatunnel.e2e.common.TestResource; -import org.apache.seatunnel.e2e.common.TestSuiteBase; -import org.apache.seatunnel.e2e.common.container.TestContainer; - -import org.apache.commons.lang3.RandomUtils; - -import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.DockerLoggerFactory; -import org.typesense.api.FieldTypes; -import org.typesense.model.Field; - -import com.google.common.collect.Lists; -import lombok.extern.slf4j.Slf4j; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.stream.Stream; - -@Slf4j -public class TypesenseIT extends TestSuiteBase implements TestResource { - - private static final String TYPESENSE_DOCKER_IMAGE = "typesense/typesense:26.0"; - - private static final String HOST = "e2e_typesense"; - - private static final int PORT = 8108; - - private GenericContainer typesenseServer; - - private TypesenseClient typesenseClient; - - private static final String sinkCollection = "typesense_test_collection"; - - private static final String sourceCollection = "typesense_test_collection_for_source"; - - @BeforeEach - @Override - public void startUp() throws Exception { - typesenseServer = - new GenericContainer<>(TYPESENSE_DOCKER_IMAGE) - .withNetwork(NETWORK) - .withNetworkAliases(HOST) - .withPrivilegedMode(true) - .withStartupAttempts(5) - .withCommand("--data-dir=/", "--api-key=xyz") - .withStartupTimeout(Duration.ofMinutes(5)) - .withLogConsumer( - new Slf4jLogConsumer( - DockerLoggerFactory.getLogger(TYPESENSE_DOCKER_IMAGE))); - typesenseServer.setPortBindings(Lists.newArrayList(String.format("%s:%s", PORT, PORT))); - Startables.deepStart(Stream.of(typesenseServer)).join(); - log.info("Typesense container started"); - Awaitility.given() - .ignoreExceptions() - .atLeast(1L, TimeUnit.SECONDS) - .pollInterval(1L, TimeUnit.SECONDS) - .atMost(120L, TimeUnit.SECONDS) - .untilAsserted(this::initConnection); - } - - private void initConnection() { - String host = typesenseServer.getContainerIpAddress(); - typesenseClient = - TypesenseClient.createInstance(Lists.newArrayList(host + ":8108"), "xyz", "http"); - } - - /** Test setting primary_keys parameter write Typesense */ - @TestTemplate - public void testFakeToTypesenseWithPrimaryKeys(TestContainer container) throws Exception { - Container.ExecResult execResult = - container.executeJob("/fake_to_typesense_with_primary_keys.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); - } - - @TestTemplate - public void testFakeToTypesenseWithRecreateSchema(TestContainer container) throws Exception { - List fields = new ArrayList<>(); - fields.add(new Field().name("T").type(FieldTypes.BOOL)); - Assertions.assertTrue(typesenseClient.createCollection(sinkCollection, fields)); - Map field = typesenseClient.getField(sinkCollection); - Container.ExecResult execResult = - container.executeJob("/fake_to_typesense_with_recreate_schema.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); - Assertions.assertNotEquals(field, typesenseClient.getField(sinkCollection)); - } - - @TestTemplate - public void testFakeToTypesenseWithErrorWhenNotExists(TestContainer container) - throws Exception { - Container.ExecResult execResult = - container.executeJob("/fake_to_typesense_with_error_when_not_exists.conf"); - Assertions.assertEquals(1, execResult.getExitCode()); - } - - @TestTemplate - public void testFakeToTypesenseWithCreateWhenNotExists(TestContainer container) - throws Exception { - Container.ExecResult execResult = - container.executeJob("/fake_to_typesense_with_create_when_not_exists.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); - } - - @TestTemplate - public void testFakeToTypesenseWithDropData(TestContainer container) throws Exception { - String initData = "{\"name\":\"Han\",\"age\":12}"; - typesenseClient.createCollection(sinkCollection); - typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); - Container.ExecResult execResult = - container.executeJob("/fake_to_typesense_with_drop_data.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 5); - } - - @TestTemplate - public void testFakeToTypesenseWithAppendData(TestContainer container) throws Exception { - String initData = "{\"name\":\"Han\",\"age\":12}"; - typesenseClient.createCollection(sinkCollection); - typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); - Container.ExecResult execResult = - container.executeJob("/fake_to_typesense_with_append_data.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 6); - } - - @TestTemplate - public void testFakeToTypesenseWithErrorWhenDataExists(TestContainer container) - throws Exception { - String initData = "{\"name\":\"Han\",\"age\":12}"; - typesenseClient.createCollection(sinkCollection); - typesenseClient.insert(sinkCollection, Lists.newArrayList(initData)); - Assertions.assertEquals(typesenseClient.search(sinkCollection, null, 0).getFound(), 1); - Container.ExecResult execResult = - container.executeJob("/fake_to_typesense_with_error_when_data_exists.conf"); - Assertions.assertEquals(1, execResult.getExitCode()); - } - - public List genTestData(int recordNum) { - ArrayList testDataList = new ArrayList<>(); - ObjectMapper objectMapper = new ObjectMapper(); - HashMap doc = new HashMap<>(); - for (int i = 0; i < recordNum; i++) { - try { - doc.put("num_employees", RandomUtils.nextInt()); - doc.put("flag", RandomUtils.nextBoolean()); - doc.put("num", RandomUtils.nextLong()); - doc.put("company_name", "A" + RandomUtils.nextInt(1, 100)); - testDataList.add(objectMapper.writeValueAsString(doc)); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - } - return testDataList; - } - - @TestTemplate - public void testTypesenseSourceAndSink(TestContainer container) throws Exception { - int recordNum = 100; - List testData = genTestData(recordNum); - typesenseClient.createCollection(sourceCollection); - typesenseClient.insert(sourceCollection, testData); - Assertions.assertEquals( - typesenseClient.search(sourceCollection, null, 0).getFound(), recordNum); - Container.ExecResult execResult = container.executeJob("/typesense_source_and_sink.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals( - typesenseClient.search(sinkCollection, null, 0).getFound(), recordNum); - } - - @TestTemplate - public void testTypesenseToTypesense(TestContainer container) throws Exception { - String typesenseToTypesenseSource = "typesense_to_typesense_source"; - String typesenseToTypesenseSink = "typesense_to_typesense_sink"; - List testData = new ArrayList<>(); - testData.add( - "{\"c_row\":{\"c_array_int\":[12,45,96,8],\"c_int\":91,\"c_string\":\"String_412\"},\"company_name\":\"Company_9986\",\"company_name_list\":[\"Company_9986_Alias_1\",\"Company_9986_Alias_2\"],\"country\":\"Country_181\",\"id\":\"9986\",\"num_employees\":1914}"); - testData.add( - "{\"c_row\":{\"c_array_int\":[60],\"c_int\":9,\"c_string\":\"String_371\"},\"company_name\":\"Company_9988\",\"company_name_list\":[\"Company_9988_Alias_1\",\"Company_9988_Alias_2\",\"Company_9988_Alias_3\"],\"country\":\"Country_86\",\"id\":\"9988\",\"num_employees\":7366}"); - typesenseClient.createCollection(typesenseToTypesenseSource); - typesenseClient.insert(typesenseToTypesenseSource, testData); - Assertions.assertEquals( - typesenseClient.search(typesenseToTypesenseSource, null, 0).getFound(), 2); - Container.ExecResult execResult = container.executeJob("/typesense_to_typesense.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals( - typesenseClient.search(typesenseToTypesenseSink, null, 0).getFound(), 2); - ObjectMapper objectMapper = new ObjectMapper(); - Map sourceData = objectMapper.readValue(testData.get(0), Map.class); - Map sinkData = - typesenseClient - .search(typesenseToTypesenseSink, null, 0) - .getHits() - .get(0) - .getDocument(); - Assertions.assertNotEquals(sourceData.remove("id"), sinkData.remove("id")); - Assertions.assertEquals(sourceData, sinkData); - } - - @TestTemplate - public void testTypesenseToTypesenseWithQuery(TestContainer container) throws Exception { - String typesenseToTypesenseSource = "typesense_to_typesense_source_with_query"; - String typesenseToTypesenseSink = "typesense_to_typesense_sink_with_query"; - List testData = new ArrayList<>(); - testData.add( - "{\"c_row\":{\"c_array_int\":[12,45,96,8],\"c_int\":91,\"c_string\":\"String_412\"},\"company_name\":\"Company_9986\",\"company_name_list\":[\"Company_9986_Alias_1\",\"Company_9986_Alias_2\"],\"country\":\"Country_181\",\"id\":\"9986\",\"num_employees\":1914}"); - testData.add( - "{\"c_row\":{\"c_array_int\":[60],\"c_int\":9,\"c_string\":\"String_371\"},\"company_name\":\"Company_9988\",\"company_name_list\":[\"Company_9988_Alias_1\",\"Company_9988_Alias_2\",\"Company_9988_Alias_3\"],\"country\":\"Country_86\",\"id\":\"9988\",\"num_employees\":7366}"); - testData.add( - "{\"c_row\":{\"c_array_int\":[18,97],\"c_int\":32,\"c_string\":\"String_48\"},\"company_name\":\"Company_9880\",\"company_name_list\":[\"Company_9880_Alias_1\",\"Company_9880_Alias_2\",\"Company_9880_Alias_3\",\"Company_9880_Alias_4\"],\"country\":\"Country_159\",\"id\":\"9880\",\"num_employees\":141}"); - typesenseClient.createCollection(typesenseToTypesenseSource); - typesenseClient.insert(typesenseToTypesenseSource, testData); - Assertions.assertEquals( - typesenseClient.search(typesenseToTypesenseSource, null, 0).getFound(), 3); - Container.ExecResult execResult = - container.executeJob("/typesense_to_typesense_with_query.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - Assertions.assertEquals( - typesenseClient.search(typesenseToTypesenseSink, null, 0).getFound(), 2); - } - - @AfterEach - @Override - public void tearDown() { - typesenseServer.close(); - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf deleted file mode 100644 index ab1c7b171dc..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_append_data.conf +++ /dev/null @@ -1,52 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - result_table_name = "typesense_test_table" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","num"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "APPEND_DATA" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf deleted file mode 100644 index 78870438e0a..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_create_when_not_exists.conf +++ /dev/null @@ -1,51 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - result_table_name = "typesense_test_table" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","num"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf deleted file mode 100644 index 01094580a62..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_drop_data.conf +++ /dev/null @@ -1,52 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - result_table_name = "typesense_test_table" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - api_key = "xyz" - max_retry_count = 3 - max_batch_size = 10 - primary_keys = ["num_employees","num"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "DROP_DATA" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf deleted file mode 100644 index 6496ceb979c..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_data_exists.conf +++ /dev/null @@ -1,52 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - result_table_name = "typesense_test_table" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","num"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "ERROR_WHEN_DATA_EXISTS" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf deleted file mode 100644 index c3538c846a8..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_error_when_not_exists.conf +++ /dev/null @@ -1,51 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - result_table_name = "typesense_test_table" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","num"] - key_delimiter = "=" - schema_save_mode = "ERROR_WHEN_SCHEMA_NOT_EXIST" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf deleted file mode 100644 index 2a767db02d2..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_primary_keys.conf +++ /dev/null @@ -1,50 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - result_table_name = "typesense_test_table" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","num"] - key_delimiter = "=" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf deleted file mode 100644 index ee7acce8a4b..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/fake_to_typesense_with_recreate_schema.conf +++ /dev/null @@ -1,51 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - result_table_name = "typesense_test_table" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","num"] - key_delimiter = "=" - schema_save_mode = "RECREATE_SCHEMA" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf deleted file mode 100644 index 25e63cf2443..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_source_and_sink.conf +++ /dev/null @@ -1,54 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" -} - -source { - Typesense { - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection_for_source" - api_key = "xyz" - schema { - fields { - company_name = string - num = long - id = string - num_employees = int - flag = boolean - } - } - result_table_name = "typesense_test_table" - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_test_collection" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","num"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "DROP_DATA" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf deleted file mode 100644 index f8c148a7afa..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense.conf +++ /dev/null @@ -1,64 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" - flink.execution.checkpointing.interval=5000 - flink.execution.restart.strategy = failure-rate - flink.execution.restart.failureInterval = 60000 - flink.execution.restart.failureRate = 100 - flink.execution.restart.delayInterval = 10000 - -} -source { - Typesense { - hosts = ["e2e_typesense:8108"] - collection = "typesense_to_typesense_source" - api_key = "xyz" - result_table_name = "typesense_test_table" - schema = { - fields { - company_name_list = array - company_name = string - num_employees = long - country = string - id = string - c_row = { - c_int = int - c_string = string - c_array_int = array - } - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_to_typesense_sink" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","id"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "APPEND_DATA" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf deleted file mode 100644 index 7b069c90793..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-typesense-e2e/src/test/resources/typesense_to_typesense_with_query.conf +++ /dev/null @@ -1,65 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -env { - parallelism = 1 - job.mode = "BATCH" - flink.execution.checkpointing.interval=5000 - flink.execution.restart.strategy = failure-rate - flink.execution.restart.failureInterval = 60000 - flink.execution.restart.failureRate = 100 - flink.execution.restart.delayInterval = 10000 - -} -source { - Typesense { - hosts = ["e2e_typesense:8108"] - collection = "typesense_to_typesense_source_with_query" - api_key = "xyz" - query = "q=*&filter_by=c_row.c_int:>10" - result_table_name = "typesense_test_table" - schema = { - fields { - company_name_list = array - company_name = string - num_employees = long - country = string - id = string - c_row = { - c_int = int - c_string = string - c_array_int = array - } - } - } - } -} - -sink { - Typesense { - source_table_name = "typesense_test_table" - hosts = ["e2e_typesense:8108"] - collection = "typesense_to_typesense_sink_with_query" - max_retry_count = 3 - max_batch_size = 10 - api_key = "xyz" - primary_keys = ["num_employees","id"] - key_delimiter = "=" - schema_save_mode = "CREATE_SCHEMA_WHEN_NOT_EXIST" - data_save_mode = "APPEND_DATA" - } -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 28be63f3cf0..4933ab02057 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -76,11 +76,8 @@ connector-hudi-e2e connector-milvus-e2e connector-activemq-e2e - connector-qdrant-e2e connector-sls-e2e - connector-typesense-e2e connector-email-e2e - connector-cdc-opengauss-e2e diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties b/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties index c3681980c72..51ace38473a 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/resources/log4j2.properties @@ -29,7 +29,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -39,7 +39,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java index 8e45bbf9de5..8389cb3c058 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-console-seatunnel-e2e/src/test/java/org/apache/seatunnel/engine/e2e/console/FakeSourceToConsoleWithEventReportIT.java @@ -109,7 +109,7 @@ public void testEventReport() throws IOException, InterruptedException { arrayNode.elements().forEachRemaining(jsonNode -> events.add(jsonNode)); } } - Assertions.assertEquals(10, events.size()); + Assertions.assertEquals(8, events.size()); Set eventTypes = events.stream().map(e -> e.get("eventType").asText()).collect(Collectors.toSet()); Assertions.assertTrue( diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml index 6fbdfc826bf..973fe6434c7 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/pom.xml @@ -107,16 +107,6 @@ hadoop-aliyun ${hadoop-aliyun.version} test - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java index f48ca3f1817..ac29b4bf355 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterFaultToleranceIT.java @@ -992,6 +992,7 @@ public void testStreamJobRestoreFromOssInAllNodeDown() throws Exception { + " fs.oss.endpoint: " + OSS_ENDPOINT + "\n" + + " fs.oss.credentials.provider: org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider\n" + " properties:\n" + " hazelcast.invocation.max.retry.count: 200\n" + " hazelcast.tcp.join.port.try.count: 30\n" diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java index 894e901596f..bf16cab75d2 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/ClusterSeaTunnelContainer.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.engine.server.rest.RestConstant; import org.awaitility.Awaitility; -import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -33,21 +32,15 @@ import org.testcontainers.utility.DockerLoggerFactory; import org.testcontainers.utility.MountableFile; -import com.hazelcast.jet.json.JsonUtil; import io.restassured.response.Response; import java.io.IOException; import java.nio.file.Path; import java.nio.file.Paths; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Stream; import static io.restassured.RestAssured.given; import static org.apache.seatunnel.e2e.common.util.ContainerUtil.PROJECT_ROOT_PATH; @@ -120,24 +113,6 @@ public void testSubmitJobWithCustomJobId() { i, paramJobName + "&jobId=" + CUSTOM_JOB_ID, true)); - - String serverLogs = server.getLogs(); - String secondServerLogs = secondServer.getLogs(); - Stream.of( - // [862969647010611201] 2024-08-24 16:01:21,155 INFO - // org.apache.seatunnel.connectors.seatunnel.fake.source.FakeSourceSplitEnumerator - Starting to calculate splits. - "\\[862969647010611201\\].* INFO org\\.apache\\.seatunnel\\.connectors\\.seatunnel\\.fake\\.source\\.FakeSourceSplitEnumerator", - // [862969647010611201] 2024-08-24 16:01:21,278 INFO - // org.apache.seatunnel.connectors.seatunnel.console.sink.ConsoleSinkWriter - // - subtaskIndex=0 rowIndex=63: SeaTunnelRow#tableId=fake - // SeaTunnelRow#kind=INSERT : qOWCd, 1033892054, 671516661 - "\\[862969647010611201\\].* INFO org\\.apache\\.seatunnel\\.connectors\\.seatunnel\\.console\\.sink\\.ConsoleSinkWriter") - .map( - regex -> { - Assertions.assertTrue( - serverLogs.matches(regex) || secondServerLogs.matches(regex)); - return regex; - }); } @Test @@ -294,167 +269,6 @@ private Response submitJob( return submitJob(jobMode, container, false, jobName, paramJobName); } - @Test - public void testStopJobs() { - Arrays.asList(server) - .forEach( - container -> { - try { - submitJobs("STREAMING", container, false, CUSTOM_JOB_ID); - - String parameters = - "[{\"jobId\":" - + CUSTOM_JOB_ID - + ",\"isStopWithSavePoint\":false},{\"jobId\":" - + (CUSTOM_JOB_ID - 1) - + ",\"isStopWithSavePoint\":false}]"; - - given().body(parameters) - .post( - http - + container.getHost() - + colon - + container.getFirstMappedPort() - + RestConstant.STOP_JOBS_URL) - .then() - .statusCode(200) - .body("[0].jobId", equalTo(CUSTOM_JOB_ID)) - .body("[1].jobId", equalTo(CUSTOM_JOB_ID - 1)); - - Awaitility.await() - .atMost(2, TimeUnit.MINUTES) - .untilAsserted( - () -> - given().get( - http - + container - .getHost() - + colon - + container - .getFirstMappedPort() - + RestConstant - .FINISHED_JOBS_INFO - + "/CANCELED") - .then() - .statusCode(200) - .body( - "[0].jobId", - equalTo( - String.valueOf( - CUSTOM_JOB_ID))) - .body( - "[0].jobId", - equalTo( - String.valueOf( - CUSTOM_JOB_ID - - 1)))); - - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - } - - @Test - public void testSubmitJobs() { - AtomicInteger i = new AtomicInteger(); - Arrays.asList(server, secondServer) - .forEach( - container -> { - try { - submitJobs("BATCH", container, false, CUSTOM_JOB_ID); - submitJobs("BATCH", container, true, CUSTOM_JOB_ID); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - } - - private void submitJobs( - String jobMode, GenericContainer container, boolean isStartWithSavePoint, Long jobId) - throws IOException { - - String requestBody = getJobJson(jobMode, isStartWithSavePoint, jobId); - - Response response = - given().body(requestBody) - .header("Content-Type", "application/json; charset=utf-8") - .post( - http - + container.getHost() - + colon - + container.getFirstMappedPort() - + RestConstant.SUBMIT_JOBS_URL); - - response.then() - .statusCode(200) - .body("[0].jobId", equalTo(String.valueOf(jobId))) - .body("[1].jobId", equalTo(String.valueOf(jobId - 1))); - - Response jobInfoResponse = - given().header("Content-Type", "application/json; charset=utf-8") - .get( - http - + container.getHost() - + colon - + container.getFirstMappedPort() - + RestConstant.JOB_INFO_URL - + "/" - + jobId); - jobInfoResponse.then().statusCode(200).body("jobStatus", equalTo("RUNNING")); - } - - private static @NotNull String getJobJson( - String jobMode, boolean isStartWithSavePoint, Long jobId) throws IOException { - List> jobList = new ArrayList<>(); - for (int i = 0; i < 2; i++) { - Map job = new HashMap<>(); - Map params = new HashMap<>(); - params.put("jobId", String.valueOf(jobId - i)); - if (isStartWithSavePoint) { - params.put("isStartWithSavePoint", "true"); - } - job.put("params", params); - - Map env = new HashMap<>(); - env.put("job.mode", jobMode); - job.put("env", env); - - List> sourceList = new ArrayList<>(); - Map source = new HashMap<>(); - source.put("plugin_name", "FakeSource"); - source.put("result_table_name", "fake"); - source.put("row.num", 1000); - - Map schema = new HashMap<>(); - Map fields = new HashMap<>(); - fields.put("name", "string"); - fields.put("age", "int"); - fields.put("card", "int"); - schema.put("fields", fields); - source.put("schema", schema); - - sourceList.add(source); - job.put("source", sourceList); - - List> transformList = new ArrayList<>(); - job.put("transform", transformList); - - List> sinkList = new ArrayList<>(); - Map sink = new HashMap<>(); - sink.put("plugin_name", "Console"); - List sourceTableName = new ArrayList<>(); - sourceTableName.add("fake"); - sink.put("source_table_name", sourceTableName); - - sinkList.add(sink); - job.put("sink", sinkList); - - jobList.add(job); - } - return JsonUtil.toJson(jobList); - } - private Response submitJob( String jobMode, GenericContainer container, diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java deleted file mode 100644 index 63736a90ae5..00000000000 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/resourceIsolation/WorkerTagClusterTest.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.engine.e2e.resourceIsolation; - -import org.apache.seatunnel.engine.common.config.ConfigProvider; -import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.e2e.TestUtils; -import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; -import org.apache.seatunnel.engine.server.resourcemanager.ResourceManager; - -import org.awaitility.Awaitility; -import org.awaitility.core.ThrowingRunnable; -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.config.Config; -import com.hazelcast.instance.impl.HazelcastInstanceImpl; -import com.hazelcast.spi.impl.NodeEngineImpl; -import lombok.extern.slf4j.Slf4j; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -@Slf4j -public class WorkerTagClusterTest { - - HazelcastInstanceImpl masterNode1 = null; - HazelcastInstanceImpl workerNode1 = null; - String testClusterName = "WorkerTagClusterTest"; - - @BeforeEach - public void before() { - SeaTunnelConfig masterNode1Config = getSeaTunnelConfig(testClusterName); - SeaTunnelConfig workerNode1Config = getSeaTunnelConfig(testClusterName); - masterNode1 = SeaTunnelServerStarter.createMasterHazelcastInstance(masterNode1Config); - workerNode1 = SeaTunnelServerStarter.createWorkerHazelcastInstance(workerNode1Config); - } - - @AfterEach - void afterClass() { - if (masterNode1 != null) { - masterNode1.shutdown(); - } - if (workerNode1 != null) { - workerNode1.shutdown(); - } - } - - @Test - public void testTagMatch() throws Exception { - Map tag = new HashMap<>(); - tag.put("group", "platform"); - tag.put("team", "team1"); - testTagFilter(tag, 1); - } - - @Test - public void testTagMatch2() throws Exception { - testTagFilter(null, 1); - } - - @Test - public void testTagNotMatch() throws Exception { - Map tag = new HashMap<>(); - tag.put("group", "platform"); - tag.put("team", "team1111111"); - testTagFilter(tag, 0); - } - - @Test - public void testTagNotMatch2() throws Exception { - testTagFilter(new HashMap<>(), 1); - } - - public void testTagFilter(Map tagFilter, int expectedWorkerCount) - throws Exception { - // waiting all node added to cluster - Awaitility.await() - .atMost(10000, TimeUnit.MILLISECONDS) - .untilAsserted( - new ThrowingRunnable() { - @Override - public void run() throws Throwable { - Thread.sleep(2000); - // check master and worker node - Assertions.assertEquals( - 2, masterNode1.getCluster().getMembers().size()); - NodeEngineImpl nodeEngine = masterNode1.node.nodeEngine; - SeaTunnelServer server = - nodeEngine.getService(SeaTunnelServer.SERVICE_NAME); - ResourceManager resourceManager = - server.getCoordinatorService().getResourceManager(); - // if tag matched, then worker count is 1 else 0 - int workerCount = resourceManager.workerCount(tagFilter); - Assertions.assertEquals(expectedWorkerCount, workerCount); - } - }); - } - - private static SeaTunnelConfig getSeaTunnelConfig(String testClusterName) { - Config hazelcastConfig = Config.loadFromString(getHazelcastConfig()); - hazelcastConfig.setClusterName(TestUtils.getClusterName(testClusterName)); - SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); - seaTunnelConfig.setHazelcastConfig(hazelcastConfig); - return seaTunnelConfig; - } - - protected static String getHazelcastConfig() { - return "hazelcast:\n" - + " cluster-name: seatunnel\n" - + " network:\n" - + " rest-api:\n" - + " enabled: true\n" - + " endpoint-groups:\n" - + " CLUSTER_WRITE:\n" - + " enabled: true\n" - + " join:\n" - + " tcp-ip:\n" - + " enabled: true\n" - + " member-list:\n" - + " - localhost\n" - + " port:\n" - + " auto-increment: true\n" - + " port-count: 100\n" - + " port: 5801\n" - + "\n" - + " properties:\n" - + " hazelcast.invocation.max.retry.count: 200\n" - + " hazelcast.tcp.join.port.try.count: 30\n" - + " hazelcast.invocation.retry.pause.millis: 2000\n" - + " hazelcast.slow.operation.detector.stacktrace.logging.enabled: true\n" - + " hazelcast.logging.type: log4j2\n" - + " hazelcast.operation.generic.thread.count: 200\n" - + " member-attributes:\n" - + " group:\n" - + " type: string\n" - + " value: platform\n" - + " team:\n" - + " type: string\n" - + " value: team1"; - } -} diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties index 05ffd76520a..f0090af0248 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties index 030095bda72..fc0c2063b7a 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server1-resources/log4j2.properties @@ -33,7 +33,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -43,7 +43,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties index 05ffd76520a..f0090af0248 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties index 030095bda72..fc0c2063b7a 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server2-resources/log4j2.properties @@ -33,7 +33,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -43,7 +43,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties index 05ffd76520a..f0090af0248 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties index 030095bda72..fc0c2063b7a 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/connector-package-service-test-server3-resources/log4j2.properties @@ -33,7 +33,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -43,7 +43,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties index 05ffd76520a..f0090af0248 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2-test.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties index ffc08447233..3b1f20dd67e 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/log4j2.properties @@ -44,7 +44,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -54,7 +54,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java deleted file mode 100644 index 0107e8b6170..00000000000 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestEmbeddingIT.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.e2e.transform; - -import org.apache.seatunnel.e2e.common.TestResource; -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.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; - -@DisabledOnContainer( - value = {}, - type = {EngineType.SPARK}, - disabledReason = "Currently SPARK not support adapt") -public class TestEmbeddingIT 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("/mock-embedding.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 + "/mock-embedding.json") - .withEnv( - "MOCKSERVER_INITIALIZATION_JSON_PATH", - TMP_DIR + "/mock-embedding.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 testEmbedding(TestContainer container) throws IOException, InterruptedException { - Container.ExecResult execResult = container.executeJob("/embedding_transform.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } - - @TestTemplate - public void testEmbeddingWithCustomModel(TestContainer container) - throws IOException, InterruptedException { - Container.ExecResult execResult = container.executeJob("/embedding_transform_custom.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } -} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java index 5a4fa640e96..6f17c5a94f7 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestLLMIT.java @@ -87,11 +87,4 @@ public void testLLMWithOpenAI(TestContainer container) Container.ExecResult execResult = container.executeJob("/llm_openai_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } - - @TestTemplate - public void testLLMWithCustomModel(TestContainer container) - throws IOException, InterruptedException { - Container.ExecResult execResult = container.executeJob("/llm_transform_custom.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf deleted file mode 100644 index aa0288a705b..00000000000 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform.conf +++ /dev/null @@ -1,295 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - schema = { - fields { - book_id = "int" - book_name = "string" - book_intro = "string" - author_biography = "string" - } - } - rows = [ - {fields = [1, "To Kill a Mockingbird", - "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", - "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." - ], kind = INSERT} - {fields = [2, "1984", - "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", - "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." - ], kind = INSERT} - {fields = [3, "Pride and Prejudice", - "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", - "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." - ], kind = INSERT} - {fields = [4, "The Great GatsbyThe Great Gatsby", - "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", - "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." - ], kind = INSERT} - {fields = [5, "Moby-Dick", - "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", - "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." - ], kind = INSERT} - ] - result_table_name = "fake" - } -} - -transform { - Embedding { - source_table_name = "fake" - model_provider = QIANFAN - model = bge_large_en - api_key = xxxxxxxx - secret_key = xxxxxxxx - api_path = "http://mockserver:1080/v1/qianfan/embedding" - oauth_path = "http://mockserver:1080/v1/qianfan/token" - single_vectorized_input_number = 2 - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - result_table_name = "embedding_output_1" - } - - Embedding { - source_table_name = "fake" - model_provider = DOUBAO - model = ep-20240830113341-wwwqd - api_key = xxxxxxxx - api_path = "http://mockserver:1080/v1/doubao/embedding" - single_vectorized_input_number = 2 - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - result_table_name = "embedding_output_2" - } - - - Embedding { - source_table_name = "fake" - model_provider = OPENAI - model = text-embedding-3-small - api_key = xxxxxxxx - api_path = "http://mockserver:1080/v1/openai/embedding" - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - result_table_name = "embedding_output_3" - } -} - -sink { - Assert { - source_table_name = "embedding_output_1" - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } - Assert { - source_table_name = "embedding_output_2" - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - } - ] - } - } - Assert { - source_table_name = "embedding_output_3" - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - 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/embedding_transform_custom.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform_custom.conf deleted file mode 100644 index 09611d44fc8..00000000000 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/embedding_transform_custom.conf +++ /dev/null @@ -1,152 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - job.mode = "BATCH" -} - -source { - FakeSource { - row.num = 5 - schema = { - fields { - book_id = "int" - book_name = "string" - book_intro = "string" - author_biography = "string" - } - } - rows = [ - {fields = [1, "To Kill a Mockingbird", - "Set in the American South during the 1930s, To Kill a Mockingbird tells the story of young Scout Finch and her brother, Jem, who are growing up in a world of racial inequality and injustice. Their father, Atticus Finch, is a lawyer who defends a black man falsely accused of raping a white woman, teaching his children valuable lessons about morality, courage, and empathy.", - "Harper Lee (1926–2016) was an American novelist best known for To Kill a Mockingbird, which won the Pulitzer Prize in 1961. Lee was born in Monroeville, Alabama, and the town served as inspiration for the fictional Maycomb in her novel. Despite the success of her book, Lee remained a private person and published only one other novel, Go Set a Watchman, which was written before To Kill a Mockingbird but released in 2015 as a sequel." - ], kind = INSERT} - {fields = [2, "1984", - "1984 is a dystopian novel set in a totalitarian society governed by Big Brother. The story follows Winston Smith, a man who works for the Party rewriting history. Winston begins to question the Party’s control and seeks truth and freedom in a society where individuality is crushed. The novel explores themes of surveillance, propaganda, and the loss of personal autonomy.", - "George Orwell (1903–1950) was the pen name of Eric Arthur Blair, an English novelist, essayist, journalist, and critic. Orwell is best known for his works 1984 and Animal Farm, both of which are critiques of totalitarian regimes. His writing is characterized by lucid prose, awareness of social injustice, opposition to totalitarianism, and support of democratic socialism. Orwell’s work remains influential, and his ideas have shaped contemporary discussions on politics and society." - ], kind = INSERT} - {fields = [3, "Pride and Prejudice", - "Pride and Prejudice is a romantic novel that explores the complex relationships between different social classes in early 19th century England. The story centers on Elizabeth Bennet, a young woman with strong opinions, and Mr. Darcy, a wealthy but reserved gentleman. The novel deals with themes of love, marriage, and societal expectations, offering keen insights into human behavior.", - "Jane Austen (1775–1817) was an English novelist known for her sharp social commentary and keen observations of the British landed gentry. Her works, including Sense and Sensibility, Emma, and Pride and Prejudice, are celebrated for their wit, realism, and biting critique of the social class structure of her time. Despite her relatively modest life, Austen’s novels have gained immense popularity, and she is considered one of the greatest novelists in the English language." - ], kind = INSERT} - {fields = [4, "The Great GatsbyThe Great Gatsby", - "The Great Gatsby is a novel about the American Dream and the disillusionment that can come with it. Set in the 1920s, the story follows Nick Carraway as he becomes entangled in the lives of his mysterious neighbor, Jay Gatsby, and the wealthy elite of Long Island. Gatsby's obsession with the beautiful Daisy Buchanan drives the narrative, exploring themes of wealth, love, and the decay of the American Dream.", - "F. Scott Fitzgerald (1896–1940) was an American novelist and short story writer, widely regarded as one of the greatest American writers of the 20th century. Born in St. Paul, Minnesota, Fitzgerald is best known for his novel The Great Gatsby, which is often considered the quintessential work of the Jazz Age. His works often explore themes of youth, wealth, and the American Dream, reflecting the turbulence and excesses of the 1920s." - ], kind = INSERT} - {fields = [5, "Moby-Dick", - "Moby-Dick is an epic tale of obsession and revenge. The novel follows the journey of Captain Ahab, who is on a relentless quest to kill the white whale, Moby Dick, that once maimed him. Narrated by Ishmael, a sailor aboard Ahab’s ship, the story delves into themes of fate, humanity, and the struggle between man and nature. The novel is also rich with symbolism and philosophical musings.", - "Herman Melville (1819–1891) was an American novelist, short story writer, and poet of the American Renaissance period. Born in New York City, Melville gained initial fame with novels such as Typee and Omoo, but it was Moby-Dick, published in 1851, that would later be recognized as his masterpiece. Melville’s work is known for its complexity, symbolism, and exploration of themes such as man’s place in the universe, the nature of evil, and the quest for meaning. Despite facing financial difficulties and critical neglect during his lifetime, Melville’s reputation soared posthumously, and he is now considered one of the great American authors." - ], kind = INSERT} - ] - result_table_name = "fake" - } -} - -transform { - Embedding { - source_table_name = "fake" - model_provider = CUSTOM - model = text-embedding-3-small - api_key = xxxxxxxx - api_path = "http://mockserver:1080/v1/custom/embedding" - single_vectorized_input_number = 2 - vectorization_fields { - book_intro_vector = book_intro - author_biography_vector = author_biography - } - custom_config={ - custom_response_parse = "$.data[*].embedding" - custom_request_headers = { - # refer to mockserver config - Authorization = "Bearer xxxxxxxx" - } - custom_request_body ={ - modelx = "${model}" - inputx = ["${input}"] - } - } - result_table_name = "embedding_output_1" - } -} - -sink { - Assert { - source_table_name = "embedding_output_1" - rules = - { - field_rules = [ - { - field_name = book_id - field_type = int - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_name - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography - field_type = string - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = book_intro_vector - field_type = float_vector - field_value = [ - { - rule_type = NOT_NULL - } - ] - }, - { - field_name = author_biography_vector - field_type = float_vector - 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/llm_transform_custom.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_transform_custom.conf deleted file mode 100644 index 8f23fa9c1b1..00000000000 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/llm_transform_custom.conf +++ /dev/null @@ -1,94 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### 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 = CUSTOM - 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" - custom_config={ - custom_response_parse = "$.choices[*].message.content" - custom_request_headers = { - Content-Type = "application/json" - Authorization = "Bearer b2e66711-10ed-495c-9f27-f233a8db09c2" - } - custom_request_body ={ - model = "${model}" - messages = [ - { - role = "system" - content = "${prompt}" - }, - { - role = "user" - content = "${input}" - }] - } - } - 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/mock-embedding.json b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mock-embedding.json deleted file mode 100644 index 1688d3b2e27..00000000000 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/mock-embedding.json +++ /dev/null @@ -1,75 +0,0 @@ -// https://www.mock-server.com/mock_server/getting_started.html#request_matchers - -[ - { - "httpRequest": { - "method": "POST", - "path": "/v1/qianfan/embedding/.*", - "queryStringParameters": { - "access_token": ["^.*$"] - } - }, - "httpResponse": { - "body":{"id":"as-7bd1tetvey","object":"embedding_list","created":1724948271,"data":[{"object":"embedding","embedding":[-0.00021059511345811188,-0.027767932042479515,-0.01854688860476017,0.032603006809949875,-0.00508118188008666,-0.0278035756200552,-0.009797265753149986,0.0006566192023456097,0.002196578774601221,0.03303470090031624,-0.0032474317122250795,0.0023599115666002035,0.045467741787433624,-0.009151561185717583,-0.016853950917720795,0.006868097465485334,0.007051482331007719,-0.03534781560301781,-0.039046160876750946,-0.009137873537838459,0.0004795161366928369,0.04441152513027191,0.018654968589544296,-0.0032995922956615686,-0.021618107333779335,-0.0176478773355484,0.05218972638249397,0.02979690209031105,0.04109659045934677,-0.024078253656625748,-0.027630683034658432,0.03802218288183212,0.018793299794197083,0.0413341224193573,0.00588208669796586,-0.020419621840119362,-0.00002904470056819264,0.01946319453418255,-0.008217964321374893,0.002777740126475692,-0.007581755518913269,-0.011274664662778378,-0.007225516252219677,-0.02712559886276722,-0.04745253548026085,-0.033444084227085114,-0.03344576060771942,-0.02444254420697689,-0.002873974619433284,-0.004047377035021782,0.003027654252946377,-0.030405033379793167,-0.034476716071367264,0.011209231801331043,-0.033511288464069366,-0.007611188106238842,0.0029583752620965242,0.02062322199344635,-0.02413185127079487,-0.03221965208649635,-0.006494476459920406,0.18443112075328827,-0.023148996755480766,0.009271507151424885,-0.020391836762428284,0.014609504491090775,-0.006966262124478817,-0.0026493698824197054,-0.05946456268429756,-0.006900311913341284,-0.00634230999276042,0.040444329380989075,0.014873902313411236,-0.011657400988042355,-0.0280571561306715,-0.019210409373044968,-0.0011373738525435328,0.004437817260622978,0.02734195999801159,-0.01763341575860977,0.0029308348894119263,0.020157339051365852,0.024618560448288918,0.005358884576708078,0.014323997311294079,0.0029505817219614983,-0.005110694095492363,0.6496252417564392,-0.053306080400943756,-0.0058554974384605885,-0.01981886848807335,-0.008071924559772015,-0.006869315169751644,0.011915079317986965,0.01261670608073473,-0.019067028537392616,-0.004722272977232933,-0.05077458918094635,-0.046526990830898285,0.020257316529750824,0.013736839406192303,0.002608766546472907,-0.0034607655834406614,-0.018692856654524803,-0.014540831558406353,0.044487159699201584,-0.056308597326278687,-0.03237638995051384,0.004933377727866173,0.018158087506890297,0.021088099107146263,-0.04402942582964897,-0.029466865584254265,0.0105899628251791,0.05357479676604271,-0.05160846561193466,-0.016380205750465393,-0.010790468193590641,0.024171648547053337,0.013854079879820347,0.03144305944442749,0.02881663851439953,0.031199457123875618,-0.028395550325512886,0.01117578987032175,0.019122391939163208,-0.009040268138051033,0.022094037383794785,-0.012739963829517365,0.00017399451462551951,-0.028970466926693916,0.01716982014477253,-0.004101032391190529,-0.03627597913146019,-0.01965869776904583,0.02041907235980034,-0.015489906072616577,-0.012333795428276062,0.019006699323654175,0.005117892753332853,0.016836678609251976,-0.02589852176606655,0.01113149244338274,-0.02863095887005329,0.004759848117828369,-0.05533807352185249,-0.013764551840722561,-0.029555510729551315,0.043936941772699356,-0.026340026408433914,-0.008651661686599255,-0.019796498119831085,-0.02661082334816456,-0.0677989050745964,-0.00811520405113697,0.04509684816002846,-0.0015024503227323294,0.015820348635315895,0.023989910259842873,-0.030045341700315475,0.02508663572371006,0.009965837001800537,0.003464705077931285,-0.011840038001537323,0.0198811162263155,-0.03810553252696991,0.01861695945262909,-0.032312821596860886,0.019130485132336617,0.04861173406243324,-0.033806804567575455,0.013468412682414055,-0.00406223488971591,0.033424291759729385,0.04291091859340668,-0.01149976346641779,0.016552245244383812,0.0069789523258805275,0.025855500251054764,-0.009393014945089817,-0.007661312352865934,0.05473153665661812,0.025690428912639618,-0.036358196288347244,0.010988211259245872,0.010048598982393742,-0.0058916471898555756,-0.005367740988731384,0.048110589385032654,-0.0007429873221553862,0.023407895117998123,-0.020141126587986946,-0.014089311473071575,-0.014083907008171082,-0.015959562733769417,-0.02766922116279602,-0.015350828878581524,0.0020960834808647633,-0.0032107033766806126,0.0037848034407943487,-0.011974534951150417,0.035355035215616226,0.037306368350982666,-0.0077398004941642284,-0.005957275163382292,0.000970572407823056,0.016372399404644966,0.013819240964949131,0.010282308794558048,0.01615927368402481,-0.008819358423352242,-0.026662444695830345,-0.017539484426379204,0.012908521108329296,0.004405930172652006,0.008539012633264065,-0.000672093010507524,-0.028431832790374756,0.006341465283185244,-0.01010969653725624,0.01827351562678814,-0.02010517567396164,-0.0162906926125288,-0.014747546054422855,-0.00887925736606121,0.04651108384132385,-0.003932764753699303,0.009381836280226707,-0.024245109409093857,-0.017034199088811874,-0.03038940392434597,-0.0013200266985222697,-0.004807321820408106,-0.020380636677145958,-0.018323810771107674,-0.008138819597661495,-0.015519209206104279,-0.024748897179961205,0.007331625558435917,-0.007141429465264082,-0.013838447630405426,-0.019334133714437485,-0.005408428609371185,0.03597620502114296,-0.010720993392169476,-0.0034316780511289835,-0.02497507818043232,-0.001224928768351674,0.02395603433251381,-0.03572462499141693,-0.03453918173909187,-0.033510107547044754,-0.006008330732584,0.006822410970926285,0.03696754574775696,0.015473157167434692,0.01460866630077362,-0.0007411232218146324,0.07114441692829132,-0.0010755411349236965,-0.015285762026906013,0.01705975830554962,0.004958299919962883,-0.04508613049983978,0.011662159115076065,0.0025561910588294268,-0.005702427122741938,0.013844580389559269,-0.01892123930156231,-0.011532346718013287,-0.008036554791033268,-0.0011347347171977162,-0.01549521740525961,-0.03018491342663765,0.024034442380070686,0.024171337485313416,-0.0015363524435088038,-0.004312901291996241,-0.041096772998571396,-0.004297324921935797,0.02075061947107315,0.028171395882964134,-0.030296403914690018,0.02048708125948906,0.012012072838842869,0.03347966820001602,-0.020060598850250244,-0.014894988387823105,0.02002022974193096,-0.02158072404563427,-0.033770836889743805,0.021192878484725952,-0.0018208179390057921,0.0009991482365876436,-0.044341955333948135,-0.011937808245420456,0.006897172890603542,-0.04503057897090912,-0.03995048627257347,-0.004204429220408201,0.021579941734671593,-0.028874646872282028,0.03665200620889664,0.03300997614860535,-0.006059388630092144,-0.0018951641395688057,-0.010933570563793182,0.010337389074265957,-0.03216280788183212,0.019717233255505562,0.017239345237612724,-0.025499241426587105,0.030989864841103554,-0.008808952756226063,0.015054614283144474,-0.008688508532941341,0.018388424068689346,-0.021100683137774467,0.01686527207493782,-0.011277221143245697,0.012690097093582153,-0.005680461414158344,0.01244433131068945,-0.02932148240506649,0.013947572559118271,0.008459887467324734,-0.0000801066926214844,0.002725276630371809,-0.00300898146815598,0.038368478417396545,-0.02121691033244133,0.008453425951302052,-0.0574941411614418,-0.0030004887375980616,-0.017873194068670273,-0.011487414129078388,0.021161673590540886,0.002342820866033435,-0.005630783271044493,0.017525827512145042,0.012779763899743557,-0.008308114483952522,0.007321945857256651,-0.02929837629199028,0.007987956516444683,-0.003551504109054804,0.04633054509758949,0.011856377124786377,-0.008557242341339588,-0.01718260534107685,0.019947312772274017,0.023221932351589203,0.013549587689340115,0.01817863993346691,0.04516642913222313,0.0033165987115353346,-0.0007992511964403093,-0.025754224509000778,0.02426086738705635,0.007627000566571951,0.018178611993789673,0.012401783838868141,-0.004206185694783926,0.008747300133109093,0.004693929571658373,-0.021600430831313133,-0.02287721261382103,0.01350963395088911,0.0053276680409908295,0.004475089721381664,0.02381257340312004,-0.019157743081450462,-0.005368090234696865,-0.03884730115532875,0.0007509654387831688,-0.003929459489881992,0.0033258332405239344,-0.01829037442803383,-0.00557937566190958,-0.03142822906374931,-0.027745844796299934,-0.028555698692798615,0.027369597926735878,-0.015915514901280403,0.02619199827313423,-0.006784170866012573,0.037110622972249985,0.010802475735545158,0.038187120109796524,-0.009386979043483734,-0.005884665530174971,-0.04339227080345154,0.02942570112645626,-0.015048210509121418,-0.0017648611683398485,-0.02051001787185669,0.0004942170926369727,0.01565675437450409,-0.043645285069942474,0.03105166181921959,0.01577756553888321,-0.03641166910529137,0.0015384277794510126,-0.18682853877544403,0.03271225094795227,0.006441325880587101,-0.008044271729886532,0.005846887361258268,-0.014891608618199825,0.017346085980534554,-0.01583673618733883,-0.028799831867218018,-0.011949403211474419,0.005177765619009733,0.009200219996273518,0.024670500308275223,0.004799164365977049,-0.010118584148585796,-0.007180654443800449,0.009807412512600422,-0.001643523690290749,0.025886712595820427,-0.02271057665348053,0.0013511634897440672,0.017349032685160637,-0.012280421331524849,0.01876072958111763,-0.004262855276465416,-0.010785785503685474,0.012419067323207855,0.015993289649486542,0.006141404155641794,-0.0486750565469265,0.01631086878478527,0.009667158126831055,0.007043206598609686,-0.030202830210328102,-0.05439183861017227,-0.007263322826474905,-0.011104041710495949,0.04070024937391281,0.009039278142154217,-0.005261984653770924,-0.0017805563984438777,-0.003699537366628647,-0.024146024137735367,-0.012978706508874893,-0.016917143017053604,0.010574422776699066,-0.0032952926121652126,-0.030544349923729897,0.003406661795452237,0.006680595222860575,-0.015289555303752422,0.003602110082283616,-0.029202651232481003,0.00042448146268725395,-0.004458544310182333,-0.006447230000048876,0.004317210055887699,0.005357617978006601,0.0019395265262573957,0.0026867224369198084,-0.0009431689977645874,-0.006399401463568211,0.010073689743876457,0.030944395810365677,0.002550299745053053,0.025951610878109932,0.003800789127126336,-0.04651292786002159,-0.048458099365234375,-0.014975139871239662,-0.002749948063865304,0.004808521363884211,0.03070972114801407,0.006722541525959969,-0.007859279401600361,0.013682885095477104,0.00957096740603447,-0.013668203726410866,-0.00016980688087642193,-0.024517782032489777,0.020281005650758743,0.03860524669289589,-0.01655668392777443,-0.05017245560884476,-0.005908954422920942,0.01387692242860794,-0.01276292186230421,-0.03422152251005173,-0.0023568233009427786,0.016376817598938942,-0.041153550148010254,-0.014757751487195492,0.03825400024652481,-0.035122163593769073,0.03596165031194687,0.038283392786979675,-0.01915409415960312,-0.014336580410599709,-0.015385901555418968,-0.029441365972161293,0.03101111762225628,0.005073732230812311,-0.006437203381210566,-0.01908678002655506,-0.002977523487061262,-0.002094009891152382,0.04122431203722954,0.004754438530653715,0.02652120590209961,0.010309034027159214,0.012997240759432316,-0.019439255818724632,0.003306824015453458,0.006698875222355127,0.005795760545879602,-0.010780135169625282,-0.023200908675789833,0.01796633005142212,0.009287238121032715,-0.003895723959431052,0.013337905518710613,0.025569044053554535,0.022346174344420433,-0.011740101501345634,0.03706115111708641,0.01642177440226078,-0.005734262056648731,-0.002154782647266984,-0.025524543598294258,-0.041741956025362015,0.005545818246901035,0.013133159838616848,-0.014538787305355072,-0.01684839278459549,0.05526084452867508,-0.010182134807109833,0.029960233718156815,0.004521056544035673,0.000838306441437453,0.022687237709760666,-0.029925795271992683,-0.009856436401605606,-0.02253301441669464,-0.009113472886383533,-0.00920250453054905,0.014004389755427837,0.029789825901389122,-0.007926435209810734,-0.021544726565480232,-0.0133292768150568,-0.008220207877457142,-0.02275729365646839,-0.052783627063035965,-0.03071376122534275,-0.0022968738339841366,0.017426472157239914,0.0038121037650853395,-0.0407782718539238,-0.00009582042548572645,0.009582215920090675,-0.030289215967059135,0.0013146387645974755,-0.035006701946258545,0.03376166895031929,-0.008873547427356243,-0.005690729711204767,-0.02085866592824459,0.023660162463784218,0.02535366639494896,-0.006353122182190418,0.0007649947656318545,-0.006274092476814985,-0.04827839881181717,-0.01785298064351082,0.010698039084672928,0.0014962840359658003,0.03554944321513176,0.012286324985325336,-0.039143819361925125,0.004069188609719276,0.018372230231761932,0.008820582181215286,-0.009328721091151237,-0.004874794743955135,-0.014218525029718876,0.019161565229296684,0.01462504081428051,0.019836289808154106,0.025463195517659187,0.009707238525152206,-0.009576020762324333,-0.0055716028437018394,-0.011893569491803646,-0.008480378426611423,0.004252410028129816,0.013349207118153572,-0.01655896194279194,-0.039842694997787476,-0.016235416755080223,-0.003012096043676138,-0.0040341513231396675,0.01641716994345188,-0.0019341664155945182,0.02344946376979351,0.015591164119541645,-0.0017627474153414369,0.01982186734676361,0.014068963937461376,-0.016694217920303345,0.021293651312589645,0.004868016578257084,-0.021520724520087242,-0.018620682880282402,-0.01744752563536167,0.01813305914402008,-0.008659204468131065,-0.009580496698617935,0.005942412186414003,-0.00136253098025918,-0.01846194826066494,0.0020596617832779884,-0.039835125207901,-0.002534691244363785,-0.032292310148477554,0.013045907020568848,0.024678610265254974,-0.023161306977272034,0.04890305921435356,-0.004793909378349781,0.0038888126146048307,-0.02832169272005558,-0.02261134423315525,-0.06501864641904831,-0.004058612510561943,0.03268272429704666,0.010768147185444832,-0.008082466199994087,-0.0017423891695216298,-0.041671812534332275,0.009975595399737358,-0.00382527569308877,-0.025365419685840607,0.009978887625038624,-0.04618224874138832,-0.009343093261122704,-0.03074515052139759,-0.021165281534194946,-0.001912703737616539,-0.010987833142280579,0.015884140506386757,0.036444295197725296,-0.015434290282428265,0.014077482745051384,-0.011205187067389488,0.030951518565416336,0.03389952704310417,-0.029726112261414528,-0.0016963136149570346,-0.02061229571700096,-0.0139729343354702,0.04218011349439621,-0.011218545027077198,-0.004003942012786865,0.01652691140770912,-0.007812418509274721,0.03990553691983223,-0.03662434220314026,-0.010878518223762512,-0.023101497441530228,0.024742165580391884,0.013572390191257,0.01166819129139185,-0.006664498709142208,0.04057473689317703,0.02378370426595211,0.009193843230605125,0.009010028094053268,-0.0010899485787376761,-0.029528630897402763,0.05595232546329498,-0.0066804904490709305,-0.0021086069755256176,-0.03645811975002289,-0.02671816200017929,0.0075734639540314674,0.022725099697709084,0.014151963405311108,-0.006069105118513107,-0.027776548638939857,-0.0017719474853947759,0.022220611572265625,0.01029113307595253,0.03194112330675125,0.02617032639682293,-0.013851424679160118,-0.0008156702970154583,0.038775257766246796,-0.013746884651482105,-0.007707295008003712,-0.03796851634979248,-0.004138441290706396,0.028397297486662865,-0.0010368649382144213,-0.008697853423655033,-0.00862084235996008,0.012430219911038876,0.00206479849293828,-0.006794648244976997,-0.009484518319368362,-0.014684796333312988,-0.025407856330275536,0.021318815648555756,0.018594937399029732,-0.015070969238877296,-0.021831439808011055,-0.019772548228502274,0.031087186187505722,-0.025306066498160362,-0.02949276752769947,-0.0328788198530674,-0.0320521742105484,-0.011163540184497833,0.02418673224747181,-0.0032977017108350992,-0.014361183159053326,-0.026074668392539024,-0.007526756729930639,0.014066735282540321,0.005600559059530497,0.029861778020858765,-0.013118032366037369,-0.01834111101925373,0.002682792954146862,-0.0009973339037969708,-0.030452396720647812,-0.003517824225127697,-0.01721714250743389,0.012537650763988495,0.03137042373418808,-0.02563992515206337,-0.014541576616466045,-0.029647422954440117,-0.03872761130332947,0.0124127846211195,0.017298279330134392,-0.01542678289115429,0.023507999256253242,0.009598481468856335,0.01405521109700203,-0.021878115832805634,-0.014565207064151764,0.009759897366166115,-0.00893075205385685,0.009577925316989422,0.020548827946186066,0.0009061423479579389,0.0004889803822152317,0.020842552185058594,0.0028179590590298176,-0.028958793729543686,-0.03639044985175133,-0.01715780794620514,0.0032596688251942396,-0.02279387041926384,0.013053672388195992,0.026006784290075302,-0.007702230010181665,0.032267000526189804,-0.0017689288360998034,-0.04426606371998787,0.027218639850616455,-0.035268381237983704,-0.03934603929519653,0.01837988942861557,-0.03718952834606171,0.029364554211497307,0.00939704105257988,-0.01175576914101839,-0.011473660357296467,-0.028851402923464775,-0.023649366572499275,-0.032468169927597046,0.01531847845762968,-0.012807006016373634,0.007838662713766098,-0.015825774520635605,0.004559505730867386,-0.01605060324072838,-0.006479652598500252,0.0012571568368002772,0.0008885100251063704,-0.01844359003007412,0.012507060542702675,0.014665930531919003,0.013433980755507946,-0.02287564054131508,-0.05416969209909439,-0.03761182725429535,0.022247064858675003,0.00992788840085268,-0.025322325527668,0.00988700706511736,-0.005227380432188511,-0.026791758835315704,0.023695098236203194,-0.000014328586075862404,-0.013037673197686672,-0.01907210424542427,0.016273565590381622,-0.021701635792851448,-0.0021066220942884684,0.03337745741009712,-0.013645888306200504,0.0018892678199335933,-0.005045998375862837,0.042833685874938965,-0.01615271344780922,0.04580359905958176,-0.0223141610622406,0.014279637485742569,0.02616218850016594,-0.015080823563039303,0.01668858341872692,-0.013901581056416035,0.009553187526762486,0.025403399020433426,0.0052113644778728485,-0.014745769090950489,-0.0009014360257424414,0.004632167983800173,0.024682113900780678,0.03319165110588074,-0.033311035484075546,-0.006520306225866079,0.01641225256025791,0.05087489262223244,0.011769518256187439,-0.0068300398997962475,0.0040258122608065605,-0.012988467700779438,0.034983839839696884,-0.017945125699043274,-0.013408321887254715,-0.02442520298063755,0.04357581213116646,-0.056211747229099274,0.009304540231823921,-0.005600585136562586,-0.03652577102184296,0.02592485398054123,-0.008577843196690083,-0.014812407083809376,0.0018684475217014551,0.005596611183136702,-0.02698114700615406,0.008817252703011036,-0.009420663118362427,-0.016018936410546303,-0.007171689998358488,-0.023529503494501114,0.04592137411236763,-0.025283947587013245,0.018765371292829514,-0.03229653462767601,0.009904840029776096,0.017309457063674927,0.0005982531001791358,-0.017608163878321648,0.010085015557706356,0.009459641762077808,-0.014238577336072922,0.015561285428702831,0.01512686163187027,-0.016820671036839485,0.009932256303727627,0.006123207975178957,-0.019477643072605133,-0.014218680560588837,0.0034777051769196987,0.02694743499159813,0.014948295429348946,0.007310180924832821,-0.01140064187347889,0.02155054546892643,-0.0031535422895103693,0.02321101352572441,0.03501196205615997,0.016104355454444885,-0.011443550698459148,0.020494714379310608,0.000037473870179383084,0.030133651569485664,0.013036134652793407,-0.010131465271115303,-0.03145812451839447,0.006029221694916487,-0.018738742917776108,-0.0026733726263046265,-0.0069665201008319855,0.019513335078954697,0.006418284960091114,0.011988451704382896,-0.020809510722756386,0.004837896674871445,0.006817515939474106,-0.007561174221336842,-0.01496717520058155,-0.005628278013318777,-0.0151112275198102,-0.02160394750535488,0.021545330062508583,-0.011424845084547997,0.04252980276942253,-0.028489435091614723,0.021204529330134392,-0.061839692294597626,0.014390102587640285,-0.008755280636250973,0.026408012956380844,-0.006401140242815018,0.017197787761688232,-0.0370585173368454,0.034825924783945084,-0.006772985681891441,-0.02114637941122055,-0.02247908152639866,0.007700352463871241,-0.03821778669953346,0.00017269796808250248,-0.03410832956433296,0.019855795428156853,0.0009135074215009809,0.035855021327733994,-0.016612650826573372,-0.040418609976768494,0.0028196251951158047,0.0040102992206811905,-0.04908296465873718,0.012688972987234592,-0.003799975384026766,0.006731805857270956,-0.03011994995176792,-0.03930181637406349,0.006181399803608656,-0.007794621866196394,-0.017922749742865562,-0.0012883433373644948,-0.04030757397413254,-0.01805681362748146,0.005565496627241373,-0.0047272671945393085,-0.014569416642189026,-0.008235592395067215,-0.003890547202900052,-0.020433317869901657,0.02066405490040779,-0.005242255982011557,-0.00019520313071552664,-0.03202907368540764,-0.029797418043017387,-0.00018843963334802538,0.016573864966630936,-0.04733728989958763,-0.0244253259152174,-0.0031971873249858618,0.014172783121466637,-0.002092082519084215,-0.005944475065916777,0.014103066176176071,0.027422163635492325,0.009740452282130718,-0.055597104132175446,-0.024621648713946342,-0.023668251931667328,-0.0015227218391373754,0.00628467695787549,0.02427695132791996,-0.029232285916805267,-0.005822507664561272,0.005340541712939739,0.017792735248804092,-0.0404917411506176,0.01850849948823452,-0.02137850783765316,0.032680269330739975,-0.04013790190219879,-0.009750901721417904,0.011691272258758545,-0.10531952232122421,0.008833343163132668,0.02102864719927311,0.01059207133948803,-0.005922437179833651,-0.000013196819054428488,-0.022282278165221214,-0.00016967281408142298,-0.026948388665914536,-0.027980580925941467,0.021381018683314323,-0.02278803288936615,-0.011419698596000671,-0.006655302830040455,-0.01766994222998619,-0.016247810795903206,0.0005969391786493361,0.030986133962869644,0.026341672986745834,-0.008760204538702965,0.0017293060664087534,-0.003207669360563159,-0.035950031131505966,0.01429867185652256,0.005979999899864197,-0.012309896759688854,-0.01963735744357109,-0.008175088092684746,0.006148855201900005,0.01289496198296547,0.04852227866649628,-0.012619220651686192,-0.0026282796170562506],"index":0},{"object":"embedding","embedding":[-0.004285297356545925,-0.014564486220479012,-0.016580479219555855,0.016443807631731033,0.004217916633933783,-0.015437845140695572,-0.006158899050205946,-0.019124578684568405,-0.006950558628886938,0.012497621588408947,0.0351058691740036,0.008919301442801952,0.035629112273454666,-0.0023913895711302757,0.007267395965754986,0.011561121791601181,-0.01238621398806572,-0.011214325204491615,-0.015068083070218563,-0.022773437201976776,0.010673204436898232,0.044169772416353226,-0.0039693983271718025,-0.015642477199435234,-0.021421419456601143,0.012200327590107918,0.008789672516286373,0.019884992390871048,0.03242874518036842,-0.012119706720113754,-0.009400391951203346,0.03798016160726547,0.015466639772057533,0.006101908162236214,0.014775496907532215,0.02007640339434147,-0.016790490597486496,0.004852014128118753,0.013239431194961071,0.011806187219917774,-0.017249641939997673,0.0033052500803023577,0.03879188746213913,-0.02303340472280979,-0.0277368426322937,-0.03020656481385231,-0.02082614041864872,-0.03701325133442879,0.004458844196051359,0.021700605750083923,-0.003344543045386672,-0.0331253781914711,-0.020582376047968864,0.05624222010374069,-0.035694669932127,0.013974400237202644,0.003263092366978526,-0.005511644762009382,-0.014846364967525005,-0.02936599962413311,-0.009299523197114468,0.19663512706756592,-0.019456950947642326,0.011017006821930408,-0.012154284864664078,0.031545598059892654,-0.006233204156160355,-0.020257527008652687,-0.02846020832657814,-0.013629027642309666,-0.011972113512456417,0.005676695145666599,0.008949648588895798,-0.01061919890344143,0.0032657296396791935,-0.020064225420355797,-0.02201222814619541,-0.02192266285419464,0.050561849027872086,0.0028951717540621758,-0.023434070870280266,0.02147388458251953,0.014437002129852772,-0.02084781974554062,-0.01056731678545475,-0.0031143685337156057,0.01706208847463131,0.6640109419822693,-0.06142793968319893,-0.009754650294780731,-0.013139267452061176,-0.028979729861021042,-0.011413425207138062,0.0024580468889325857,-0.0210629403591156,-0.009800789877772331,-0.010361839085817337,-0.039458002895116806,-0.04414813965559006,-0.009862912818789482,0.039889171719551086,-0.00729469396173954,-0.03144644573330879,-0.024813562631607056,-0.004071842413395643,0.02081509307026863,-0.03137628734111786,-0.014005686156451702,-0.007677961140871048,0.018851477652788162,-0.006114735268056393,-0.056475453078746796,0.011689091101288795,-0.011373203247785568,0.007448793854564428,-0.039242036640644073,0.007213531527668238,-0.014041850343346596,0.024084540084004402,-0.008465197868645191,0.023254621773958206,0.008521183393895626,0.00899258442223072,-0.03143690526485443,0.013184809125959873,0.025586312636733055,-0.022307634353637695,0.050704192370176315,-0.0009345149737782776,-0.0012701196828857064,-0.002420233329758048,0.017340997233986855,-0.007981940172612667,-0.01048630103468895,-0.014014068059623241,-0.013617749325931072,0.028746429830789566,0.00424461392685771,0.0015173522988334298,-0.0012624065857380629,0.0006192005821503699,-0.006423024460673332,0.013799447566270828,-0.025131532922387123,0.01045960746705532,-0.012492459267377853,-0.02260611020028591,0.021308323368430138,0.0036427201703190804,-0.03241097927093506,0.004548081196844578,0.0031649810262024403,0.0028049550019204617,-0.0182278361171484,0.007720143999904394,0.054627809673547745,0.0007530491566285491,0.01669897697865963,0.005791725590825081,-0.021497447043657303,-0.010770023800432682,-0.022960832342505455,-0.03926060348749161,-0.0028847239445894957,0.0018401116831228137,-0.003583054058253765,0.013903351500630379,-0.04713357985019684,0.0013615776551887393,0.002380107529461384,0.009916814044117928,0.01951373554766178,-0.013420704752206802,0.015164395794272423,0.02784680761396885,-0.005289891269057989,0.002555672312155366,0.009907773695886135,0.02710774540901184,0.01775195077061653,0.007064831908792257,0.048922691494226456,0.0004531689337454736,-0.02901897020637989,0.022401420399546623,-0.022551216185092926,-0.000843644724227488,-0.004428130574524403,0.03575357049703598,0.0009244136745110154,0.042618829756975174,-0.009692701511085033,0.01043030433356762,0.0038911611773073673,-0.018474670127034187,-0.02080972120165825,-0.019145945087075233,0.025530647486448288,-0.004237101413309574,0.018756091594696045,-0.01180137600749731,0.02230081334710121,0.023273218423128128,-0.0391106940805912,-0.017901204526424408,0.020478615537285805,0.008464116603136063,0.009504350833594799,-0.012657145038247108,0.02349432185292244,-0.021555786952376366,-0.0019459519535303116,-0.030501462519168854,-0.0017687676008790731,0.015922917053103447,0.04266855865716934,0.010693078860640526,-0.017475955188274384,-0.021369535475969315,0.009963973425328732,0.04677752032876015,0.0024647170212119818,-0.014198452234268188,-0.00824415497481823,0.002477082656696439,0.024890942499041557,0.006369042210280895,0.00020600203424692154,-0.010448234155774117,-0.0023461419623345137,0.007621110882610083,0.03437066450715065,-0.03733120858669281,-0.03150142729282379,0.002128505613654852,0.004461865406483412,0.021113652735948563,-0.009577988646924496,-0.034292206168174744,-0.003404168179258704,0.00008232207619585097,-0.018383057788014412,-0.012374766170978546,-0.010677404701709747,-0.028470903635025024,-0.021290645003318787,-0.0031482786871492863,-0.006115273106843233,0.006543636322021484,-0.0007899189949966967,-0.00857260636985302,-0.03981125354766846,-0.0280560664832592,0.003977597691118717,0.0579090379178524,-0.015463583171367645,0.02602853812277317,-0.018383540213108063,0.014274277724325657,0.02344614639878273,-0.006901269778609276,0.0043588485568761826,0.025912975892424583,-0.03577928990125656,0.047188080847263336,-0.002287505427375436,-0.013656373135745525,-0.0022315282840281725,-0.037634190171957016,-0.012077958323061466,0.03787172958254814,-0.000589891045819968,0.006841442547738552,-0.05457588657736778,-0.013828745111823082,0.03929060697555542,-0.010475962422788143,-0.003952181898057461,-0.012839345261454582,-0.013511652126908302,-0.014495810493826866,0.009007317945361137,-0.04348411038517952,0.009792719967663288,-0.010241426527500153,0.009871255606412888,0.007920924574136734,-0.030454762279987335,0.002537030028179288,-0.009922289289534092,-0.03304130211472511,0.010475628077983856,-0.008054936304688454,0.037432774901390076,-0.02610715478658676,-0.012445738539099693,0.02444445714354515,-0.03828082233667374,-0.03207232803106308,-0.012661836110055447,0.0030418329406529665,-0.018977487459778786,-0.021577944979071617,0.024432986974716187,-0.007854047231376171,0.005464739631861448,-0.016096334904432297,0.00376148265786469,-0.01685277186334133,0.006472278852015734,-0.013275789096951485,-0.02851736731827259,0.012032454833388329,-0.0019194848136976361,0.045898307114839554,-0.028387319296598434,0.016325538977980614,-0.005413290578871965,-0.005501685664057732,-0.003401385620236397,-0.009151612408459187,0.005875143222510815,0.02995760180056095,-0.009996180422604084,0.004154312424361706,0.012755542062222958,0.0041709179058671,0.0008314028382301331,-0.02778591401875019,-0.010996638797223568,-0.011812896467745304,0.016545087099075317,-0.033632829785346985,-0.017562076449394226,-0.028860215097665787,0.011606150306761265,-0.0010538806673139334,-0.02621433138847351,-0.018384141847491264,0.050438541918992996,-0.005153534468263388,-0.015172770246863365,-0.03160208463668823,0.004491359461098909,0.010390433482825756,-0.024569563567638397,0.018246639519929886,0.03956255316734314,-0.01111405435949564,-0.0005722676869481802,0.04335317015647888,0.015209430828690529,0.028309915214776993,0.006345074158161879,0.00763111375272274,0.004354933276772499,-0.007664976641535759,-0.03524396941065788,0.006746530067175627,-0.030275514349341393,0.03031207248568535,0.027718152850866318,0.04550035297870636,-0.009212451986968517,-0.007164254318922758,-0.009993841871619225,-0.02746083214879036,0.019127987325191498,-0.012546233832836151,-0.023727668449282646,0.006587502546608448,-0.024809980764985085,0.0141473188996315,-0.013625837862491608,-0.012030841782689095,0.001633543404750526,0.008806606754660606,-0.0029283168260008097,0.002919907448813319,-0.009159025736153126,-0.01866384781897068,-0.04996678978204727,0.020794417709112167,0.013871634379029274,0.013637245632708073,0.014808045700192451,0.011251740157604218,0.026380857452750206,0.03816097974777222,0.00008325099770445377,-0.00430482579395175,-0.03483356162905693,0.055521171540021896,-0.011507807299494743,-0.021945221349596977,-0.02403912879526615,-0.024212120100855827,0.008661230094730854,-0.01638766936957836,0.028347833082079887,-0.008635062724351883,-0.016870172694325447,-0.03763468191027641,-0.20208165049552917,0.027341028675436974,-0.002945993561297655,0.0035014082677662373,0.004758790601044893,-0.01142621785402298,0.035212136805057526,-0.003012634813785553,-0.02983098104596138,0.012239260599017143,-0.011561859399080276,-0.013696473091840744,0.006615175865590572,0.0007685653981752694,0.023931700736284256,-0.04886975511908531,0.015155188739299774,-0.039185937494039536,0.026939528062939644,-0.0030653858557343483,-0.008654006756842136,0.011049889959394932,0.0007772246026434004,0.017968233674764633,-0.002294074511155486,-0.03704323247075081,-0.009562411345541477,0.0013991565210744739,0.01862112060189247,-0.009166751056909561,0.028347564861178398,0.005998472683131695,0.007365141995251179,-0.012594997882843018,-0.05549926310777664,0.010497687384486198,-0.03980746865272522,0.01720789261162281,0.024997714906930923,-0.025926917791366577,0.004801096394658089,-0.033253345638513565,0.00344124436378479,-0.00523914210498333,-0.004632190335541964,-0.031936343759298325,-0.013963598757982254,-0.00831675436347723,0.008605812676250935,0.008931395597755909,-0.04545494541525841,0.00015152715786825866,-0.04069611802697182,-0.0008555970271117985,0.004400073550641537,0.012143315747380257,0.0070645990781486034,0.0047237626276910305,0.0017272194381803274,0.011750890873372555,0.006575545761734247,-0.018963251262903214,0.0007710521458648145,0.02249409630894661,0.008462783880531788,0.00477238604798913,0.0016029777470976114,-0.03513152524828911,-0.02871382236480713,0.0023913830518722534,-0.017433786764740944,0.00119790097232908,0.028445323929190636,-0.013209926895797253,0.012617296539247036,0.028333215042948723,0.01438191533088684,-0.013831940479576588,-0.017087753862142563,-0.013826519250869751,0.016261164098978043,0.015808813273906708,-0.0033666789531707764,-0.016928475350141525,-0.025441525503993034,0.017038721591234207,-0.014180978760123253,0.006150401197373867,-0.011126064695417881,0.0289481021463871,-0.02729945257306099,-0.009256374090909958,0.014649285934865475,0.010429946705698967,0.009939033538103104,0.054632559418678284,-0.023171652108430862,0.018928486853837967,-0.0033703099470585585,-0.005358698777854443,0.039696455001831055,0.019350869581103325,-0.019738400354981422,-0.040208470076322556,0.008610597811639309,0.005552174989134073,0.04277408495545387,0.03762585669755936,-0.004798519425094128,-0.005145879462361336,0.013238409534096718,-0.023252611979842186,-0.007956072688102722,-0.007360454648733139,-0.006025645416229963,0.0019996999762952328,-0.023733634501695633,0.03507727384567261,0.024928174912929535,-0.0094306580722332,0.03118046000599861,0.033407676964998245,-0.009490770287811756,-0.016119850799441338,0.015381553210318089,-0.008191749453544617,0.0008451060857623816,0.0026274705305695534,0.004227403085678816,-0.0269999448210001,-0.012398885563015938,0.007053732872009277,0.01273365318775177,0.025427738204598427,0.0011117614340037107,0.009466675110161304,0.00901126954704523,-0.01754252426326275,-0.00019764728494919837,0.024138011038303375,-0.032829221338033676,-0.03675679489970207,-0.012212643399834633,-0.007770919241011143,0.008708507753908634,0.00558779202401638,0.039335306733846664,0.00006601445784326643,0.01122430618852377,-0.02178182266652584,-0.015776459127664566,0.0054691568948328495,-0.01675081066787243,-0.03336552157998085,0.0166954156011343,0.0024392888881266117,0.003997897729277611,-0.002229025587439537,0.015642855316400528,0.001834203489124775,-0.033062949776649475,0.03803792595863342,-0.01253955066204071,-0.008650057017803192,-0.011906759813427925,-0.009489190764725208,0.014507987536489964,-0.029505277052521706,0.02621149830520153,-0.015256255865097046,0.007301995065063238,0.01432863064110279,-0.036672890186309814,-0.026238076388835907,0.030241230502724648,-0.015589285641908646,0.00874402653425932,0.023671485483646393,-0.03277534991502762,0.007715675979852676,0.02306244894862175,0.011130577884614468,0.03420516848564148,0.0025111325085163116,0.02822883613407612,0.02705836109817028,0.030841918662190437,0.024715416133403778,0.023459019139409065,0.026125172153115273,0.0022017727605998516,0.00509035587310791,-0.0043801055289804935,-0.019084477797150612,-0.03443866968154907,-0.012868187390267849,-0.005454624071717262,-0.013477494940161705,-0.01139871310442686,0.004025402013212442,-0.02896619401872158,0.013030430302023888,-0.0314362533390522,0.017526881769299507,0.00688193691894412,0.0015910121146589518,-0.004511791281402111,-0.0047766980715096,0.010657932609319687,0.025815216824412346,0.00406211894005537,-0.010275744833052158,0.014373905956745148,0.016398504376411438,0.0132300378754735,-0.003038227092474699,-0.020517529919743538,-0.0011860469821840525,-0.01791244186460972,-0.029571333900094032,-0.026921171694993973,-0.024518650025129318,-0.003963025286793709,-0.03323286399245262,0.029462462291121483,0.023246020078659058,-0.03247830644249916,0.032463159412145615,0.006981177255511284,-0.013332989998161793,-0.035614851862192154,-0.028454722836613655,-0.048840075731277466,-0.002592537784948945,0.03852313756942749,0.03448646515607834,-0.000010741460755525623,-0.010983542539179325,-0.02426745370030403,0.03015029802918434,-0.03640568256378174,-0.015703972429037094,-0.0120193837210536,-0.03578052297234535,-0.004227074328809977,-0.04714713990688324,-0.008670351468026638,-0.023414140567183495,-0.00647988636046648,0.023887991905212402,-0.017443330958485603,-0.009883265011012554,0.004583965055644512,0.0045952401123940945,0.004000450484454632,0.020097874104976654,-0.03839832916855812,0.004807732533663511,-0.013137241825461388,0.006424373481422663,0.013701317831873894,-0.02419520914554596,-0.000354949472239241,-0.0010745523031800985,-0.012133711017668247,0.06865239888429642,-0.03662195801734924,-0.004433418624103069,-0.0320836678147316,-0.006161244120448828,-0.007129179313778877,0.019383039325475693,0.018517648801207542,0.03318283334374428,0.01993001624941826,-0.03151266276836395,0.01657181605696678,-0.04769771173596382,-0.023217162117362022,0.03115389496088028,0.0037344854790717363,0.019596990197896957,-0.026819203048944473,-0.010630406439304352,-0.01917910948395729,0.007786108180880547,-0.008129295893013477,-0.013014078140258789,-0.025056708604097366,-0.001359487883746624,0.02083597704768181,-0.0024170821998268366,0.026222048327326775,0.003635784611105919,-0.00789398979395628,0.0015712741296738386,0.0362250842154026,-0.031276918947696686,-0.006678259000182152,-0.03371664509177208,-0.008746802806854248,0.015244794078171253,0.0018131999531760812,0.005407710559666157,-0.0005441837129183114,-0.00586351752281189,-0.006155264098197222,-0.014328676275908947,0.003661924507468939,0.002441331511363387,-0.025601623579859734,-0.012678084895014763,-0.006185202859342098,-0.03537547588348389,-0.0033379066735506058,-0.009049040265381336,0.011929714120924473,-0.036004748195409775,-0.02195044234395027,-0.03921817988157272,0.0024661284405738115,0.015271213836967945,-0.01674790307879448,-0.010218728333711624,-0.0052257198840379715,0.009074671193957329,0.03286176919937134,-0.02335970848798752,0.01016333419829607,0.0442512147128582,-0.020692570134997368,-0.013103711418807507,0.0056952862069010735,-0.021073592826724052,-0.039438072592020035,-0.006213201675564051,-0.042385492473840714,0.007222500629723072,0.015634113922715187,-0.041070736944675446,0.0009582927450537682,-0.03664640709757805,-0.04938074201345444,0.001490876660682261,0.01197650283575058,-0.001951630343683064,0.015785934403538704,0.01437116228044033,0.025310048833489418,-0.03669779375195503,-0.019031353294849396,0.009278659708797932,0.007139967288821936,-0.008264263160526752,0.03240935504436493,-0.0036554262042045593,0.0037953818682581186,0.011005212552845478,0.007403810974210501,-0.01439606212079525,-0.02725701965391636,0.014846962876617908,-0.007086945232003927,-0.014345903880894184,-0.009519629180431366,-0.015709487721323967,0.001809190260246396,0.03050282783806324,-0.0015988610684871674,-0.03703836724162102,0.021602381020784378,-0.026087500154972076,-0.026468560099601746,0.02036127820611,-0.013567350804805756,0.019908210262656212,-0.016476161777973175,-0.0070074936375021935,-0.004955696873366833,-0.03437802195549011,0.0014351793797686696,0.0015881286235526204,0.009430079720914364,0.014634589664638042,0.02186407335102558,-0.03837528079748154,0.010351221077144146,-0.016911588609218597,-0.014886453747749329,0.013385550118982792,0.010058059357106686,-0.010260870680212975,0.020600154995918274,0.0007443947251886129,0.007274840027093887,-0.02829049341380596,-0.03897042199969292,-0.04904303327202797,0.012711732648313046,0.007413977757096291,-0.03721877560019493,0.012386798858642578,-0.021901436150074005,-0.01706158183515072,-0.006582002621144056,-0.03286066651344299,0.003577388823032379,-0.0116258654743433,0.033767685294151306,-0.009672567248344421,0.01853187382221222,0.017301080748438835,-0.012196091935038567,0.0007180116372182965,-0.013477363623678684,0.025062477216124535,-0.030618542805314064,0.058702465146780014,0.0029667892958968878,0.015435351058840752,0.015869298949837685,-0.017837613821029663,-0.0013025450753048062,0.0006691031157970428,0.0012306577991694212,-0.005210723262280226,0.01562783122062683,-0.026647359132766724,-0.033227283507585526,0.0034284016583114862,0.041636619716882706,-0.007551092654466629,-0.015632979571819305,-0.022543368861079216,0.021474895998835564,0.015658073127269745,0.023935707286000252,-0.000008678483936819248,0.01143769733607769,0.022784100845456123,0.0193592831492424,0.0005993618979118764,0.015519926324486732,-0.021937448531389236,0.016499070450663567,-0.03145519644021988,0.003398893168196082,-0.03416622057557106,-0.03869443014264107,0.01505317073315382,-0.0025696989614516497,0.008020944893360138,0.013248646631836891,-0.001916668494231999,-0.02654431201517582,0.0005295672453939915,0.014294037595391273,-0.021158931776881218,-0.021527625620365143,-0.02215154841542244,0.035656727850437164,0.0029146927408874035,-0.0024859781842678785,-0.020434454083442688,-0.02422427013516426,0.03750944137573242,0.006173830013722181,-0.02485613524913788,0.03312735632061958,0.026846840977668762,-0.0203867107629776,-0.009074349887669086,0.016166184097528458,-0.02515099197626114,0.01875888556241989,0.0024134088307619095,-0.0027398590464144945,-0.02792530134320259,0.011878864839673042,0.02505829744040966,0.00039012328488752246,-0.006761571858078241,-0.01600414514541626,-0.0015294201439246535,-0.023948417976498604,-0.01289227046072483,0.04074708744883537,0.023160865530371666,-0.024107523262500763,0.011518262326717377,-0.01088558230549097,0.03635071963071823,0.010532603599131107,-0.008299595676362514,-0.0685909166932106,0.010029605589807034,0.0004576777864713222,0.016736051067709923,-0.00915433932095766,0.028094463050365448,-0.01615842990577221,0.0001378582528559491,0.0037299012765288353,0.0008524635341018438,0.009157671593129635,0.021297553554177284,-0.016759026795625687,-0.0007924129022285342,0.002374124713242054,-0.020575031638145447,0.01547573134303093,0.004493643995374441,0.053134579211473465,-0.02813638001680374,0.02432250790297985,-0.04440009221434593,-0.0316280797123909,0.008527335710823536,0.021472688764333725,0.018138103187084198,0.00021756731439381838,-0.04883526638150215,0.0011839086655527353,0.009020301513373852,-0.02158375456929207,-0.023044917732477188,0.028649644926190376,-0.03312399983406067,-0.009188404306769371,-0.027762670069932938,-0.024221809580922127,0.00405800249427557,0.05258597433567047,-0.04477847367525101,-0.04022582247853279,0.020190922543406487,0.01400039717555046,-0.025313694030046463,-0.001916348235681653,0.005557131487876177,-0.015018303878605366,-0.025281036272644997,-0.006455676630139351,0.01639048010110855,-0.025934258475899696,-0.010504878126084805,0.010294954292476177,-0.029208291321992874,0.009740525856614113,-0.003083680523559451,0.006991423666477203,-0.0048240674659609795,0.0033262157812714577,-0.015392951667308807,-0.012937430292367935,0.015343974344432354,-0.02448674850165844,-0.029327604919672012,-0.017669834196567535,-0.04447150230407715,0.017003554850816727,0.010389355011284351,-0.06664980947971344,-0.0012785647995769978,-0.02246190421283245,0.005143077112734318,0.006811458617448807,-0.02333330176770687,-0.017299499362707138,0.028497187420725822,0.012595298700034618,-0.06986308097839355,0.009842248633503914,-0.04555542394518852,-0.008612144738435745,0.024109914898872375,0.020183617249131203,-0.020054178312420845,0.01033328752964735,0.01785575971007347,0.023270340636372566,-0.02451990731060505,0.014299891889095306,-0.008909483440220356,0.022880008444190025,-0.0392623096704483,-0.0018133302219212055,0.03437480330467224,-0.0891222432255745,0.007946223020553589,0.0067281522788107395,0.015900803729891777,-0.003918438218533993,0.0033753952011466026,0.0021835961379110813,-0.0026962822303175926,-0.0050804796628654,-0.03597750514745712,0.0003162282519042492,-0.04856378957629204,0.0006047003553248942,0.015648940578103065,0.009548251517117023,-0.009370777755975723,0.006808259058743715,0.0010861477348953485,0.001406832248903811,0.03751282021403313,0.00678780535236001,0.022363783791661263,-0.020217739045619965,0.00324359443038702,0.0008289300021715462,-0.024063166230916977,-0.029920633882284164,0.002625570399686694,0.02280960977077484,0.01854970119893551,-0.002961813472211361,0.01971779577434063,-0.009154127910733223],"index":1}],"usage":{"prompt_tokens":84,"total_tokens":84}}, - "headers": { - "Content-Type": "application/json" - } - } - }, - { - "httpRequest": { - "method": "GET", - "path": "/v1/qianfan/token/*" - }, - "httpResponse": { - "body": {"refresh_token":"25.739889c11bd3da5314feb915f7a2fbb9.315360000.2040272418.282335-111798018","expires_in":2592000,"session_key":"9mzdDAf85Y5boZ09O03b02PJ4A8E3TsMKj7vdOtV37xs\/jC6CN9MpEIvHlgjfpybyPOM7xNj86XLqFkQPLxBDxQ7LRXFm0c=","access_token":"24.7214280a4e58a8bc2550e8527fe93058.2592000.1727504418.282335-111798018","scope":"public ai_custom_qianfan_bloomz_7b_compressed ai_custom_yiyan_com ai_custom_yiyan_com_128k ai_custom_yiyan_com_adv_pro ai_custom_yiyan_com_ai_apaas ai_custom_yiyan_com_ai_apaas_lite ai_custom_yiyan_com_aquilachat_7b ai_custom_yiyan_com_bce_reranker_base ai_custom_yiyan_com_bloomz7b1 ai_custom_yiyan_com_chatglm2_6b_32k ai_custom_yiyan_com_chatlaw ai_custom_yiyan_com_codellama_7b_ins ai_custom_yiyan_com_eb_instant ai_custom_yiyan_com_eb_pro ai_custom_yiyan_com_eb_turbo_pro ai_custom_yiyan_com_eb_turbo_pro_128k ai_custom_yiyan_com_emb_bge_large_en ai_custom_yiyan_com_emb_bge_large_zh ai_custom_yiyan_com_emb_tao_8k ai_custom_yiyan_com_emb_text ai_custom_yiyan_com_ernie_3.5_8k_0701 ai_custom_yiyan_com_ernie_35_8k_0329 ai_custom_yiyan_com_ernie_35_8k_0613 ai_custom_yiyan_com_ernie_35_8k_preview ai_custom_yiyan_com_ernie_40_8k_0329 ai_custom_yiyan_com_ernie_40_8k_0613 ai_custom_yiyan_com_ernie_40_8k_beta ai_custom_yiyan_com_ernie_40_8k_preview ai_custom_yiyan_com_ernie_40_turbo_8k(2) ai_custom_yiyan_com_ernie_40_turbo_8k_preview ai_custom_yiyan_com_ernie_char_8k ai_custom_yiyan_com_ernie_char_fiction_8k ai_custom_yiyan_com_ernie_func_8k ai_custom_yiyan_com_ernie_lite_8k ai_custom_yiyan_com_ernie_novel_8k ai_custom_yiyan_com_ernie_tiny_8k ai_custom_yiyan_com_fuyu_8b ai_custom_yiyan_com_gemma_7b_it ai_custom_yiyan_com_llama2_13b ai_custom_yiyan_com_llama2_70b ai_custom_yiyan_com_llama2_7b ai_custom_yiyan_com_llama3_70b ai_custom_yiyan_com_llama3_8b ai_custom_yiyan_com_mixtral_8x7b ai_custom_yiyan_com_qf_chinese_llama_2_13b ai_custom_yiyan_com_qf_chinese_llama_2_70b ai_custom_yiyan_com_qianfan_chinese_llama_2_7b ai_custom_yiyan_com_qianfan_dynamic_8k ai_custom_yiyan_com_sd_xl ai_custom_yiyan_com_sqlcoder_7b ai_custom_yiyan_com_tokenizer_eb ai_custom_yiyan_com_xuanyuan_70b_chat ai_custom_yiyan_com_yi_34b brain_all_scope wenxinworkshop_mgr wise_adapt lebo_resource_base lightservice_public hetu_basic lightcms_map_poi kaidian_kaidian ApsMisTest_Test\u6743\u9650 vis-classify_flower lpq_\u5f00\u653e cop_helloScope ApsMis_fangdi_permission smartapp_snsapi_base smartapp_mapp_dev_manage iop_autocar oauth_tp_app smartapp_smart_game_openapi oauth_sessionkey smartapp_swanid_verify smartapp_opensource_openapi smartapp_opensource_recapi fake_face_detect_\u5f00\u653eScope vis-ocr_\u865a\u62df\u4eba\u7269\u52a9\u7406 idl-video_\u865a\u62df\u4eba\u7269\u52a9\u7406 smartapp_component smartapp_search_plugin avatar_video_test b2b_tp_openapi b2b_tp_openapi_online smartapp_gov_aladin_to_xcx","session_secret":"ea141799c04c6cdbbdf4b7ab2b04f358"}, - "headers": { - "Content-Type": "application/json" - } - } - }, - { - "httpRequest": { - "method": "POST", - "path": "/v1/doubao/embedding" - }, - "httpResponse": { - "body": {"created":1725001256,"id":"02172500125677376580aba8475a41c550bbf05104842f0405ef5","data":[{"embedding":[-1.625,0.07958984375,-1.5703125,-3.03125,-1.4609375,3.46875,-0.73046875,-2.578125,-0.66796875,1.71875,0.361328125,2,5.125,2.25,4.6875,1.4921875,-0.77734375,-0.466796875,0.0439453125,-2.46875,3.59375,4.96875,2.34375,-5.34375,0.11083984375,-5.875,3.0625,4.09375,3.4375,0.2265625,9,-1.9296875,2.25,0.765625,3.671875,-2.484375,-1.171875,-1.6171875,4.1875,2.390625,-6.90625,0.369140625,0.259765625,3.671875,-2.9375,-1.9140625,-0.71875,-1.6640625,0.29296875,0.396484375,-4.625,-1.9921875,5.15625,-1.3984375,3.015625,-3.203125,-1.453125,4,-8.75,-5.625,1.0546875,-3.28125,-1.2265625,0.287109375,2.09375,4.6875,0.1572265625,0.42578125,0.79296875,3.234375,-0.169921875,0.9296875,7.40625,-3.296875,5.53125,3.890625,0.62109375,1.1171875,-0.373046875,4.125,-2.78125,0.333984375,3.9375,4.59375,6,1.53125,-0.373046875,1.109375,-4.0625,1.96875,1.421875,0.57421875,-0.56640625,-2.390625,0.734375,1.1875,-2.859375,-6.09375,-5.96875,1.8359375,-3,0.80859375,-0.130859375,-5.3125,-2.859375,1.484375,-4.53125,-6.90625,-2.25,0.7734375,-1.2734375,1.1484375,3.421875,-3.484375,2.65625,1.3359375,1.1484375,-4.09375,-5.625,2.625,-0.283203125,-3.46875,2.3125,-0.220703125,4.21875,3.75,-0.37109375,0.9609375,7.25,-0.87890625,7.03125,2.34375,4.5,-1.609375,-6.46875,-6.125,-2.59375,2.234375,3.78125,1.3046875,-5.5,1.953125,-3.421875,-5.9375,3.25,-3.4375,-8.3125,-2.546875,3.640625,0.267578125,-0.220703125,0.294921875,-0.4140625,2.515625,-1.0546875,-5.21875,6.6875,3.640625,0.2314453125,-2.5,1,1.6640625,0.59765625,2.75,1.1328125,1.1328125,-4.96875,4.53125,-0.349609375,3.390625,-0.193359375,7.625,2.921875,-3.484375,4.1875,8.5,-1.9140625,6.3125,2.5625,3.0625,0.40234375,0.76953125,-4.78125,3.53125,-2.765625,0.1591796875,-0.1025390625,-3.875,2.203125,0.03076171875,1.765625,1.859375,2.15625,-1.2578125,-4.40625,-0.62890625,4.4375,-1.78125,2.671875,2.765625,-1.7890625,-8.3125,-0.02197265625,1.640625,-3.96875,-3.15625,2.796875,1.1875,2,1.15625,2.359375,1.3984375,4.21875,-2.953125,8.5,3.46875,3.578125,0.90625,-1.8828125,2.15625,3.921875,4.125,-0.9609375,-2.171875,2.328125,2.921875,1.9765625,1.0703125,4.03125,6.28125,-3.59375,-0.94921875,5.6875,-1.9140625,-5.1875,-4.25,-7.71875,1.7109375,-1.59375,3.765625,-5.3125,-3.9375,-3.796875,2.90625,2.859375,-2.203125,-1.78125,-3.796875,0.1708984375,-5.15625,0.298828125,1.828125,-7.1875,1.6953125,6.125,2.671875,0.1728515625,3.375,0.609375,-4.78125,0.40625,-3.875,-6.4375,0.6953125,1.171875,-2.140625,5.8125,-1.640625,5.90625,-0.1650390625,4.9375,-2.421875,1.609375,-3.171875,-4.71875,7.6875,-1.09375,-1.9296875,0.033447265625,-3.46875,-2.671875,-8.875,2.4375,-1.1015625,4.40625,-3.53125,1.546875,2.359375,-3.15625,10.625,7.46875,-3.0625,-0.044677734375,0.90234375,-5.28125,-3,-1.2890625,0.59375,-6.34375,-1.8203125,5.40625,5.78125,-1.578125,2.46875,-2.171875,-1.71875,-0.38671875,-2.21875,-0.150390625,4.65625,-3.46875,1.5625,4.4375,-2.609375,1.6875,-2.828125,-6.03125,5.15625,-2.296875,-1.65625,-2.3125,-4.75,-3.3125,-3.703125,-1.9296875,-6.59375,3.640625,-0.62890625,4.8125,0.11279296875,2.515625,0.9921875,-3.03125,-5.40625,7.5625,-1.765625,4.4375,4.25,-0.140625,3.671875,-2.984375,-2.734375,2.203125,-6.96875,-1.1640625,2.390625,1.3515625,-1.75,2.96875,-3.75,-0.109375,2.5,0.796875,5.21875,7.8125,-4,1.171875,0.435546875,1.2734375,-3.015625,1.90625,-1.21875,5.9375,-0.31640625,-4.03125,-3.59375,1.09375,4.65625,-0.81640625,-2.046875,0.80859375,-5.375,2,-2.265625,5.34375,-0.46875,-1.3359375,-2.953125,-4.875,-0.53515625,-3,1.8203125,-2.59375,-1.4765625,6.28125,2.09375,0.1318359375,2.40625,-0.09130859375,-2.421875,-1.78125,1.59375,0.48828125,-0.310546875,-0.2353515625,0.1748046875,0.4453125,2.078125,-1.046875,1.46875,0.6953125,-0.52734375,-0.19140625,-2.28125,-0.515625,0.47265625,-1.2421875,-8.3125,1.1875,2.015625,-4.46875,3.734375,1.453125,-2.8125,-2.78125,5.875,-0.38671875,1.171875,-6.5,1.8046875,-2.15625,4,3.375,-0.78125,0.87890625,-1.796875,-1.265625,2.59375,3.96875,1.7421875,2.296875,2.78125,-5.8125,-2.046875,-0.1201171875,-4.1875,3.96875,-3.484375,-4.125,1.21875,3.484375,0.3828125,3.8125,1.90625,-8.3125,-2.15625,2.578125,2.578125,-1.34375,-3.359375,4.71875,-1.640625,-3.484375,2.046875,3.0625,-1.03125,-2.96875,6.96875,3.703125,-0.29296875,-0.423828125,2.640625,-1.265625,3.9375,-0.314453125,-4.15625,-2.171875,0.2734375,6.375,-6.21875,-6.3125,4.6875,-0.053466796875,0.045166015625,2.765625,2.953125,1.078125,-0.453125,1.96875,-6.71875,-3.375,-4.1875,2.515625,-0.5390625,-1.9296875,-4.03125,1.1953125,8.1875,1.0078125,0.80859375,-1.15625,-1.53125,2.875,-3.921875,1.953125,4.09375,6.59375,-4.5625,-1.2109375,3.5,-8.1875,0.294921875,-3.453125,-0.9921875,-2.015625,1.5,0.6328125,-4.90625,-2.765625,1.0546875,4.25,-2.390625,-5.96875,7.15625,-5.4375,-3.953125,1.15625,-0.017822265625,2.90625,2.78125,-2.21875,3.390625,1.9453125,2.171875,1.8671875,-1.125,-3.65625,-1.359375,0.96484375,2.5625,-2.9375,1.2734375,4.15625,-6,-0.2021484375,-1.8515625,-0.56640625,-1.671875,1.546875,5.8125,-0.640625,1.140625,-5.71875,-0.40625,0.5390625,-1.6640625,0.3203125,-2.375,4.9375,-2.453125,-1.59375,0.1669921875,1.6796875,-0.81640625,1.765625,-3.125,-1.234375,0.84375,-0.96484375,0.232421875,-0.01300048828125,-6.03125,4.25,5.625,0.65625,-1.6015625,1.390625,5.65625,3.0625,0.287109375,-0.08203125,4.15625,-1.5703125,-0.609375,-6.34375,2.203125,-3.84375,-2.53125,-3.390625,3.15625,4.59375,-4.46875,5.0625,-3.09375,3.328125,-0.65625,1.8515625,-9.375,1.609375,-1.515625,-2.5625,-2.953125,0.734375,2.375,1.3515625,0.390625,1.8671875,0.07080078125,1.328125,3.6875,0.2421875,0.73828125,3.1875,1.65625,2.75,2.859375,-2.8125,-7.75,1.53125,-1.1015625,-1.6875,6.3125,3.03125,-2.46875,0.77734375,-0.34765625,-1.78125,-1.4453125,3.40625,3.140625,-3.96875,3.984375,-3.21875,5.375,-2.890625,2.90625,-2.375,-6.1875,-2.4375,0.34375,-4.46875,-2.421875,3.40625,-1.2578125,4.59375,4.125,-6,0.003936767578125,1.1484375,2.359375,4.09375,0.5703125,-1.328125,-6.03125,4.5,3.234375,-2.140625,5.03125,-2.640625,0.041748046875,-0.90234375,4.375,-2.125,-0.1630859375,2.421875,-2.078125,1.1328125,-3.53125,1.0234375,-0.2734375,-9.125,-6.03125,0.73828125,-0.87109375,6.59375,-0.65625,-2.109375,-3.359375,2.40625,-0.0157470703125,5.96875,2.390625,3.078125,5.65625,5.09375,-1.5859375,1.78125,-0.921875,-8.0625,7.0625,-5.71875,-2.375,2.359375,2.65625,-1.453125,-1.2265625,1.984375,-2.125,-5.46875,-5.25,-1.78125,-4.28125,3.375,-2.09375,1.984375,-0.75,-5.0625,1.46875,-1.8671875,-2.875,-1.859375,2.609375,-5.5,2.484375,5.65625,1.875,-0.94921875,3.890625,4.125,0.8984375,-2.796875,0.95703125,-7.9375,1.7890625,3.453125,-1.9296875,-0.69140625,-5.84375,2.171875,-3.4375,2.921875,0.890625,-2.203125,-2.375,-1.6328125,-2.65625,0.8515625,-7.28125,2.484375,1.6484375,-0.8359375,-0.859375,0.232421875,1.921875,0.73046875,-0.30078125,1.515625,4.9375,0.7109375,-0.43359375,-3.140625,-2.796875,-0.2431640625,2.265625,-2.53125,6.875,-0.54296875,-1.5625,3.96875,0.44921875,-3.640625,-4.25,4.375,-1.875,0.45703125,-1.2265625,5.65625,0.298828125,3.921875,-1.703125,-2.8125,-3.328125,1.7578125,3.3125,-1.6875,-3.234375,2.09375,2.375,5.40625,-3.234375,-7.09375,1.984375,4.125,-0.8046875,-2.71875,8.6875,-1.296875,-2.625,-3,-3.78125,1.359375,1.515625,2.875,0.11279296875,-1.5859375,1.078125,3.46875,-1.390625,0.6328125,0.24609375,-3.765625,3.515625,0.380859375,2.609375,-0.80078125,-2.484375,-2.15625,-1.3203125,0.02490234375,4.03125,8.25,-1.5234375,-1.1953125,1.2109375,0.3125,-1.7421875,5.625,-0.76953125,5.90625,1.15625,0.1640625,1.390625,0.82421875,-0.322265625,3.21875,-4.65625,-4.5,-1.765625,3.171875,-4.3125,-1.4375,-2.546875,-0.9140625,4.28125,0.609375,-3.171875,3.671875,0.48046875,-0.9140625,-4,-2.4375,-5.34375,-1.96875,0.828125,1.953125,-2.140625,-2.59375,-0.353515625,4.78125,-4.09375,-3.921875,0.03173828125,1.8359375,1.3984375,-0.65234375,-1.15625,0.1611328125,0.50390625,2.90625,-1.875,-3.40625,0.498046875,8.75,3.90625,-4.53125,0.67578125,-0.765625,1.8359375,-5.3125,-2.15625,-0.6796875,-1.8984375,-3.046875,-1.7734375,-1.390625,-2.71875,-2.015625,5.84375,-3.28125,0.55859375,0.8046875,3.984375,0.99609375,3.015625,0.458984375,5.3125,3.1875,-1.2421875,-5.84375,-1.3828125,-0.04052734375,-5.75,-1.8828125,3.234375,6,3.171875,1.5703125,-2.828125,0.033203125,-0.953125,0.640625,5.3125,-5.75,-3.78125,-1.984375,-7.9375,-6.84375,-3.859375,-2.65625,-3.15625,-6.84375,-0.9765625,-1.375,-7.1875,-1.1328125,-2.109375,-1.546875,-1,0.640625,4.625,-4.65625,2.3125,3.703125,2.6875,3.0625,-2.28125,3.34375,0.474609375,-1.46875,0.34765625,-2.03125,5.25,-1.4609375,5.875,3.984375,-0.87890625,-3.8125,4.46875,4.40625,5.90625,-4.875,-3.53125,-2.53125,-1.8125,-0.39453125,-1.2421875,2.203125,-3.828125,-3.59375,-1.0859375,-3.453125,0.1845703125,5.625,0.421875,5.3125,-1.3671875,0.30859375,1.5234375,2.953125,0.1064453125,2.59375,1.5546875,-4.46875,3.609375,-0.81640625,1.390625,0.8359375,-2.78125,2.125,-1.6875,0.365234375,2.234375,3.875,10.4375,1.15625,2.328125,-0.09326171875,-0.76171875,-2.609375,-2.96875,2.40625,1.6796875,1.4921875,-3.65625,0.74609375,-0.8828125,2.03125,-0.306640625,-16.875,-3.328125,-5.53125,-2.109375,4.625,-1.0546875,-1.984375,1.0625,3.6875,2.671875,7.09375,-1.484375,4.03125,-1.078125,-0.7265625,2.515625,-4.3125,1.578125,3.6875,1.890625,4.625,1.7734375,-1.8125,-2.828125,6.9375,5.0625,-4.5,0.193359375,5.09375,-1.3515625,-1.140625,4.40625,-2.96875,2.078125,-4.75,3.078125,7.09375,2.75,-2.953125,-4.125,-2.375,-2.0625,1.0234375,3.046875,-2.578125,1.578125,2.921875,-5.65625,2.28125,2.28125,-0.259765625,-3.484375,-0.37109375,2.71875,1.625,-0.158203125,-4.5,2.5625,0.98828125,3.84375,4.8125,-2.796875,-2.140625,2.34375,2.90625,2.1875,1.5546875,2.578125,2.8125,-1.8515625,-2.984375,0.310546875,-1.328125,-0.0234375,-1.9765625,0.83984375,3.65625,2.046875,-4.5625,2.171875,2.234375,-2.109375,-0.0439453125,-4.0625,-3.5,2.09375,-2.21875,-2.5,0.703125,-2.953125,-1.28125,3.234375,-4.6875,4.1875,-2.484375,8.75,-0.53125,-1.8203125,1.171875,-3.0625,4.78125,-2.484375,-3.453125,3.765625,-2.6875,1.5625,-3.828125,1.9296875,-1.765625,1.2421875,5.0625,-4.65625,-2.0625,0.53125,3.265625,-2.875,-2.296875,0.29296875,3.859375,0.123046875,-4.46875,4.09375,-2.796875,3.96875,-3.890625,1.875,-4.46875,-0.5078125,2.140625,0.3203125,4.84375,5.03125,-5.34375,-4.96875,-1.3203125,-5.03125,-4.875,-4.5625,5.03125,-2.625,-0.75,1.046875,2.109375,-0.130859375,1.890625,-1.8125,2.53125,6.53125,-2.09375,0.87890625,-0.41015625,-0.412109375,-4.09375,-2.421875,-4.46875,6.40625,0.43359375,1.2578125,3.734375,-1.7109375,2.953125,1.8125,-1.1171875,-1.7109375,2.15625,1.859375,-2.015625,-2.25,1.7734375,-3.578125,4.15625,-3.328125,-3.28125,-4.71875,2.953125,1.40625,-0.287109375,1.5703125,3.53125,1.578125,3.171875,-4.34375,-3.125,5.78125,3.453125,-2.046875,4.3125,-1.2265625,-1.84375,0.640625,2.625,0.12890625,-3.25,-4.6875,5.28125,2.65625,2.015625,-4.4375,-5.75,-3.625,4.0625,4.59375,-0.78125,-2.484375,-2.03125,-3.75,1.6875,-4.15625,2.734375,-1.65625,-3.453125,-0.89453125,3.71875,2.453125,-4.15625,2.09375,0.82421875,-2.03125,0.052978515625,4.4375,1.734375,-3.71875,1.375,-0.349609375,-1.75,-7,3.59375,-2.625,-0.427734375,-4.40625,-3.84375,-3.265625,-3.796875,0.74609375,2.65625,1.6171875,3.609375,-0.7890625,3.890625,2.796875,-0.8671875,-0.43359375,2.15625,-1.7578125,-3.640625,2.375,-4.65625,-3.5,1.3984375,-7.1875,-1.5,5.0625,-2.625,4.0625,-1.171875,3.796875,-1.453125,-2.9375,-4,-1.3046875,0.91796875,6.59375,0.64453125,-0.91796875,0.64453125,2.703125,2.1875,-2.296875,-1.015625,-1.9921875,5,-0.298828125,2.953125,-5.125,-5.03125,5.375,-1.1328125,-4.46875,-0.5546875,-3.09375,1.5703125,5.34375,0.765625,-4.46875,-2.421875,-6.75,2.8125,-1.6171875,3.109375,-5.59375,0.87109375,-4.875,2.53125,4.46875,-7.21875,-3.1875,2.4375,3,5.1875,1.84375,-2.625,-6.21875,2.21875,0.306640625,-1.90625,-4.09375,-2.34375,-1.3046875,-3.875,4.4375,-2.328125,2.546875,-3.875,-2.40625,0.80078125,0.34765625,1,0.828125,1.4453125,-0.859375,3.03125,1.109375,5.15625,1.1953125,-3.8125,2.734375,4.21875,0.345703125,-1.2109375,2.0625,-0.79296875,-2.8125,2.109375,2.96875,-2.90625,5.15625,3.359375,4.3125,-5.53125,-2.875,1.515625,3.515625,-2.75,1.7109375,-4.9375,0.7265625,3.71875,-0.4765625,1.34375,0.049560546875,2.796875,-1.421875,-1.7890625,1.5,2.3125,4.21875,1.6875,3.015625,3.3125,-1.1640625,3.546875,-0.375,-1.2265625,-1.59375,3.609375,-3.015625,-2.546875,-4.625,1.046875,-1.796875,4.75,2.515625,1.1484375,0.8984375,-1.4140625,-2.328125,0.037841796875,-5.78125,-1.5859375,0.11669921875,3.015625,-0.83984375,0.84375,-0.82421875,0.96484375,4.0625,0.0400390625,4.25,-2.28125,1.3515625,1,1.5625,-2.8125,3.15625,-2.609375,-0.142578125,1.578125,-2.875,3.75,-4.3125,-1.359375,-2.578125,-0.69140625,2.84375,3.75,-4.75,-5.5625,0.84765625,0.380859375,5.125,3.0625,-3.140625,-0.93359375,0.73046875,0.0303955078125,4.3125,0.85546875,2.703125,-4.28125,5.625,5.90625,0.4296875,0.76953125,-0.9140625,-1.71875,-4.5,3.828125,-0.4609375,2.21875,-1.9453125,2.5,4.15625,1.8984375,3.984375,-5.75,2.953125,0.2734375,3.890625,-0.76171875,-3.90625,0.337890625,1.96875,0.69140625,-0.70703125,3.578125,0.046142578125,0.765625,-2.734375,4.28125,4.3125,2.578125,-4.40625,1.921875,-2.90625,1.7734375,-3.90625,1.1484375,-5.625,1.65625,2.703125,5.34375,-1.9296875,-6.1875,4.5,1.5625,-0.9140625,-3.953125,4.65625,4.5625,2.484375,-5.15625,-2.375,1.625,-1.328125,-0.26171875,-5.25,3.328125,2.0625,-3.609375,-3.71875,1.6171875,1.046875,-3.1875,-3.71875,-3.34375,1.9609375,2.5625,3.609375,1.59375,-2.484375,4.125,-0.80078125,1.9140625,4.78125,-1.09375,0.140625,3.171875,-3.578125,2.640625,-0.6640625,-2.65625,-1.4375,0.47265625,-2.46875,2.6875,-2.515625,-1.0234375,-2.09375,-0.138671875,-0.5078125,1.5,4.15625,-3.09375,0.158203125,4.4375,-1.96875,-3,-1.9609375,2.09375,-1.7734375,-1.09375,-1.8984375,3.3125,1.9765625,0.8671875,0.2890625,0.66796875,-1.9765625,-3.640625,-4.90625,2.0625,-4.0625,3.59375,-0.84765625,-6.21875,1.515625,3.890625,3.640625,-0.2734375,-2.046875,0.875,3.78125,0.07470703125,-1.078125,-1.4921875,3.671875,-2.796875,-3.6875,2.75,2.78125,-5.40625,1.7890625,-4.28125,-2.265625,-0.98046875,4.46875,0.173828125,-2.25,-2.875,-3.84375,-1.7421875,-1.6171875,-3.21875,1.9140625,1.7421875,2.671875,1.09375,1.4375,-3.5,2.59375,19.125,0.0101318359375,-8.4375,1.3515625,-3.625,4.4375,4.65625,1.8125,0.423828125,-1.5,0.62890625,4.21875,0.609375,0.5390625,-2.390625,0.984375,-0.79296875,2.078125,-3.703125,-3.109375,-2.265625,-1.0234375,-0.328125,1.9765625,2.5,2.375,0.8046875,-2.265625,1.2734375,-3.390625,-4.375,-4.71875,3.765625,-2.921875,3.125,-3.171875,4.65625,-0.7890625,-3.3125,-2.984375,-3.296875,-2.796875,2.375,-0.12255859375,-3.21875,5.21875,0.1982421875,0.2138671875,-1.1796875,-0.130859375,-4.34375,-1.4453125,-2.5,6.3125,1.0625,-6.15625,-0.5703125,-3.203125,-3.546875,-1.375,2.9375,-0.53515625,1.7578125,2.71875,-1.9453125,-2.640625,-3.046875,0.49609375,1.0078125,-3,-4.84375,0.2119140625,1.2265625,1.3515625,1.609375,-4.84375,2.46875,2.140625,2.171875,1.75,0.67578125,-0.60546875,-2.46875,-2.234375,-0.9453125,1.2421875,-3.15625,0.006744384765625,3.359375,-1.765625,8.375,-8.3125,5.8125,5.15625,-2.0625,-0.470703125,1.5,-0.30859375,-2.421875,-0.2294921875,0.95703125,1.8828125,4.84375,-0.68359375,4.625,1.359375,0.373046875,0.83203125,2.640625,4.34375,0.7578125,3.109375,-0.412109375,-2,2.15625,-0.08349609375,-3.140625,-3,-3.703125,-2.5625,3.6875,1.7890625,-3.296875,0.89453125,-7.5,-5.40625,-2.25,-7.625,4.34375,-1.34375,-0.14453125,3.515625,-2.46875,-1.2109375,-2.46875,-3.921875,1.265625,3.65625,1.4375,-1.46875,-5.03125,2.59375,3.890625,-2.765625,-2.4375,0.353515625,-4.21875,4.4375,-0.376953125,3.9375,-2.09375,3.96875,3.234375,-2.203125,-6.875,5.15625,-3.6875,-4.34375,-6.625,-2.90625,-4.9375,-3.34375,0.412109375,-0.9453125,-0.5703125,-1.3046875,3.21875,-0.65234375,-1.6796875,3.171875,3.453125,-4.4375,-1.2578125,0.828125,1.1796875,-4.375,0.1787109375,4,0.53515625,1.328125,-0.546875,0.271484375,-0.5546875,-3.859375,-0.2216796875,0.86328125,-4.53125,-1.3828125,-0.60546875,-5.46875,-1.3515625,-1.2890625,-3.734375,2.9375,2.40625,-3.984375,0.875,-2.953125,-0.9765625,-1.6328125,-1.25,3.96875,1.6953125,0.0072021484375,5.875,-0.921875,-3.46875,-3.140625,-0.26953125,0.2265625,-2.09375,7.0625,-1.09375,0.30078125,-6.03125,5.34375,2.359375,1.6640625,-0.99609375,4.625,4.25,-2.484375,-4,0.89453125,3.0625,4.1875,-4.28125,3.953125,0.6328125,-0.74609375,-1.53125,2.015625,-1.1796875,1.03125,-1.6484375,-5.4375,0.3671875,1.8125,-0.326171875,1.546875,4.03125,-3.34375,0.484375,2.5,-1.4140625,3.34375,4.25,-1.7890625,1.09375,2.171875,5.34375,-1.5625,0.98828125,-5.09375,-3.625,-2.640625,-2.46875,3.109375,-2.515625,0.09033203125,0.21484375,-3.921875,3.125,-4.1875,1.2109375,1.3671875,1.1875,-5.4375,4.59375,3.890625,-2.8125,3.328125,-5.125,-1.9765625,-1.4296875,2.34375,-2.71875,-5.875,3.125,3.453125,-1.515625,3.546875,2.265625,-0.52734375,1.9375,-2.859375,2.703125,-3.359375,4.75,1.2734375,3.09375,3.65625,-0.255859375,-0.1044921875,-5.75,-0.3359375,-0.77734375,-2.234375,6.1875,-3.84375,0.19921875,4.25,6.4375,-10.5,-1.5078125,0.7265625,0.2890625,3.921875,5.0625,0.09814453125,0.68359375,3.109375,1.015625,2.671875,0.0257568359375,-0.4765625,-4,5.15625,0.2314453125,-4.6875,3.1875,3.984375,-2.609375,3.4375,-2.375,-3.734375,-0.07568359375,2.75,-5.3125,1.9296875,4.625,-1.6484375,2.875,3.734375,-1.34375,3.875,-1.9921875,-11.3125,-1.53125,3.296875,5.71875,0.80859375,1.7578125,0.48046875,-2.015625,1.4765625,-0.5546875,0.71484375,-0.7578125,-11.1875,0.9765625,-3,-0.09765625,-1.9453125,-3.8125,-2.5,4.375,1.65625,1.1015625,3.328125,2.84375,0.84375,4.5625,0.11279296875,-5.84375,1.1484375,1.7578125,-4.8125,-0.59765625,3.234375,1.125,-1.859375,-2.515625,3.78125,-1.7421875,-0.69921875,5.8125,3.765625,1.578125,-1.84375,-5.03125,0.984375,-3.375,-1.9140625,1.1953125,-0.384765625,2.8125,-2.203125,2.828125,1.1171875,-3.75,-4.15625,-2.25,-3.5625,1.5,2.671875,2.171875,-2.609375,-1.7265625,2.8125,2.5,-0.455078125,-1.546875,2.1875,-0.1884765625,-2.984375,-1.4765625,2.0625,-4.46875,-2.90625,4.0625,1.8359375,0.443359375,-0.7734375,-3.140625,2.171875,1.734375,-1.8515625,-1.84375,-1.234375,2.15625,5.34375,-2.484375,-5.6875,-1.2734375,0.1806640625,-4.375,-3.5625,0.89453125,-1.15625,0.75,3.09375,-2.25,1.1875,4.6875,-1.3359375,-3.875,3.53125,4.4375,-2.671875,-0.75,-0.458984375,-2.53125,3.8125,5,-1.2421875,-2.109375,-0.50390625,-2.734375,-4.90625,1.0234375,2.421875,-3.34375,-10.125,6.46875,3.671875,5.40625,1.546875,-2.59375,3.8125,-1.6953125,3.703125,-0.423828125,0.82421875,1.515625,-7.59375,-2.40625,-2.0625,-5.0625,0.59375,-0.345703125,-4.75,1.4921875,6.25,-2.15625,-1.8671875,-2.703125,-3.9375,4.28125,-3.484375,-5.9375,1.984375,-7.4375,1.4609375,-1.9609375,3.265625,-5.875,1.8359375,-0.017333984375,2.046875,-0.5859375,-0.671875,-2.328125,1.1953125,-2.65625,3.625,0.7890625,3.9375,-0.365234375,2.90625,-1.2421875,0.314453125,-3.265625,1.6640625,1.7109375,0.60546875,0.384765625,2.296875,-2.28125,-0.8046875,-1.0546875,1.046875,2.796875,0.61328125,-0.625,0.10693359375,4.21875,-0.6484375,2.03125,-2.3125,-0.173828125,-1.015625,-0.224609375,0.74609375,-0.86328125,0.0145263671875,0.1318359375,1.7109375,1.421875,0.486328125,-0.19921875,0.140625,1.2734375,1.015625,1.5625,-1.65625,-0.45703125,-0.435546875,-0.0206298828125,1.828125,1.734375,-2.734375,1.65625,-2.09375,-0.6875,-0.2421875,2.125,1.1015625,0.1064453125,1.59375,-1.875,1.828125,0.15234375,-1.2421875,1.25,-0.765625,-2.265625,2.34375,-2.109375,-0.921875,0.6640625,-1.2734375,-1.4765625,-0.73828125,2.21875,-0.84375,1.328125,-1.171875,-0.181640625,0.306640625,-1.171875,0.279296875,0.94140625,1.171875,-3.921875,3.15625,1.2421875,0.52734375,-0.1630859375,1.0390625,-1.46875,-0.08447265625,1.0390625,-0.37109375,0.921875,1.859375,-1.8046875,0.54296875,-0.8203125,-1.09375,1.1640625,1.515625,0.54296875,-1.65625,-1,1.5234375,1.4453125,-1.1953125,0.359375,-0.062255859375,-2.09375,3.03125,1.21875,-3.15625,-0.357421875,-0.169921875,0.546875,-0.73828125,-0.126953125,1.046875,-2.75,-0.2314453125,0.2421875,0.306640625,-1.1328125,1.8984375,0.00469970703125,3.9375,0.8515625,1.1328125,1.1875,1.3984375,2.046875,-1.3515625,0.25390625,-0.9921875,3.234375,-0.373046875,0.8828125,1.3828125,-1.921875,-0.484375,-0.81640625,0.61328125,1.4296875,-0.70703125,-0.404296875,2.53125,1.625,0.494140625,2.375,-2.03125,0.33984375,0.291015625,-0.68359375,-1.625,1.625,-0.478515625,0.349609375,-2.0625,-1.25,-0.1484375,-0.44140625,0.67578125,0.3671875,0.4921875,0.236328125,1.1953125,0.5078125,-2.375,1.3671875,-0.341796875,0.6328125,-1.7265625,-1.328125,0.84375,-0.08935546875,1.0625,0.90625,1.984375,2.828125,1.109375,-1.3671875,1.03125,1.0625,1.75,0.263671875,-1.234375,-0.09228515625,-0.13671875,0.271484375,0.58203125,-0.9375,-1.28125,0.4609375,-0.95703125,-0.1552734375,-1.5703125,3.375,-0.9609375,-1.1796875,-0.419921875,-1.5,0.58984375,-1.3125,1,-1.578125,2.484375,1.34375,3.34375,1.4296875,-0.671875,-0.984375,0.30859375,0.72265625,-0.337890625,-0.06982421875,-1.125,-0.44921875,-0.62890625,5.40625,0.263671875,1.0390625,-2.03125,3.296875,0.68359375,-0.10986328125,-1.078125,-0.2412109375,-2.078125,-0.13671875,-1.4375,-1.390625,0.29296875,-1.1484375,-4.0625,-2.703125,-0.302734375,0.77734375,-1.640625,-0.0390625,3.890625,0.375,1.2890625,1.5,2.640625,0.19140625,-1.78125,-0.5859375,1.6328125,-1.234375,2,0.8125,-1.9453125,-2.78125,-0.3671875,-2.328125,-1.9453125,-0.59375,-0.8046875,1.9921875,-0.265625,-0.03515625,-1.3125,-1.5234375,-3.03125,-0.458984375,-0.1279296875,2.375,1.53125,0.67578125,-0.55078125,-0.4296875,0.515625,-1.75,0.6640625,-1.65625,4.25,-0.326171875,-1.4296875,2.53125,0.396484375,3.140625,0.859375,-1.3671875,-1.8828125,-0.828125,0.45703125,0.7109375,3.0625,-0.2578125,0.6328125,0.57421875,-0.85546875,0.5625,1.0234375,-0.296875,-4.84375,-1.578125,-0.486328125,2.59375,-1.2109375,0.09765625,2.59375,-0.87109375,-0.7890625,-1.7421875,-2.34375,-0.2490234375,-0.82421875,0.8046875,2.078125,-0.7265625,-0.10400390625,-0.703125,-1.046875,0.46875,-1.7734375,1.09375,-0.30859375,0.0181884765625,0.2734375,-2.703125,-0.470703125,0.67578125,-1.921875,-1.0078125,1.6328125,0.2021484375,1.359375,1.6796875,-1.6015625,1.5703125,0.6484375,-2.859375,-0.63671875,-0.8359375,1.34375,0.0556640625,0.4375,1.765625,-1.1484375,-1.90625,-1.453125,0.57421875,0.84375,-0.349609375,0.251953125,-0.0927734375,0.416015625,-0.40625,-2.71875,-0.48046875,0.4140625,-0.2109375,0.96484375,1.0859375,1.453125,1.15625,1.375,-0.478515625,1.375,-1.8828125,1.6484375,0.9921875,-2.171875,0.5859375,2.03125,-2.125,0.314453125,1.1796875,-0.4921875,-0.72265625,-0.80078125,0.5546875,-0.52734375,0.58203125,-0.52734375,1.9453125,1.71875,-0.328125,1.453125,-2.203125,-2.09375,-2.625,0.2177734375,-0.82421875,0.3359375,-2.203125,1.375,-1.7578125,-0.072265625,-0.4765625,-0.38671875,-1.9453125,1.5625,1.7578125,0.4453125,0.640625,0.0255126953125,-0.5703125,3.796875,-1.0703125,-0.1201171875,0.93359375,1.15625,-2.078125,3.484375,0.5234375,2.109375,0.0037078857421875,1.3359375,-0.796875,1.25,0.1455078125,0.86328125,0.478515625,1.828125,0.31640625,-0.296875,-0.154296875,-1.53125,-1.1640625,0.6484375,1.0703125,-5.375,0.86328125,0.890625,0.48828125,0.84765625,-2.828125,1.1015625,0.4765625,3.296875,-0.00408935546875,-0.40234375,3.421875,0.61328125,-1.46875,1.1875,0.953125,0.0771484375,-2.78125,-1.171875,-0.86328125,2.9375,-1.0703125,0.1015625,-0.279296875,-0.90625,3.046875,0.6796875,-1.6640625,1.453125,0.443359375,-0.439453125,-1.453125,-3.40625,-0.1689453125,1.71875,-0.9453125,2.234375,0.158203125,0.87109375,0.66796875,-1.640625,1,0.265625,0.267578125,-0.90625,1.75,-0.2041015625,-1.59375,1.65625,-1.1484375,-1.78125,2.421875,1.6953125,-2.328125,0.027587890625,-0.494140625,-0.3203125,-0.01953125,0.58203125,-2.28125,0.546875,0.62109375,0.90625,-0.921875,-1.53125,2.484375,1.890625,2.953125,2.359375,-0.90234375,0.171875,-2.234375,0.33984375,-0.45703125,-0.87109375,0.08251953125,1.8671875,-1.0078125,1.5703125,-0.30078125,0.921875,-1.8046875,1.609375,2.703125,0.92578125,0.40625,-0.26171875,-0.322265625,-1.8671875,-0.5,-2.296875,0.62109375,0.6953125,1.1640625,0.1376953125,-1.4296875,1.5390625],"index":0,"object":"embedding"},{"embedding":[-2.28125,-0.7734375,-0.8359375,-2.3125,3.046875,4.125,-1.0390625,-2.890625,0.0103759765625,1.9296875,0.1015625,1.75,2.4375,2.015625,5.09375,1.203125,-2.140625,-2.828125,-1.328125,-4.6875,1.0078125,6.8125,0.578125,-4.71875,-0.80859375,-6.25,1.578125,4.25,4.46875,-1.0078125,8,-2.3125,2.546875,-0.00555419921875,1.5625,-1.8671875,-2.375,-2.53125,5.25,-0.69140625,-2.96875,-0.68359375,1.6171875,2.96875,-3.015625,-1.734375,0.4140625,-2.9375,2.53125,-1.6640625,-4.5625,-1.9296875,3.234375,-2.734375,2.359375,-4.125,-3.046875,4.5,-5.875,-2.984375,-1.8515625,-2.8125,-0.7734375,0.46484375,1.3984375,5.28125,0.68359375,-1.3359375,0.51171875,8.625,-0.055908203125,3.578125,6.5,-2.390625,6.34375,5.5625,0.7265625,1.578125,-2.921875,4.90625,-2.953125,-0.62890625,2.453125,3.46875,4.5625,2.671875,-1.9140625,0.859375,-3.03125,1.703125,1.96875,0.59375,-1.4140625,-3.140625,-1.2109375,1.2890625,-3.21875,-6.5625,-6.78125,2.765625,-0.78515625,-0.3515625,1.8125,-4.53125,-5.03125,2.171875,-1.8515625,-5.46875,-1.78125,0.380859375,2.640625,1.65625,3.640625,-2.140625,2.46875,1.21875,4.28125,-2.796875,-4.40625,2.796875,-2.0625,-1.9765625,4.28125,-0.6796875,4.4375,4.28125,-4.03125,-0.01416015625,5.53125,-1.4609375,7.25,3.578125,3.6875,-2.375,-8.0625,-4.71875,-1.9453125,3.71875,4.3125,4.40625,-5.03125,3.21875,-3.734375,-6.625,4.1875,-3.4375,-6.4375,-3.15625,3.859375,-1.9140625,-1.78125,1.8046875,0.5,2.3125,-1.2421875,-4.375,4.0625,3.875,0.1259765625,-1.0546875,2.015625,3.328125,1.1484375,1.7265625,1.8046875,-0.462890625,-5.625,3.6875,-1.0390625,2.5625,0.90625,10.4375,4.28125,-4.5625,1.9765625,8.625,-1.328125,8.625,1.4609375,2.203125,0.81640625,-0.640625,-2.90625,4.53125,-2.15625,1.5,0.12255859375,-5.6875,3.140625,1.2890625,1.578125,1.5625,2.71875,-1,-4.84375,-1.8671875,3.484375,-2.578125,3.4375,0.1025390625,-1.40625,-7.375,1.4921875,1.5546875,-4.71875,-3.765625,2.703125,-1.71875,3.078125,-0.380859375,2.265625,0.24609375,3.21875,-2.0625,7.65625,2.640625,2.734375,2.046875,1.8359375,2.46875,4.53125,3.484375,1.8359375,-2.078125,-0.83984375,2.03125,5.8125,0.439453125,3.75,8.6875,0.251953125,0.408203125,6.84375,-2.515625,-1.78125,-3.578125,-3.78125,1.6015625,-0.279296875,2.671875,-5.65625,-4.0625,-2.328125,2.984375,3.515625,-3.359375,-2.34375,-2.703125,-0.51171875,-6.4375,1.484375,3.671875,-9.0625,1.8828125,5.625,3.96875,1.984375,1.265625,-0.33203125,-4.125,0.333984375,-2.4375,-5.875,-0.58203125,1.890625,-2.390625,5.09375,-1.5546875,3.515625,-0.7421875,5.1875,-2.28125,-0.0927734375,-3.046875,-4.3125,8.8125,-0.232421875,-1.90625,1.0703125,-3.078125,-3.5625,-10.25,2.5,1.1171875,4.96875,-2.921875,1.40625,0.40234375,-3.640625,12.75,3.90625,-1.8203125,1.9921875,-0.63671875,-6.03125,-1.984375,-2.046875,2.046875,-5.59375,1.84375,3.6875,4.5,-1.9296875,3.4375,-1.7421875,-0.9296875,-1.109375,-4.5625,-1.9375,2.671875,-3.765625,2.34375,9.625,-4.75,2.03125,-2.109375,-6.1875,4.75,-0.03662109375,-0.11376953125,-2.140625,-5.125,-1.9921875,-2.78125,-1.4296875,-6.65625,4.96875,-0.984375,5.375,0.97265625,3,3.296875,-4.1875,-5.03125,8.4375,-1.5,3.296875,5.71875,0.55078125,0.68359375,-3.515625,-4.6875,2.46875,-5.46875,0.953125,5.71875,3.328125,-1.640625,1.0234375,-6.21875,2.40625,2.328125,-0.68359375,6.53125,6.90625,-2.265625,2.78125,1.9140625,-0.71484375,-2.28125,-0.2294921875,-1.078125,6.34375,1.1875,-3.890625,-3.796875,-0.5859375,5.03125,-2.375,0.7734375,-1.21875,-4.15625,2.59375,-1.15625,3.6875,0.91796875,0.90625,-1.8046875,-5.125,0.087890625,-2.625,0.29296875,-1.7734375,-3.28125,4.25,1.515625,-0.484375,1.59375,0.67578125,-3.53125,-0.46484375,0.59765625,-1.15625,0.65625,2.5625,-0.5703125,-0.984375,1.5546875,-0.3828125,-2.21875,1.0546875,-1.2734375,2.40625,-6.9375,-0.6484375,-0.2490234375,-2.125,-8.375,-0.4765625,1.0703125,-3.78125,2.71875,1.96875,-1.2578125,-3.0625,4.4375,1.421875,1.8671875,-6.90625,2.15625,-1.8828125,3.328125,2.140625,-1.7421875,0.59375,-1.4296875,-2.765625,4.375,3.546875,-0.69921875,3.453125,0.68359375,-3.265625,-3.625,0.1630859375,-4.90625,4.75,-0.236328125,-1.859375,5.21875,2.203125,-1.5,1.625,0.98828125,-6.28125,-4.78125,2.96875,3.171875,-3.078125,-3.96875,0.470703125,-1.4296875,-4.4375,3.078125,3.84375,-1.1171875,-2.8125,3.40625,4.375,-2.203125,0.0830078125,1.1171875,0.52734375,2.703125,-1.9375,-3.140625,-0.1103515625,0.130859375,4.71875,-5.8125,-6.84375,3.015625,-2.875,0.2001953125,1.15625,4.5625,0.46875,-1.8984375,-1.9296875,-3.0625,-3.46875,-2.828125,3.53125,-1.078125,-2.53125,-2.90625,0.29296875,8.3125,1.90625,0.369140625,-2.375,-0.11572265625,2.453125,-1.71875,0.50390625,4.4375,7.90625,-4.03125,-0.63671875,3.53125,-8.125,0.94921875,-1.375,-1.15625,-0.94921875,2.3125,2.1875,-6.25,-0.7890625,0.0115966796875,5.03125,-3.453125,-3.828125,5.15625,-4.8125,-3.09375,1.859375,-0.6875,4.0625,1.296875,-1.34375,2.875,2.984375,2.65625,1.8203125,-2.53125,-3.640625,-3.3125,1.2890625,2.265625,-2.234375,2.296875,4,-5.4375,0.90234375,-2.25,-0.6953125,-0.212890625,-0.515625,5.90625,2.125,2.25,-6.09375,1.2578125,0.50390625,-0.416015625,-0.7421875,-1.1484375,6.71875,-0.5,-0.2294921875,0.94921875,2.09375,-1.1953125,1.640625,-3.796875,-2.453125,-3.109375,-1.796875,-1.0234375,-4.03125,-5.5,4.4375,6,-1.234375,-1.6796875,2.171875,5.5,3.984375,-0.84375,1.515625,3.421875,-2.5,0.23828125,-5.40625,2.609375,-7.84375,-2.53125,-1.6875,2.921875,3.75,-4.15625,3.765625,-2.578125,2.4375,-1.4375,4.4375,-10.5625,2.046875,-2.15625,-2.796875,-2.28125,-0.57421875,3.171875,-0.44921875,2.109375,1.3671875,-0.75,3.953125,5.46875,-1.5,1.765625,2.1875,2.46875,-0.5859375,2.515625,-2.125,-8.25,1.3125,-1.1484375,1.09375,7.5625,1.9375,-1.7734375,2.46875,0.88671875,-1.5703125,-1.7265625,4.0625,3.015625,-1.546875,4.25,-3.90625,5.40625,-3.28125,1.7265625,-3.265625,-6.15625,0.279296875,1.9296875,-5.5625,-4.09375,2.859375,0.216796875,5.78125,3.421875,-5.375,1.21875,-0.41796875,1.109375,2,0.30078125,-0.03759765625,-4.75,3.921875,4.1875,-2.40625,7.03125,-1.5703125,-1.6484375,-1.1171875,2.40625,-1.7734375,0.373046875,1.84375,0.287109375,-0.78125,-3.484375,0.96484375,0.5703125,-6.625,-7.21875,1.7265625,-1.7734375,7.0625,0.73046875,-0.859375,-3.15625,2,1.5546875,6.375,3.3125,3.765625,4.5,3.765625,-2.390625,2.671875,-3.6875,-6.09375,7,-6.53125,-1.8515625,1.015625,0.859375,-0.2578125,-1.0234375,-0.3515625,-0.71484375,-3.484375,-6.09375,-2.359375,-1.875,2.015625,-1.6484375,2.203125,0.57421875,-4.09375,-0.5703125,-1.6484375,-1.6875,-1.6640625,4.15625,-5.625,1.484375,5.71875,2.046875,-1.5234375,4.15625,3.09375,-0.47265625,-4.78125,0.7109375,-6.875,1.6015625,1.46875,-0.6015625,0.50390625,-8,2.03125,-2.4375,3.5,-0.671875,-0.05078125,-1.265625,-3.296875,-1.3984375,-0.91796875,-5.40625,-0.171875,1.6953125,1.125,-1.8359375,0.671875,3.078125,-0.52734375,0.384765625,-1.125,2.046875,0.40625,2.34375,-4.78125,-2.90625,1.28125,0.9140625,-2.03125,6.53125,0.91796875,0.79296875,3.546875,1.7265625,-5.5,-5.78125,3.921875,-2.8125,-1.796875,-3.25,2.421875,-1.359375,6.53125,-2.21875,-5.53125,-3.703125,1.6484375,3.15625,-2.609375,-3.09375,4.78125,1.8359375,2.765625,-2.15625,-7.5,1.609375,0.98828125,-0.146484375,-1.140625,8.625,-1.9296875,-0.4765625,-4.4375,-3.234375,2.046875,0.875,2.046875,-0.76171875,-1.2734375,0.69921875,0.4765625,-2.34375,-0.55078125,0.6015625,-2.546875,1.75,0.07177734375,4.875,-2.53125,0.3984375,-1.2734375,-0.50390625,-0.10009765625,4.3125,8.75,-1.765625,-0.96875,0.35546875,2.984375,-3.59375,6.6875,1.3515625,7.75,-1.1640625,0.25,1.03125,0.375,-2.171875,4.59375,-5.25,-2.84375,-1.890625,1.21875,-2.5625,0.671875,-3.984375,-0.498046875,4.40625,-0.455078125,-0.007568359375,2.609375,0.79296875,-0.201171875,-3.09375,-1.3125,-4.71875,-2.515625,-0.14453125,2.03125,-3.03125,-0.4921875,-0.33984375,5.84375,-0.357421875,-1.4453125,-2.59375,1.53125,1.859375,1.171875,-0.8046875,0.255859375,0.58984375,3.3125,-1.015625,-4.34375,-0.94921875,8.4375,4.21875,-6.875,1.5703125,-0.43359375,1.4453125,-4.8125,-1.4609375,-2.15625,-1.4921875,-4.1875,1.1328125,0.419921875,-3,-0.06494140625,4.5,-1.2890625,-0.15625,3.46875,4.0625,0.478515625,2.96875,-2.125,4.375,2.21875,-2.09375,-5.96875,-1.703125,0.48046875,-2.75,-1.4140625,2.03125,6.15625,0.55859375,2.625,-1.0625,2.28125,-1.6953125,3.78125,5.125,-4.59375,-2.703125,-2.3125,-9.5625,-4.03125,-1.7421875,-2.921875,-5.34375,-4.25,-0.86328125,-1.2421875,-8,0.0966796875,-2.234375,-3.265625,1.4453125,2.953125,1.7578125,-5.75,3.125,4.125,2.578125,2.546875,0.84765625,5.46875,-0.050537109375,-2.96875,1.4453125,-3.4375,4.15625,-1.03125,3.546875,6.25,-0.453125,-4.96875,4.78125,2.96875,5.53125,-7.375,-2.625,-0.337890625,-1.671875,-0.458984375,-1.7578125,2.546875,-4.5,-5.5,1.078125,-3.203125,1.2265625,4.6875,-0.8046875,6.78125,1.6328125,0.419921875,2.140625,2.71875,0.62109375,0.169921875,1.7421875,-5.9375,3.234375,-2.171875,3.265625,-0.296875,-1.5234375,2.734375,-0.7578125,-0.310546875,2.8125,2.734375,10.3125,0.515625,4,-2.3125,0.63671875,-1.7265625,-0.2392578125,2.25,2.015625,0.79296875,-1.4765625,0.7890625,-0.44921875,0.478515625,-0.4609375,-13.25,-1.9609375,-7.25,-1.9296875,7.0625,-2.1875,-1.9921875,1.4296875,2.6875,3.484375,5.125,-0.58984375,3.375,-0.60546875,0.80859375,5.96875,-4.25,1.03125,3.359375,2.546875,5.21875,0.154296875,-0.44921875,-3.203125,8,2.25,-1.4140625,0.8359375,2.796875,-1.3046875,-2.34375,3.09375,-3.171875,2.96875,-4.9375,0.5859375,4.15625,0.65625,-3.890625,-3.4375,-2,-0.62890625,1.3828125,1.375,-2.59375,0.18359375,0.94921875,-4.1875,3.328125,-0.59375,0.140625,-5.53125,1.03125,4.65625,0.703125,-0.109375,-1.8515625,1.4453125,-0.8984375,4.3125,2.78125,-2.734375,0.2734375,2.21875,1.7421875,-0.125,1.03125,1.1328125,2.921875,-3.09375,-0.353515625,-0.44140625,-1.625,1.4765625,-3.1875,1.6640625,3.203125,1.3984375,-3.984375,2.21875,0.79296875,-0.11669921875,2.96875,-5.125,-1.9921875,-1.1015625,-0.71484375,-4.0625,-0.9140625,-4.375,-0.1455078125,5.46875,-5,3.4375,-2.515625,8.1875,0.1298828125,-1.421875,1.2890625,-2.828125,2.59375,-3.390625,-1.234375,3.484375,-0.92578125,2.125,-3.546875,1.8984375,-2.078125,-0.46484375,6.09375,-3.953125,-1.9765625,0.7421875,3.21875,-5.0625,-3.296875,0.1611328125,0.8515625,0.009765625,-1.8984375,1.4765625,-2.03125,4.4375,-4.75,3.390625,-4.65625,-3.90625,0.28125,0.07568359375,7.90625,4.25,-3.796875,-3.421875,-0.6015625,-7.0625,-3.421875,-3.859375,6.65625,-0.52734375,0.96875,2.078125,2.390625,-0.01031494140625,1.46875,-2.96875,3.203125,5.28125,0.294921875,3.046875,2.1875,-1.125,-4.40625,0.3125,-3.171875,7.0625,3.0625,0.404296875,3,-1.8984375,1.484375,-1.03125,-1.0625,-2.828125,2.171875,1.71875,-2.5,-3.28125,1.046875,-3.859375,0.72265625,-5.40625,-2.578125,-5.3125,2.765625,2.3125,-0.81640625,-0.7578125,4.4375,0.318359375,3.328125,-5.53125,-3.890625,3.8125,0.9765625,0.333984375,2.84375,-0.6796875,-5.03125,-0.9375,0.201171875,1.9140625,-4.1875,-3.609375,3.328125,2.46875,0.283203125,-3.9375,-4.40625,-3.453125,2.390625,4.1875,-0.96484375,0.353515625,0.06005859375,-1.53125,2.171875,-2.65625,4.5,-3.109375,-4.15625,-0.47265625,0.734375,3.578125,-3.203125,-1.0703125,1.4296875,-3.4375,0.7578125,1.2734375,-0.11279296875,-1.9453125,3.171875,-2,-3.65625,-5.4375,5.78125,-2.0625,0.45703125,-3.875,-2.65625,-3.1875,-1.421875,-0.6640625,1.7421875,0.0703125,5.78125,-0.63671875,2.8125,0.478515625,-0.8828125,0.0712890625,3.453125,-0.271484375,-2.90625,1.8359375,-4.59375,-4.65625,0.7578125,-8.0625,-2.0625,2.90625,-2.40625,2.671875,-2.671875,2.375,-1.1015625,-2.21875,-1.8203125,-0.8203125,0.83984375,5.375,2.171875,0.2216796875,0.38671875,1.8984375,0.859375,-1.109375,-1.8515625,-0.25,5.34375,0.62109375,2.765625,-3.359375,-2.34375,4.46875,-0.59375,-3.75,0.8984375,-0.357421875,0.6640625,4.5625,0.9609375,-3.796875,-2.9375,-6.15625,4.03125,0.73828125,1.828125,-4.625,1.5,-3.0625,0.1748046875,2.03125,-6.5625,-2.546875,3.328125,2.828125,5.46875,1.328125,-2.421875,-4.53125,2.203125,-0.396484375,-1.6171875,-2.234375,-1.7265625,-0.96875,-3.765625,4.125,-2.515625,4.25,-1.3359375,-2.8125,-0.8671875,0.61328125,-0.203125,0.47265625,-0.353515625,-0.88671875,4.0625,-0.3515625,7,2.171875,-4.0625,4.59375,2.515625,0.412109375,-1.5625,3.75,-1.109375,-2.3125,3.921875,2.890625,-4.0625,4.96875,2.125,3.375,-3.46875,-2.1875,-0.9921875,4.5625,0.287109375,1.28125,-4.34375,0.1630859375,4.0625,-0.1884765625,0.8671875,-1.765625,0.3046875,0.65234375,0.52734375,2,1.921875,3.4375,-0.52734375,1,-0.92578125,-1.2265625,2.328125,-0.1328125,-0.703125,-1.8828125,3.21875,-1.6953125,-1.875,-6,1.2421875,-3.46875,2.21875,3.1875,2.875,2.234375,-2.828125,-1.625,-2.640625,-5.25,-3.140625,1.75,1.09375,-1.75,1.875,-0.1181640625,2.546875,5.84375,0.130859375,4.6875,-3.109375,2.5,1.140625,0.875,0.046630859375,4.3125,-1.8203125,-2.21875,3.640625,-4.46875,3.71875,-4.53125,-3.078125,-0.63671875,-0.10986328125,2.640625,6.625,-4.5625,-3.953125,5.21875,1.328125,4.59375,3.78125,-2.078125,-1.484375,0.79296875,1.3515625,5.46875,0.93359375,2.953125,-2.734375,6.9375,5.65625,0.90625,2.359375,0.166015625,-2.6875,-6.4375,5.125,1.3984375,1.984375,-2.375,1.6875,3.109375,0.1533203125,3.640625,-5.5,0.8671875,1.2109375,0.90625,0.5234375,-3.15625,0.103515625,2.640625,0.33203125,-1.6875,5.84375,0.97265625,4.125,-0.72265625,3.34375,2.328125,3.703125,-2.03125,1.5234375,-3.46875,3.578125,-1.3984375,2.15625,-5.5,1.0546875,3.640625,4.3125,-1.625,-3.5625,2.21875,0.275390625,-0.5,-4.46875,4.21875,3.59375,2.5625,-6.9375,-3.328125,-0.05029296875,0.2060546875,1.234375,-3.484375,1.171875,1.6796875,-4.625,-3.265625,1.296875,1.625,-5.65625,-6.0625,-3.203125,1.65625,1.3203125,3.1875,3.21875,-0.8203125,3.40625,-0.55078125,3.046875,4.28125,-1.1328125,1.5546875,0.9375,-2.75,4.125,-0.263671875,-2.671875,1.5546875,-0.50390625,-2.140625,0.50390625,-2.296875,-1.0703125,-4.21875,-0.85546875,2.328125,-1.09375,5.125,-3.96875,0.30078125,3.609375,-1.4375,-2.28125,-2.65625,0.5703125,-2.921875,-2.578125,-1.9140625,3.609375,2.984375,2.046875,0.58203125,-0.6015625,-3.265625,-6.40625,-5.65625,3.578125,-2.515625,2.859375,0.439453125,-4.25,2.078125,2.8125,1.78125,-0.1640625,-0.55859375,2.765625,4.59375,0.455078125,-1.7265625,-0.466796875,3.609375,-4.5625,-3.78125,0.515625,1,-3.171875,2.28125,-3.125,-1.8359375,0.79296875,4.5,-0.5078125,-2.859375,-1.75,-2.40625,-2.875,-3.03125,-2.859375,2.5625,1.859375,3.296875,0.1689453125,-0.421875,-5,3.71875,16.875,0.9375,-4.71875,2.421875,-3.140625,2.65625,3.171875,4.8125,-1.7109375,-1.96875,-2.1875,1.765625,0.01031494140625,1.4140625,-2.140625,1.7421875,1.9921875,-0.48828125,-4.125,-1.9765625,-1.328125,0.84765625,-0.7578125,2.96875,0.408203125,2.265625,-0.734375,-0.259765625,0.2333984375,-3.234375,-4.46875,-4.4375,2.265625,-1.7578125,4.75,-4.25,5.375,0.1845703125,-2.9375,-2.09375,-3.296875,-3.171875,1.0234375,-0.75,-1.9453125,4.34375,-0.72265625,1.09375,0.37890625,-0.337890625,-3.546875,-3.046875,-2.6875,7.25,0.62890625,-5.71875,-1.546875,-4.84375,-4.5625,0.58984375,2.796875,-2.328125,1.6328125,1.453125,-1.828125,-2.171875,-1.953125,0.85546875,3,-5.125,-5.625,0.13671875,1.5546875,3.359375,2.796875,-4.0625,1.5703125,5.3125,2.6875,0.69140625,-0.75,1.4453125,-1.3828125,-2.5,-0.91015625,1.4609375,-4.03125,1.109375,1.4453125,-4.875,11.25,-8.625,4.8125,4.0625,-4.75,-0.1865234375,2.796875,1.796875,-1.6796875,-0.169921875,2.953125,2.453125,3.359375,-0.306640625,6.09375,1.5234375,0.388671875,0.73828125,2.9375,3.578125,2.4375,2.9375,-0.828125,-1.9609375,1.3046875,1.7734375,-2.484375,-3.46875,-1.4609375,-4.4375,6,1.6171875,-2.765625,-1.2578125,-10.5,-3.421875,-2.328125,-5.84375,4.5,-2.65625,2.46875,3.421875,-0.609375,-1.078125,-2.53125,-5,2.296875,4.0625,0.208984375,-0.3984375,-6.0625,2.84375,3.546875,-3.984375,-2.09375,1.4453125,-3.265625,3.296875,-0.1923828125,4.9375,-3.578125,3.9375,2.03125,-2.546875,-5.8125,3.171875,-3.765625,-2.234375,-5.3125,-2.453125,-2.078125,-3.328125,-0.6171875,-0.35546875,-2.078125,-1.03125,1.6171875,-0.60546875,-3.15625,2.921875,2.96875,-4.375,-2.625,0.58203125,0.73046875,-4.28125,1.1875,5.1875,-0.54296875,1.5,0.55078125,0.078125,-0.3203125,-4.34375,0.81640625,1.71875,-4.03125,-0.71875,-1.359375,-2.828125,-2.4375,-2.78125,-3.375,3.875,3.59375,-5.0625,1.9609375,-0.34765625,0.014892578125,-1.4453125,-1.546875,6.4375,2.234375,-1.6484375,5.59375,1.03125,-4.15625,-2,-2.046875,-1.1484375,-1.2734375,6.3125,1.2578125,2.375,-5.90625,7.53125,2.453125,1.7265625,-0.43359375,2.34375,1.6796875,-3.71875,-5.40625,2.46875,2.75,3.84375,-4.59375,0.6328125,0.53515625,0.53125,-4.28125,1.90625,-0.259765625,0.482421875,-3.140625,-7.59375,-0.109375,0.90625,-1.8828125,1.5234375,4.25,-2.96875,1.3828125,0.95703125,-0.58984375,3.640625,3.28125,-2.828125,1.90625,-0.1904296875,2.625,-2.34375,1.4921875,-3.71875,-4.96875,-3.109375,-1.765625,1.8828125,-2.625,0.67578125,-0.357421875,-4.1875,2.109375,-2.25,1.125,1.09375,0.2578125,-6.25,3.984375,5.1875,-4.15625,4.4375,-5.53125,-2.4375,-1.640625,2.21875,-1.9140625,-6.46875,2.0625,4.5,-3.390625,2.203125,3.546875,-1.625,-0.4453125,-2.25,5.3125,-1.015625,4.78125,-0.6953125,3.953125,3.9375,-1.28125,-0.061279296875,-5.125,0.470703125,-2.28125,-3.84375,5.53125,-1.921875,2.46875,5.21875,4.9375,-9,-1.96875,0.54296875,-0.1845703125,3.578125,3.109375,-1.3671875,1.0234375,0.028076171875,-0.30859375,4.4375,-0.9296875,-1.46875,-3.65625,4.96875,-0.1728515625,-4.0625,2.984375,2.609375,-4.15625,4.34375,-2.75,-2.6875,-0.6875,-0.1396484375,-5.625,1.8046875,2.6875,-0.92578125,3.4375,3.109375,1.203125,3.59375,-2.640625,-10.0625,0.0703125,2.75,5.3125,1.7265625,2.3125,0.0859375,-1.0625,3.640625,-4.5625,0.46875,-1.484375,-9.5,0.255859375,-4.15625,-1.609375,-3.453125,-1.4921875,-1.9453125,3.90625,1.3984375,-0.8515625,3.5,2.921875,0.453125,4.15625,-0.361328125,-3.578125,1.2734375,1.75,-5.28125,-1.90625,4.8125,3.578125,-2.203125,-2.0625,3.84375,-4.28125,-0.70703125,4.3125,4.28125,2.15625,-0.828125,-3.234375,2.84375,-2.546875,-2.828125,1.703125,-3.421875,2.453125,-1.4375,2.578125,1.296875,-2.640625,-2.03125,-4.15625,-2.71875,3.484375,0.28515625,0.9765625,-2.265625,-1.1171875,3.234375,3.5625,-2.359375,-2.109375,2.796875,-1.3515625,-4.28125,-1.0859375,1.0859375,-5.90625,-2.609375,2.734375,3.4375,-2.5625,-3.5625,-2.125,1.6171875,1.3046875,-0.8984375,-0.1318359375,-3.53125,2.65625,5.0625,-2.9375,-3.75,-1.6171875,-0.486328125,-5.03125,-3.609375,-0.1767578125,1.140625,-0.73046875,3.890625,-1.40625,0.47265625,4.4375,-3.65625,-3.21875,3.96875,3.359375,-3.203125,-1.46875,2.25,-3.375,1.03125,5.4375,-2.390625,-2.234375,0.41796875,-2.171875,-4.28125,2.34375,1.2265625,-3.734375,-7.875,5.96875,1.0703125,4.34375,4.125,-3.90625,4.0625,-4.6875,1.8828125,-1.265625,1.015625,1.3828125,-5.65625,-1.1875,-2.5,-3.5,0.5390625,-1.734375,-3.5625,0.66015625,8.0625,-1.328125,-2.59375,-2.953125,-3.515625,3.3125,-4.15625,-7.625,0.1181640625,-7.34375,1.734375,-2.1875,1.75,-5.59375,1.9140625,-1.078125,1.734375,-2.984375,0.27734375,-0.384765625,1.21875,0.54296875,4.6875,1.2109375,1.984375,-0.1484375,2.71875,0.0791015625,1.875,-1.453125,-0.4921875,1.21875,-1.234375,0.33203125,0.69921875,-2.734375,0.1708984375,-1.7578125,-0.263671875,-1.015625,1.7578125,2.9375,-0.640625,-0.291015625,-1.6875,1.703125,-4.5,1.3125,-1.796875,0.859375,-0.78515625,-1.0078125,1.9609375,-2.328125,1.6640625,1.015625,1.640625,0.01068115234375,-1.5,2.234375,2.6875,-0.031982421875,-2.328125,-1.8046875,-0.55859375,-1.7421875,1.7421875,0.55078125,-2.0625,2.9375,-1.640625,-0.41015625,0.890625,1.7265625,0.44140625,-1.6484375,2.40625,-1.8671875,1.2890625,1.0859375,-1.5234375,2.609375,0.63671875,1.03125,1.2734375,0.9765625,-2,0.64453125,0.2578125,-1.4375,-0.291015625,3.484375,-1.7265625,0.31640625,-1.078125,-0.5625,1.0859375,-0.8671875,1.2109375,0.15625,-0.396484375,-2.75,2.640625,-2.125,-1.2578125,-0.42578125,0.29296875,-0.5703125,0.8984375,0.08935546875,1.2109375,-0.29296875,2.28125,-0.73828125,2.171875,-0.020263671875,-0.2060546875,1.3359375,3.421875,-1.984375,0.7421875,-2.0625,-1.1328125,1.3203125,-0.3046875,1.15625,-0.93359375,-2,1.2421875,1.1328125,-2.984375,-0.734375,2.265625,-0.189453125,-1.1328125,-0.609375,1.2265625,-0.75390625,-0.38671875,0.419921875,-0.89453125,2,3.265625,-1.0625,2.5,-1.453125,0.396484375,0.73046875,1.046875,2.3125,0.07958984375,-2.34375,-0.9296875,2.71875,-1.4375,0.37109375,0.890625,-1.53125,-0.1396484375,1.3359375,0.5703125,1.640625,-0.06982421875,-1.859375,-0.330078125,-0.6796875,1.609375,1.65625,-1.6875,0.68359375,-1.8359375,-0.53125,-1.015625,2.765625,-1.7578125,-2.140625,-0.78515625,-1.1015625,-0.83203125,-0.498046875,0.11962890625,-0.1298828125,0.60546875,1.125,1.5,0.4296875,-0.609375,1.4375,-0.08056640625,0.68359375,-1.1875,-1.5234375,1.484375,1.2421875,2.34375,-1.359375,1.34375,0.9296875,0.8828125,-1.1796875,1.9453125,-0.5234375,0.314453125,0.010986328125,-0.1181640625,1.40625,2.21875,0.318359375,0.5859375,-0.1328125,1.40625,0.69921875,1.375,-1.3046875,-2.203125,-1.0078125,-1.4296875,-2.125,0.361328125,-0.0615234375,-1.3046875,-0.1904296875,0.034912109375,-0.86328125,1.375,1.1796875,1.5390625,-0.828125,-0.58203125,0.1787109375,-0.328125,0.25390625,0.8828125,-0.8046875,-0.78125,-1.1171875,-2.0625,1.578125,0.88671875,-1.09375,-0.2890625,2.0625,-1.5,1.0078125,-2.78125,0.55078125,-1.828125,-0.341796875,0.0859375,-3.265625,0.34765625,-0.12451171875,-2.15625,-3.078125,-1.75,-0.85546875,-2.375,-0.3203125,4,-0.81640625,-1.21875,2.03125,0.08203125,-1.0078125,-0.94921875,1.7578125,2.84375,-0.8203125,3.859375,0.349609375,-0.16015625,-1.3984375,-1.265625,0.52734375,-1.2890625,0.294921875,-0.84765625,-0.8046875,-1.6796875,-3.109375,0.05859375,-4.1875,-2.125,0.1337890625,0.90625,1.890625,-0.08447265625,-0.7421875,-0.56640625,-0.96875,2.796875,-0.267578125,0.18359375,1.4375,0.27734375,0.46875,-1.4140625,0.92578125,-0.84375,2.953125,-1.171875,-0.50390625,-2.65625,-1.5546875,-4.1875,1.453125,2.484375,0.421875,2.96875,1.3671875,-0.5546875,-2.5625,0.07421875,0.00909423828125,-4.75,-0.373046875,-0.7265625,0.07275390625,-1.4140625,-0.7109375,-0.1318359375,-0.609375,-1.328125,-0.51953125,-1.828125,-0.271484375,-2.28125,2.984375,1.7890625,1.875,2.3125,0.3125,-0.31640625,1.1875,2.359375,1.1484375,0.6953125,0.255859375,0.408203125,-1.09375,2.09375,0.337890625,0.4609375,-1.2265625,0.2275390625,1.1875,2.5625,1.734375,-0.76171875,0.85546875,0.328125,-1.9140625,-1.40625,0.31640625,0.296875,1.140625,0.333984375,1.03125,-1.2890625,0.416015625,-0.6875,0.9453125,1.7578125,-1.953125,1.109375,-0.134765625,0.1787109375,-1.5,1.203125,1.15625,1.8203125,-0.48046875,2.140625,1.1640625,0.48828125,1.8515625,2.609375,-0.361328125,1.421875,-0.86328125,1.953125,0.51953125,-2.484375,3.15625,-0.34375,-0.47265625,-0.56640625,1.2890625,1.359375,-0.60546875,-0.25,-0.38671875,2.015625,0.52734375,0.14453125,1.8828125,0.67578125,-0.546875,-0.77734375,-0.6015625,-1.09375,-2.328125,-1.0078125,-3.0625,-0.37109375,-0.9375,1.765625,-0.828125,-1.484375,-0.142578125,1.390625,-0.02099609375,1.3203125,1.6171875,-1.0859375,2.09375,0.154296875,0.1962890625,0.89453125,-0.97265625,-1.2421875,1.15625,0.82421875,-0.59765625,4.625,0.1962890625,2.28125,-0.65625,-1.0390625,-0.78515625,3.59375,-0.44921875,-0.4375,-1.6953125,1.140625,-0.296875,-1.25,-0.76953125,-1.3984375,-0.9765625,1.78125,-0.87109375,-3.234375,-2.171875,0.330078125,-1.875,0.48828125,-1.859375,-1.0390625,2.40625,1.734375,-0.63671875,0.216796875,1.125,-1.0234375,0.58984375,-0.4296875,0.3515625,1.6015625,-1.2109375,1.765625,0.5859375,2.796875,-3.921875,-0.298828125,2.171875,1.578125,-0.458984375,-1.015625,-0.51171875,2.109375,0.369140625,-0.018798828125,-0.50390625,-4.46875,0.0135498046875,-0.043212890625,-3.21875,-0.09423828125,0.4921875,1.2421875,0.6640625,-3.15625,0.73046875,-1.5078125,-1.6328125,3.46875,-0.55078125,-0.41796875,0.58203125,1.1640625,-0.83203125,-0.84765625,1.53125,0.17578125,-3.484375,-1.1015625,-0.1591796875,-0.875,0.59765625,0.01373291015625,0.099609375,0.546875,-0.36328125,-1.171875,-1.1328125,-0.33984375,-0.08056640625,1.015625,4,1.1484375,1.265625,1.2109375,-2.125,4.5625,-2.515625,-0.96484375,1.1015625,1.3515625,-1.1796875,3.921875,1.109375,0.2265625,-2,0.55859375,2.96875,0.765625,0.9453125,0.671875,1.28125,1.7421875,1.78125,-1,-1.8671875,1.5,-0.35546875,-2.5,0.012451171875,0.2578125],"index":1,"object":"embedding"}],"model":"doubao-embedding-text-240715","object":"list","usage":{"prompt_tokens":7,"total_tokens":7}}, - "headers": { - "Content-Type": "application/json" - } - } - }, - { - "httpRequest": { - "method": "POST", - "path": "/v1/openai/embedding" - }, - "httpResponse": { - "body": {"object":"list","data":[{"object":"embedding","index":0,"embedding":[-0.006929283495992422,-0.005336422007530928,-4.547132266452536e-05,-0.024047505110502243]}],"model":"text-embedding-3-small","usage":{"prompt_tokens":5,"total_tokens":5}}, - "headers": { - "Content-Type": "application/json" - } - } - }, - { - "httpRequest": { - "method": "POST", - "path": "/v1/custom/embedding", - "headers": { - "Authorization": [ - "Bearer xxxxxxxx" - ] - } - }, - "httpResponse": { - "body": {"created":1725001256,"id":"02172500125677376580aba8475a41c550bbf05104842f0405ef5","data":[{"embedding":[-1.625,0.07958984375,-1.5703125,-3.03125,-1.4609375,3.46875,-0.73046875,-2.578125,-0.66796875,1.71875,0.361328125,2,5.125,2.25,4.6875,1.4921875,-0.77734375,-0.466796875,0.0439453125,-2.46875,3.59375,4.96875,2.34375,-5.34375,0.11083984375,-5.875,3.0625,4.09375,3.4375,0.2265625,9,-1.9296875,2.25,0.765625,3.671875,-2.484375,-1.171875,-1.6171875,4.1875,2.390625,-6.90625,0.369140625,0.259765625,3.671875,-2.9375,-1.9140625,-0.71875,-1.6640625,0.29296875,0.396484375,-4.625,-1.9921875,5.15625,-1.3984375,3.015625,-3.203125,-1.453125,4,-8.75,-5.625,1.0546875,-3.28125,-1.2265625,0.287109375,2.09375,4.6875,0.1572265625,0.42578125,0.79296875,3.234375,-0.169921875,0.9296875,7.40625,-3.296875,5.53125,3.890625,0.62109375,1.1171875,-0.373046875,4.125,-2.78125,0.333984375,3.9375,4.59375,6,1.53125,-0.373046875,1.109375,-4.0625,1.96875,1.421875,0.57421875,-0.56640625,-2.390625,0.734375,1.1875,-2.859375,-6.09375,-5.96875,1.8359375,-3,0.80859375,-0.130859375,-5.3125,-2.859375,1.484375,-4.53125,-6.90625,-2.25,0.7734375,-1.2734375,1.1484375,3.421875,-3.484375,2.65625,1.3359375,1.1484375,-4.09375,-5.625,2.625,-0.283203125,-3.46875,2.3125,-0.220703125,4.21875,3.75,-0.37109375,0.9609375,7.25,-0.87890625,7.03125,2.34375,4.5,-1.609375,-6.46875,-6.125,-2.59375,2.234375,3.78125,1.3046875,-5.5,1.953125,-3.421875,-5.9375,3.25,-3.4375,-8.3125,-2.546875,3.640625,0.267578125,-0.220703125,0.294921875,-0.4140625,2.515625,-1.0546875,-5.21875,6.6875,3.640625,0.2314453125,-2.5,1,1.6640625,0.59765625,2.75,1.1328125,1.1328125,-4.96875,4.53125,-0.349609375,3.390625,-0.193359375,7.625,2.921875,-3.484375,4.1875,8.5,-1.9140625,6.3125,2.5625,3.0625,0.40234375,0.76953125,-4.78125,3.53125,-2.765625,0.1591796875,-0.1025390625,-3.875,2.203125,0.03076171875,1.765625,1.859375,2.15625,-1.2578125,-4.40625,-0.62890625,4.4375,-1.78125,2.671875,2.765625,-1.7890625,-8.3125,-0.02197265625,1.640625,-3.96875,-3.15625,2.796875,1.1875,2,1.15625,2.359375,1.3984375,4.21875,-2.953125,8.5,3.46875,3.578125,0.90625,-1.8828125,2.15625,3.921875,4.125,-0.9609375,-2.171875,2.328125,2.921875,1.9765625,1.0703125,4.03125,6.28125,-3.59375,-0.94921875,5.6875,-1.9140625,-5.1875,-4.25,-7.71875,1.7109375,-1.59375,3.765625,-5.3125,-3.9375,-3.796875,2.90625,2.859375,-2.203125,-1.78125,-3.796875,0.1708984375,-5.15625,0.298828125,1.828125,-7.1875,1.6953125,6.125,2.671875,0.1728515625,3.375,0.609375,-4.78125,0.40625,-3.875,-6.4375,0.6953125,1.171875,-2.140625,5.8125,-1.640625,5.90625,-0.1650390625,4.9375,-2.421875,1.609375,-3.171875,-4.71875,7.6875,-1.09375,-1.9296875,0.033447265625,-3.46875,-2.671875,-8.875,2.4375,-1.1015625,4.40625,-3.53125,1.546875,2.359375,-3.15625,10.625,7.46875,-3.0625,-0.044677734375,0.90234375,-5.28125,-3,-1.2890625,0.59375,-6.34375,-1.8203125,5.40625,5.78125,-1.578125,2.46875,-2.171875,-1.71875,-0.38671875,-2.21875,-0.150390625,4.65625,-3.46875,1.5625,4.4375,-2.609375,1.6875,-2.828125,-6.03125,5.15625,-2.296875,-1.65625,-2.3125,-4.75,-3.3125,-3.703125,-1.9296875,-6.59375,3.640625,-0.62890625,4.8125,0.11279296875,2.515625,0.9921875,-3.03125,-5.40625,7.5625,-1.765625,4.4375,4.25,-0.140625,3.671875,-2.984375,-2.734375,2.203125,-6.96875,-1.1640625,2.390625,1.3515625,-1.75,2.96875,-3.75,-0.109375,2.5,0.796875,5.21875,7.8125,-4,1.171875,0.435546875,1.2734375,-3.015625,1.90625,-1.21875,5.9375,-0.31640625,-4.03125,-3.59375,1.09375,4.65625,-0.81640625,-2.046875,0.80859375,-5.375,2,-2.265625,5.34375,-0.46875,-1.3359375,-2.953125,-4.875,-0.53515625,-3,1.8203125,-2.59375,-1.4765625,6.28125,2.09375,0.1318359375,2.40625,-0.09130859375,-2.421875,-1.78125,1.59375,0.48828125,-0.310546875,-0.2353515625,0.1748046875,0.4453125,2.078125,-1.046875,1.46875,0.6953125,-0.52734375,-0.19140625,-2.28125,-0.515625,0.47265625,-1.2421875,-8.3125,1.1875,2.015625,-4.46875,3.734375,1.453125,-2.8125,-2.78125,5.875,-0.38671875,1.171875,-6.5,1.8046875,-2.15625,4,3.375,-0.78125,0.87890625,-1.796875,-1.265625,2.59375,3.96875,1.7421875,2.296875,2.78125,-5.8125,-2.046875,-0.1201171875,-4.1875,3.96875,-3.484375,-4.125,1.21875,3.484375,0.3828125,3.8125,1.90625,-8.3125,-2.15625,2.578125,2.578125,-1.34375,-3.359375,4.71875,-1.640625,-3.484375,2.046875,3.0625,-1.03125,-2.96875,6.96875,3.703125,-0.29296875,-0.423828125,2.640625,-1.265625,3.9375,-0.314453125,-4.15625,-2.171875,0.2734375,6.375,-6.21875,-6.3125,4.6875,-0.053466796875,0.045166015625,2.765625,2.953125,1.078125,-0.453125,1.96875,-6.71875,-3.375,-4.1875,2.515625,-0.5390625,-1.9296875,-4.03125,1.1953125,8.1875,1.0078125,0.80859375,-1.15625,-1.53125,2.875,-3.921875,1.953125,4.09375,6.59375,-4.5625,-1.2109375,3.5,-8.1875,0.294921875,-3.453125,-0.9921875,-2.015625,1.5,0.6328125,-4.90625,-2.765625,1.0546875,4.25,-2.390625,-5.96875,7.15625,-5.4375,-3.953125,1.15625,-0.017822265625,2.90625,2.78125,-2.21875,3.390625,1.9453125,2.171875,1.8671875,-1.125,-3.65625,-1.359375,0.96484375,2.5625,-2.9375,1.2734375,4.15625,-6,-0.2021484375,-1.8515625,-0.56640625,-1.671875,1.546875,5.8125,-0.640625,1.140625,-5.71875,-0.40625,0.5390625,-1.6640625,0.3203125,-2.375,4.9375,-2.453125,-1.59375,0.1669921875,1.6796875,-0.81640625,1.765625,-3.125,-1.234375,0.84375,-0.96484375,0.232421875,-0.01300048828125,-6.03125,4.25,5.625,0.65625,-1.6015625,1.390625,5.65625,3.0625,0.287109375,-0.08203125,4.15625,-1.5703125,-0.609375,-6.34375,2.203125,-3.84375,-2.53125,-3.390625,3.15625,4.59375,-4.46875,5.0625,-3.09375,3.328125,-0.65625,1.8515625,-9.375,1.609375,-1.515625,-2.5625,-2.953125,0.734375,2.375,1.3515625,0.390625,1.8671875,0.07080078125,1.328125,3.6875,0.2421875,0.73828125,3.1875,1.65625,2.75,2.859375,-2.8125,-7.75,1.53125,-1.1015625,-1.6875,6.3125,3.03125,-2.46875,0.77734375,-0.34765625,-1.78125,-1.4453125,3.40625,3.140625,-3.96875,3.984375,-3.21875,5.375,-2.890625,2.90625,-2.375,-6.1875,-2.4375,0.34375,-4.46875,-2.421875,3.40625,-1.2578125,4.59375,4.125,-6,0.003936767578125,1.1484375,2.359375,4.09375,0.5703125,-1.328125,-6.03125,4.5,3.234375,-2.140625,5.03125,-2.640625,0.041748046875,-0.90234375,4.375,-2.125,-0.1630859375,2.421875,-2.078125,1.1328125,-3.53125,1.0234375,-0.2734375,-9.125,-6.03125,0.73828125,-0.87109375,6.59375,-0.65625,-2.109375,-3.359375,2.40625,-0.0157470703125,5.96875,2.390625,3.078125,5.65625,5.09375,-1.5859375,1.78125,-0.921875,-8.0625,7.0625,-5.71875,-2.375,2.359375,2.65625,-1.453125,-1.2265625,1.984375,-2.125,-5.46875,-5.25,-1.78125,-4.28125,3.375,-2.09375,1.984375,-0.75,-5.0625,1.46875,-1.8671875,-2.875,-1.859375,2.609375,-5.5,2.484375,5.65625,1.875,-0.94921875,3.890625,4.125,0.8984375,-2.796875,0.95703125,-7.9375,1.7890625,3.453125,-1.9296875,-0.69140625,-5.84375,2.171875,-3.4375,2.921875,0.890625,-2.203125,-2.375,-1.6328125,-2.65625,0.8515625,-7.28125,2.484375,1.6484375,-0.8359375,-0.859375,0.232421875,1.921875,0.73046875,-0.30078125,1.515625,4.9375,0.7109375,-0.43359375,-3.140625,-2.796875,-0.2431640625,2.265625,-2.53125,6.875,-0.54296875,-1.5625,3.96875,0.44921875,-3.640625,-4.25,4.375,-1.875,0.45703125,-1.2265625,5.65625,0.298828125,3.921875,-1.703125,-2.8125,-3.328125,1.7578125,3.3125,-1.6875,-3.234375,2.09375,2.375,5.40625,-3.234375,-7.09375,1.984375,4.125,-0.8046875,-2.71875,8.6875,-1.296875,-2.625,-3,-3.78125,1.359375,1.515625,2.875,0.11279296875,-1.5859375,1.078125,3.46875,-1.390625,0.6328125,0.24609375,-3.765625,3.515625,0.380859375,2.609375,-0.80078125,-2.484375,-2.15625,-1.3203125,0.02490234375,4.03125,8.25,-1.5234375,-1.1953125,1.2109375,0.3125,-1.7421875,5.625,-0.76953125,5.90625,1.15625,0.1640625,1.390625,0.82421875,-0.322265625,3.21875,-4.65625,-4.5,-1.765625,3.171875,-4.3125,-1.4375,-2.546875,-0.9140625,4.28125,0.609375,-3.171875,3.671875,0.48046875,-0.9140625,-4,-2.4375,-5.34375,-1.96875,0.828125,1.953125,-2.140625,-2.59375,-0.353515625,4.78125,-4.09375,-3.921875,0.03173828125,1.8359375,1.3984375,-0.65234375,-1.15625,0.1611328125,0.50390625,2.90625,-1.875,-3.40625,0.498046875,8.75,3.90625,-4.53125,0.67578125,-0.765625,1.8359375,-5.3125,-2.15625,-0.6796875,-1.8984375,-3.046875,-1.7734375,-1.390625,-2.71875,-2.015625,5.84375,-3.28125,0.55859375,0.8046875,3.984375,0.99609375,3.015625,0.458984375,5.3125,3.1875,-1.2421875,-5.84375,-1.3828125,-0.04052734375,-5.75,-1.8828125,3.234375,6,3.171875,1.5703125,-2.828125,0.033203125,-0.953125,0.640625,5.3125,-5.75,-3.78125,-1.984375,-7.9375,-6.84375,-3.859375,-2.65625,-3.15625,-6.84375,-0.9765625,-1.375,-7.1875,-1.1328125,-2.109375,-1.546875,-1,0.640625,4.625,-4.65625,2.3125,3.703125,2.6875,3.0625,-2.28125,3.34375,0.474609375,-1.46875,0.34765625,-2.03125,5.25,-1.4609375,5.875,3.984375,-0.87890625,-3.8125,4.46875,4.40625,5.90625,-4.875,-3.53125,-2.53125,-1.8125,-0.39453125,-1.2421875,2.203125,-3.828125,-3.59375,-1.0859375,-3.453125,0.1845703125,5.625,0.421875,5.3125,-1.3671875,0.30859375,1.5234375,2.953125,0.1064453125,2.59375,1.5546875,-4.46875,3.609375,-0.81640625,1.390625,0.8359375,-2.78125,2.125,-1.6875,0.365234375,2.234375,3.875,10.4375,1.15625,2.328125,-0.09326171875,-0.76171875,-2.609375,-2.96875,2.40625,1.6796875,1.4921875,-3.65625,0.74609375,-0.8828125,2.03125,-0.306640625,-16.875,-3.328125,-5.53125,-2.109375,4.625,-1.0546875,-1.984375,1.0625,3.6875,2.671875,7.09375,-1.484375,4.03125,-1.078125,-0.7265625,2.515625,-4.3125,1.578125,3.6875,1.890625,4.625,1.7734375,-1.8125,-2.828125,6.9375,5.0625,-4.5,0.193359375,5.09375,-1.3515625,-1.140625,4.40625,-2.96875,2.078125,-4.75,3.078125,7.09375,2.75,-2.953125,-4.125,-2.375,-2.0625,1.0234375,3.046875,-2.578125,1.578125,2.921875,-5.65625,2.28125,2.28125,-0.259765625,-3.484375,-0.37109375,2.71875,1.625,-0.158203125,-4.5,2.5625,0.98828125,3.84375,4.8125,-2.796875,-2.140625,2.34375,2.90625,2.1875,1.5546875,2.578125,2.8125,-1.8515625,-2.984375,0.310546875,-1.328125,-0.0234375,-1.9765625,0.83984375,3.65625,2.046875,-4.5625,2.171875,2.234375,-2.109375,-0.0439453125,-4.0625,-3.5,2.09375,-2.21875,-2.5,0.703125,-2.953125,-1.28125,3.234375,-4.6875,4.1875,-2.484375,8.75,-0.53125,-1.8203125,1.171875,-3.0625,4.78125,-2.484375,-3.453125,3.765625,-2.6875,1.5625,-3.828125,1.9296875,-1.765625,1.2421875,5.0625,-4.65625,-2.0625,0.53125,3.265625,-2.875,-2.296875,0.29296875,3.859375,0.123046875,-4.46875,4.09375,-2.796875,3.96875,-3.890625,1.875,-4.46875,-0.5078125,2.140625,0.3203125,4.84375,5.03125,-5.34375,-4.96875,-1.3203125,-5.03125,-4.875,-4.5625,5.03125,-2.625,-0.75,1.046875,2.109375,-0.130859375,1.890625,-1.8125,2.53125,6.53125,-2.09375,0.87890625,-0.41015625,-0.412109375,-4.09375,-2.421875,-4.46875,6.40625,0.43359375,1.2578125,3.734375,-1.7109375,2.953125,1.8125,-1.1171875,-1.7109375,2.15625,1.859375,-2.015625,-2.25,1.7734375,-3.578125,4.15625,-3.328125,-3.28125,-4.71875,2.953125,1.40625,-0.287109375,1.5703125,3.53125,1.578125,3.171875,-4.34375,-3.125,5.78125,3.453125,-2.046875,4.3125,-1.2265625,-1.84375,0.640625,2.625,0.12890625,-3.25,-4.6875,5.28125,2.65625,2.015625,-4.4375,-5.75,-3.625,4.0625,4.59375,-0.78125,-2.484375,-2.03125,-3.75,1.6875,-4.15625,2.734375,-1.65625,-3.453125,-0.89453125,3.71875,2.453125,-4.15625,2.09375,0.82421875,-2.03125,0.052978515625,4.4375,1.734375,-3.71875,1.375,-0.349609375,-1.75,-7,3.59375,-2.625,-0.427734375,-4.40625,-3.84375,-3.265625,-3.796875,0.74609375,2.65625,1.6171875,3.609375,-0.7890625,3.890625,2.796875,-0.8671875,-0.43359375,2.15625,-1.7578125,-3.640625,2.375,-4.65625,-3.5,1.3984375,-7.1875,-1.5,5.0625,-2.625,4.0625,-1.171875,3.796875,-1.453125,-2.9375,-4,-1.3046875,0.91796875,6.59375,0.64453125,-0.91796875,0.64453125,2.703125,2.1875,-2.296875,-1.015625,-1.9921875,5,-0.298828125,2.953125,-5.125,-5.03125,5.375,-1.1328125,-4.46875,-0.5546875,-3.09375,1.5703125,5.34375,0.765625,-4.46875,-2.421875,-6.75,2.8125,-1.6171875,3.109375,-5.59375,0.87109375,-4.875,2.53125,4.46875,-7.21875,-3.1875,2.4375,3,5.1875,1.84375,-2.625,-6.21875,2.21875,0.306640625,-1.90625,-4.09375,-2.34375,-1.3046875,-3.875,4.4375,-2.328125,2.546875,-3.875,-2.40625,0.80078125,0.34765625,1,0.828125,1.4453125,-0.859375,3.03125,1.109375,5.15625,1.1953125,-3.8125,2.734375,4.21875,0.345703125,-1.2109375,2.0625,-0.79296875,-2.8125,2.109375,2.96875,-2.90625,5.15625,3.359375,4.3125,-5.53125,-2.875,1.515625,3.515625,-2.75,1.7109375,-4.9375,0.7265625,3.71875,-0.4765625,1.34375,0.049560546875,2.796875,-1.421875,-1.7890625,1.5,2.3125,4.21875,1.6875,3.015625,3.3125,-1.1640625,3.546875,-0.375,-1.2265625,-1.59375,3.609375,-3.015625,-2.546875,-4.625,1.046875,-1.796875,4.75,2.515625,1.1484375,0.8984375,-1.4140625,-2.328125,0.037841796875,-5.78125,-1.5859375,0.11669921875,3.015625,-0.83984375,0.84375,-0.82421875,0.96484375,4.0625,0.0400390625,4.25,-2.28125,1.3515625,1,1.5625,-2.8125,3.15625,-2.609375,-0.142578125,1.578125,-2.875,3.75,-4.3125,-1.359375,-2.578125,-0.69140625,2.84375,3.75,-4.75,-5.5625,0.84765625,0.380859375,5.125,3.0625,-3.140625,-0.93359375,0.73046875,0.0303955078125,4.3125,0.85546875,2.703125,-4.28125,5.625,5.90625,0.4296875,0.76953125,-0.9140625,-1.71875,-4.5,3.828125,-0.4609375,2.21875,-1.9453125,2.5,4.15625,1.8984375,3.984375,-5.75,2.953125,0.2734375,3.890625,-0.76171875,-3.90625,0.337890625,1.96875,0.69140625,-0.70703125,3.578125,0.046142578125,0.765625,-2.734375,4.28125,4.3125,2.578125,-4.40625,1.921875,-2.90625,1.7734375,-3.90625,1.1484375,-5.625,1.65625,2.703125,5.34375,-1.9296875,-6.1875,4.5,1.5625,-0.9140625,-3.953125,4.65625,4.5625,2.484375,-5.15625,-2.375,1.625,-1.328125,-0.26171875,-5.25,3.328125,2.0625,-3.609375,-3.71875,1.6171875,1.046875,-3.1875,-3.71875,-3.34375,1.9609375,2.5625,3.609375,1.59375,-2.484375,4.125,-0.80078125,1.9140625,4.78125,-1.09375,0.140625,3.171875,-3.578125,2.640625,-0.6640625,-2.65625,-1.4375,0.47265625,-2.46875,2.6875,-2.515625,-1.0234375,-2.09375,-0.138671875,-0.5078125,1.5,4.15625,-3.09375,0.158203125,4.4375,-1.96875,-3,-1.9609375,2.09375,-1.7734375,-1.09375,-1.8984375,3.3125,1.9765625,0.8671875,0.2890625,0.66796875,-1.9765625,-3.640625,-4.90625,2.0625,-4.0625,3.59375,-0.84765625,-6.21875,1.515625,3.890625,3.640625,-0.2734375,-2.046875,0.875,3.78125,0.07470703125,-1.078125,-1.4921875,3.671875,-2.796875,-3.6875,2.75,2.78125,-5.40625,1.7890625,-4.28125,-2.265625,-0.98046875,4.46875,0.173828125,-2.25,-2.875,-3.84375,-1.7421875,-1.6171875,-3.21875,1.9140625,1.7421875,2.671875,1.09375,1.4375,-3.5,2.59375,19.125,0.0101318359375,-8.4375,1.3515625,-3.625,4.4375,4.65625,1.8125,0.423828125,-1.5,0.62890625,4.21875,0.609375,0.5390625,-2.390625,0.984375,-0.79296875,2.078125,-3.703125,-3.109375,-2.265625,-1.0234375,-0.328125,1.9765625,2.5,2.375,0.8046875,-2.265625,1.2734375,-3.390625,-4.375,-4.71875,3.765625,-2.921875,3.125,-3.171875,4.65625,-0.7890625,-3.3125,-2.984375,-3.296875,-2.796875,2.375,-0.12255859375,-3.21875,5.21875,0.1982421875,0.2138671875,-1.1796875,-0.130859375,-4.34375,-1.4453125,-2.5,6.3125,1.0625,-6.15625,-0.5703125,-3.203125,-3.546875,-1.375,2.9375,-0.53515625,1.7578125,2.71875,-1.9453125,-2.640625,-3.046875,0.49609375,1.0078125,-3,-4.84375,0.2119140625,1.2265625,1.3515625,1.609375,-4.84375,2.46875,2.140625,2.171875,1.75,0.67578125,-0.60546875,-2.46875,-2.234375,-0.9453125,1.2421875,-3.15625,0.006744384765625,3.359375,-1.765625,8.375,-8.3125,5.8125,5.15625,-2.0625,-0.470703125,1.5,-0.30859375,-2.421875,-0.2294921875,0.95703125,1.8828125,4.84375,-0.68359375,4.625,1.359375,0.373046875,0.83203125,2.640625,4.34375,0.7578125,3.109375,-0.412109375,-2,2.15625,-0.08349609375,-3.140625,-3,-3.703125,-2.5625,3.6875,1.7890625,-3.296875,0.89453125,-7.5,-5.40625,-2.25,-7.625,4.34375,-1.34375,-0.14453125,3.515625,-2.46875,-1.2109375,-2.46875,-3.921875,1.265625,3.65625,1.4375,-1.46875,-5.03125,2.59375,3.890625,-2.765625,-2.4375,0.353515625,-4.21875,4.4375,-0.376953125,3.9375,-2.09375,3.96875,3.234375,-2.203125,-6.875,5.15625,-3.6875,-4.34375,-6.625,-2.90625,-4.9375,-3.34375,0.412109375,-0.9453125,-0.5703125,-1.3046875,3.21875,-0.65234375,-1.6796875,3.171875,3.453125,-4.4375,-1.2578125,0.828125,1.1796875,-4.375,0.1787109375,4,0.53515625,1.328125,-0.546875,0.271484375,-0.5546875,-3.859375,-0.2216796875,0.86328125,-4.53125,-1.3828125,-0.60546875,-5.46875,-1.3515625,-1.2890625,-3.734375,2.9375,2.40625,-3.984375,0.875,-2.953125,-0.9765625,-1.6328125,-1.25,3.96875,1.6953125,0.0072021484375,5.875,-0.921875,-3.46875,-3.140625,-0.26953125,0.2265625,-2.09375,7.0625,-1.09375,0.30078125,-6.03125,5.34375,2.359375,1.6640625,-0.99609375,4.625,4.25,-2.484375,-4,0.89453125,3.0625,4.1875,-4.28125,3.953125,0.6328125,-0.74609375,-1.53125,2.015625,-1.1796875,1.03125,-1.6484375,-5.4375,0.3671875,1.8125,-0.326171875,1.546875,4.03125,-3.34375,0.484375,2.5,-1.4140625,3.34375,4.25,-1.7890625,1.09375,2.171875,5.34375,-1.5625,0.98828125,-5.09375,-3.625,-2.640625,-2.46875,3.109375,-2.515625,0.09033203125,0.21484375,-3.921875,3.125,-4.1875,1.2109375,1.3671875,1.1875,-5.4375,4.59375,3.890625,-2.8125,3.328125,-5.125,-1.9765625,-1.4296875,2.34375,-2.71875,-5.875,3.125,3.453125,-1.515625,3.546875,2.265625,-0.52734375,1.9375,-2.859375,2.703125,-3.359375,4.75,1.2734375,3.09375,3.65625,-0.255859375,-0.1044921875,-5.75,-0.3359375,-0.77734375,-2.234375,6.1875,-3.84375,0.19921875,4.25,6.4375,-10.5,-1.5078125,0.7265625,0.2890625,3.921875,5.0625,0.09814453125,0.68359375,3.109375,1.015625,2.671875,0.0257568359375,-0.4765625,-4,5.15625,0.2314453125,-4.6875,3.1875,3.984375,-2.609375,3.4375,-2.375,-3.734375,-0.07568359375,2.75,-5.3125,1.9296875,4.625,-1.6484375,2.875,3.734375,-1.34375,3.875,-1.9921875,-11.3125,-1.53125,3.296875,5.71875,0.80859375,1.7578125,0.48046875,-2.015625,1.4765625,-0.5546875,0.71484375,-0.7578125,-11.1875,0.9765625,-3,-0.09765625,-1.9453125,-3.8125,-2.5,4.375,1.65625,1.1015625,3.328125,2.84375,0.84375,4.5625,0.11279296875,-5.84375,1.1484375,1.7578125,-4.8125,-0.59765625,3.234375,1.125,-1.859375,-2.515625,3.78125,-1.7421875,-0.69921875,5.8125,3.765625,1.578125,-1.84375,-5.03125,0.984375,-3.375,-1.9140625,1.1953125,-0.384765625,2.8125,-2.203125,2.828125,1.1171875,-3.75,-4.15625,-2.25,-3.5625,1.5,2.671875,2.171875,-2.609375,-1.7265625,2.8125,2.5,-0.455078125,-1.546875,2.1875,-0.1884765625,-2.984375,-1.4765625,2.0625,-4.46875,-2.90625,4.0625,1.8359375,0.443359375,-0.7734375,-3.140625,2.171875,1.734375,-1.8515625,-1.84375,-1.234375,2.15625,5.34375,-2.484375,-5.6875,-1.2734375,0.1806640625,-4.375,-3.5625,0.89453125,-1.15625,0.75,3.09375,-2.25,1.1875,4.6875,-1.3359375,-3.875,3.53125,4.4375,-2.671875,-0.75,-0.458984375,-2.53125,3.8125,5,-1.2421875,-2.109375,-0.50390625,-2.734375,-4.90625,1.0234375,2.421875,-3.34375,-10.125,6.46875,3.671875,5.40625,1.546875,-2.59375,3.8125,-1.6953125,3.703125,-0.423828125,0.82421875,1.515625,-7.59375,-2.40625,-2.0625,-5.0625,0.59375,-0.345703125,-4.75,1.4921875,6.25,-2.15625,-1.8671875,-2.703125,-3.9375,4.28125,-3.484375,-5.9375,1.984375,-7.4375,1.4609375,-1.9609375,3.265625,-5.875,1.8359375,-0.017333984375,2.046875,-0.5859375,-0.671875,-2.328125,1.1953125,-2.65625,3.625,0.7890625,3.9375,-0.365234375,2.90625,-1.2421875,0.314453125,-3.265625,1.6640625,1.7109375,0.60546875,0.384765625,2.296875,-2.28125,-0.8046875,-1.0546875,1.046875,2.796875,0.61328125,-0.625,0.10693359375,4.21875,-0.6484375,2.03125,-2.3125,-0.173828125,-1.015625,-0.224609375,0.74609375,-0.86328125,0.0145263671875,0.1318359375,1.7109375,1.421875,0.486328125,-0.19921875,0.140625,1.2734375,1.015625,1.5625,-1.65625,-0.45703125,-0.435546875,-0.0206298828125,1.828125,1.734375,-2.734375,1.65625,-2.09375,-0.6875,-0.2421875,2.125,1.1015625,0.1064453125,1.59375,-1.875,1.828125,0.15234375,-1.2421875,1.25,-0.765625,-2.265625,2.34375,-2.109375,-0.921875,0.6640625,-1.2734375,-1.4765625,-0.73828125,2.21875,-0.84375,1.328125,-1.171875,-0.181640625,0.306640625,-1.171875,0.279296875,0.94140625,1.171875,-3.921875,3.15625,1.2421875,0.52734375,-0.1630859375,1.0390625,-1.46875,-0.08447265625,1.0390625,-0.37109375,0.921875,1.859375,-1.8046875,0.54296875,-0.8203125,-1.09375,1.1640625,1.515625,0.54296875,-1.65625,-1,1.5234375,1.4453125,-1.1953125,0.359375,-0.062255859375,-2.09375,3.03125,1.21875,-3.15625,-0.357421875,-0.169921875,0.546875,-0.73828125,-0.126953125,1.046875,-2.75,-0.2314453125,0.2421875,0.306640625,-1.1328125,1.8984375,0.00469970703125,3.9375,0.8515625,1.1328125,1.1875,1.3984375,2.046875,-1.3515625,0.25390625,-0.9921875,3.234375,-0.373046875,0.8828125,1.3828125,-1.921875,-0.484375,-0.81640625,0.61328125,1.4296875,-0.70703125,-0.404296875,2.53125,1.625,0.494140625,2.375,-2.03125,0.33984375,0.291015625,-0.68359375,-1.625,1.625,-0.478515625,0.349609375,-2.0625,-1.25,-0.1484375,-0.44140625,0.67578125,0.3671875,0.4921875,0.236328125,1.1953125,0.5078125,-2.375,1.3671875,-0.341796875,0.6328125,-1.7265625,-1.328125,0.84375,-0.08935546875,1.0625,0.90625,1.984375,2.828125,1.109375,-1.3671875,1.03125,1.0625,1.75,0.263671875,-1.234375,-0.09228515625,-0.13671875,0.271484375,0.58203125,-0.9375,-1.28125,0.4609375,-0.95703125,-0.1552734375,-1.5703125,3.375,-0.9609375,-1.1796875,-0.419921875,-1.5,0.58984375,-1.3125,1,-1.578125,2.484375,1.34375,3.34375,1.4296875,-0.671875,-0.984375,0.30859375,0.72265625,-0.337890625,-0.06982421875,-1.125,-0.44921875,-0.62890625,5.40625,0.263671875,1.0390625,-2.03125,3.296875,0.68359375,-0.10986328125,-1.078125,-0.2412109375,-2.078125,-0.13671875,-1.4375,-1.390625,0.29296875,-1.1484375,-4.0625,-2.703125,-0.302734375,0.77734375,-1.640625,-0.0390625,3.890625,0.375,1.2890625,1.5,2.640625,0.19140625,-1.78125,-0.5859375,1.6328125,-1.234375,2,0.8125,-1.9453125,-2.78125,-0.3671875,-2.328125,-1.9453125,-0.59375,-0.8046875,1.9921875,-0.265625,-0.03515625,-1.3125,-1.5234375,-3.03125,-0.458984375,-0.1279296875,2.375,1.53125,0.67578125,-0.55078125,-0.4296875,0.515625,-1.75,0.6640625,-1.65625,4.25,-0.326171875,-1.4296875,2.53125,0.396484375,3.140625,0.859375,-1.3671875,-1.8828125,-0.828125,0.45703125,0.7109375,3.0625,-0.2578125,0.6328125,0.57421875,-0.85546875,0.5625,1.0234375,-0.296875,-4.84375,-1.578125,-0.486328125,2.59375,-1.2109375,0.09765625,2.59375,-0.87109375,-0.7890625,-1.7421875,-2.34375,-0.2490234375,-0.82421875,0.8046875,2.078125,-0.7265625,-0.10400390625,-0.703125,-1.046875,0.46875,-1.7734375,1.09375,-0.30859375,0.0181884765625,0.2734375,-2.703125,-0.470703125,0.67578125,-1.921875,-1.0078125,1.6328125,0.2021484375,1.359375,1.6796875,-1.6015625,1.5703125,0.6484375,-2.859375,-0.63671875,-0.8359375,1.34375,0.0556640625,0.4375,1.765625,-1.1484375,-1.90625,-1.453125,0.57421875,0.84375,-0.349609375,0.251953125,-0.0927734375,0.416015625,-0.40625,-2.71875,-0.48046875,0.4140625,-0.2109375,0.96484375,1.0859375,1.453125,1.15625,1.375,-0.478515625,1.375,-1.8828125,1.6484375,0.9921875,-2.171875,0.5859375,2.03125,-2.125,0.314453125,1.1796875,-0.4921875,-0.72265625,-0.80078125,0.5546875,-0.52734375,0.58203125,-0.52734375,1.9453125,1.71875,-0.328125,1.453125,-2.203125,-2.09375,-2.625,0.2177734375,-0.82421875,0.3359375,-2.203125,1.375,-1.7578125,-0.072265625,-0.4765625,-0.38671875,-1.9453125,1.5625,1.7578125,0.4453125,0.640625,0.0255126953125,-0.5703125,3.796875,-1.0703125,-0.1201171875,0.93359375,1.15625,-2.078125,3.484375,0.5234375,2.109375,0.0037078857421875,1.3359375,-0.796875,1.25,0.1455078125,0.86328125,0.478515625,1.828125,0.31640625,-0.296875,-0.154296875,-1.53125,-1.1640625,0.6484375,1.0703125,-5.375,0.86328125,0.890625,0.48828125,0.84765625,-2.828125,1.1015625,0.4765625,3.296875,-0.00408935546875,-0.40234375,3.421875,0.61328125,-1.46875,1.1875,0.953125,0.0771484375,-2.78125,-1.171875,-0.86328125,2.9375,-1.0703125,0.1015625,-0.279296875,-0.90625,3.046875,0.6796875,-1.6640625,1.453125,0.443359375,-0.439453125,-1.453125,-3.40625,-0.1689453125,1.71875,-0.9453125,2.234375,0.158203125,0.87109375,0.66796875,-1.640625,1,0.265625,0.267578125,-0.90625,1.75,-0.2041015625,-1.59375,1.65625,-1.1484375,-1.78125,2.421875,1.6953125,-2.328125,0.027587890625,-0.494140625,-0.3203125,-0.01953125,0.58203125,-2.28125,0.546875,0.62109375,0.90625,-0.921875,-1.53125,2.484375,1.890625,2.953125,2.359375,-0.90234375,0.171875,-2.234375,0.33984375,-0.45703125,-0.87109375,0.08251953125,1.8671875,-1.0078125,1.5703125,-0.30078125,0.921875,-1.8046875,1.609375,2.703125,0.92578125,0.40625,-0.26171875,-0.322265625,-1.8671875,-0.5,-2.296875,0.62109375,0.6953125,1.1640625,0.1376953125,-1.4296875,1.5390625],"index":0,"object":"embedding"},{"embedding":[-2.28125,-0.7734375,-0.8359375,-2.3125,3.046875,4.125,-1.0390625,-2.890625,0.0103759765625,1.9296875,0.1015625,1.75,2.4375,2.015625,5.09375,1.203125,-2.140625,-2.828125,-1.328125,-4.6875,1.0078125,6.8125,0.578125,-4.71875,-0.80859375,-6.25,1.578125,4.25,4.46875,-1.0078125,8,-2.3125,2.546875,-0.00555419921875,1.5625,-1.8671875,-2.375,-2.53125,5.25,-0.69140625,-2.96875,-0.68359375,1.6171875,2.96875,-3.015625,-1.734375,0.4140625,-2.9375,2.53125,-1.6640625,-4.5625,-1.9296875,3.234375,-2.734375,2.359375,-4.125,-3.046875,4.5,-5.875,-2.984375,-1.8515625,-2.8125,-0.7734375,0.46484375,1.3984375,5.28125,0.68359375,-1.3359375,0.51171875,8.625,-0.055908203125,3.578125,6.5,-2.390625,6.34375,5.5625,0.7265625,1.578125,-2.921875,4.90625,-2.953125,-0.62890625,2.453125,3.46875,4.5625,2.671875,-1.9140625,0.859375,-3.03125,1.703125,1.96875,0.59375,-1.4140625,-3.140625,-1.2109375,1.2890625,-3.21875,-6.5625,-6.78125,2.765625,-0.78515625,-0.3515625,1.8125,-4.53125,-5.03125,2.171875,-1.8515625,-5.46875,-1.78125,0.380859375,2.640625,1.65625,3.640625,-2.140625,2.46875,1.21875,4.28125,-2.796875,-4.40625,2.796875,-2.0625,-1.9765625,4.28125,-0.6796875,4.4375,4.28125,-4.03125,-0.01416015625,5.53125,-1.4609375,7.25,3.578125,3.6875,-2.375,-8.0625,-4.71875,-1.9453125,3.71875,4.3125,4.40625,-5.03125,3.21875,-3.734375,-6.625,4.1875,-3.4375,-6.4375,-3.15625,3.859375,-1.9140625,-1.78125,1.8046875,0.5,2.3125,-1.2421875,-4.375,4.0625,3.875,0.1259765625,-1.0546875,2.015625,3.328125,1.1484375,1.7265625,1.8046875,-0.462890625,-5.625,3.6875,-1.0390625,2.5625,0.90625,10.4375,4.28125,-4.5625,1.9765625,8.625,-1.328125,8.625,1.4609375,2.203125,0.81640625,-0.640625,-2.90625,4.53125,-2.15625,1.5,0.12255859375,-5.6875,3.140625,1.2890625,1.578125,1.5625,2.71875,-1,-4.84375,-1.8671875,3.484375,-2.578125,3.4375,0.1025390625,-1.40625,-7.375,1.4921875,1.5546875,-4.71875,-3.765625,2.703125,-1.71875,3.078125,-0.380859375,2.265625,0.24609375,3.21875,-2.0625,7.65625,2.640625,2.734375,2.046875,1.8359375,2.46875,4.53125,3.484375,1.8359375,-2.078125,-0.83984375,2.03125,5.8125,0.439453125,3.75,8.6875,0.251953125,0.408203125,6.84375,-2.515625,-1.78125,-3.578125,-3.78125,1.6015625,-0.279296875,2.671875,-5.65625,-4.0625,-2.328125,2.984375,3.515625,-3.359375,-2.34375,-2.703125,-0.51171875,-6.4375,1.484375,3.671875,-9.0625,1.8828125,5.625,3.96875,1.984375,1.265625,-0.33203125,-4.125,0.333984375,-2.4375,-5.875,-0.58203125,1.890625,-2.390625,5.09375,-1.5546875,3.515625,-0.7421875,5.1875,-2.28125,-0.0927734375,-3.046875,-4.3125,8.8125,-0.232421875,-1.90625,1.0703125,-3.078125,-3.5625,-10.25,2.5,1.1171875,4.96875,-2.921875,1.40625,0.40234375,-3.640625,12.75,3.90625,-1.8203125,1.9921875,-0.63671875,-6.03125,-1.984375,-2.046875,2.046875,-5.59375,1.84375,3.6875,4.5,-1.9296875,3.4375,-1.7421875,-0.9296875,-1.109375,-4.5625,-1.9375,2.671875,-3.765625,2.34375,9.625,-4.75,2.03125,-2.109375,-6.1875,4.75,-0.03662109375,-0.11376953125,-2.140625,-5.125,-1.9921875,-2.78125,-1.4296875,-6.65625,4.96875,-0.984375,5.375,0.97265625,3,3.296875,-4.1875,-5.03125,8.4375,-1.5,3.296875,5.71875,0.55078125,0.68359375,-3.515625,-4.6875,2.46875,-5.46875,0.953125,5.71875,3.328125,-1.640625,1.0234375,-6.21875,2.40625,2.328125,-0.68359375,6.53125,6.90625,-2.265625,2.78125,1.9140625,-0.71484375,-2.28125,-0.2294921875,-1.078125,6.34375,1.1875,-3.890625,-3.796875,-0.5859375,5.03125,-2.375,0.7734375,-1.21875,-4.15625,2.59375,-1.15625,3.6875,0.91796875,0.90625,-1.8046875,-5.125,0.087890625,-2.625,0.29296875,-1.7734375,-3.28125,4.25,1.515625,-0.484375,1.59375,0.67578125,-3.53125,-0.46484375,0.59765625,-1.15625,0.65625,2.5625,-0.5703125,-0.984375,1.5546875,-0.3828125,-2.21875,1.0546875,-1.2734375,2.40625,-6.9375,-0.6484375,-0.2490234375,-2.125,-8.375,-0.4765625,1.0703125,-3.78125,2.71875,1.96875,-1.2578125,-3.0625,4.4375,1.421875,1.8671875,-6.90625,2.15625,-1.8828125,3.328125,2.140625,-1.7421875,0.59375,-1.4296875,-2.765625,4.375,3.546875,-0.69921875,3.453125,0.68359375,-3.265625,-3.625,0.1630859375,-4.90625,4.75,-0.236328125,-1.859375,5.21875,2.203125,-1.5,1.625,0.98828125,-6.28125,-4.78125,2.96875,3.171875,-3.078125,-3.96875,0.470703125,-1.4296875,-4.4375,3.078125,3.84375,-1.1171875,-2.8125,3.40625,4.375,-2.203125,0.0830078125,1.1171875,0.52734375,2.703125,-1.9375,-3.140625,-0.1103515625,0.130859375,4.71875,-5.8125,-6.84375,3.015625,-2.875,0.2001953125,1.15625,4.5625,0.46875,-1.8984375,-1.9296875,-3.0625,-3.46875,-2.828125,3.53125,-1.078125,-2.53125,-2.90625,0.29296875,8.3125,1.90625,0.369140625,-2.375,-0.11572265625,2.453125,-1.71875,0.50390625,4.4375,7.90625,-4.03125,-0.63671875,3.53125,-8.125,0.94921875,-1.375,-1.15625,-0.94921875,2.3125,2.1875,-6.25,-0.7890625,0.0115966796875,5.03125,-3.453125,-3.828125,5.15625,-4.8125,-3.09375,1.859375,-0.6875,4.0625,1.296875,-1.34375,2.875,2.984375,2.65625,1.8203125,-2.53125,-3.640625,-3.3125,1.2890625,2.265625,-2.234375,2.296875,4,-5.4375,0.90234375,-2.25,-0.6953125,-0.212890625,-0.515625,5.90625,2.125,2.25,-6.09375,1.2578125,0.50390625,-0.416015625,-0.7421875,-1.1484375,6.71875,-0.5,-0.2294921875,0.94921875,2.09375,-1.1953125,1.640625,-3.796875,-2.453125,-3.109375,-1.796875,-1.0234375,-4.03125,-5.5,4.4375,6,-1.234375,-1.6796875,2.171875,5.5,3.984375,-0.84375,1.515625,3.421875,-2.5,0.23828125,-5.40625,2.609375,-7.84375,-2.53125,-1.6875,2.921875,3.75,-4.15625,3.765625,-2.578125,2.4375,-1.4375,4.4375,-10.5625,2.046875,-2.15625,-2.796875,-2.28125,-0.57421875,3.171875,-0.44921875,2.109375,1.3671875,-0.75,3.953125,5.46875,-1.5,1.765625,2.1875,2.46875,-0.5859375,2.515625,-2.125,-8.25,1.3125,-1.1484375,1.09375,7.5625,1.9375,-1.7734375,2.46875,0.88671875,-1.5703125,-1.7265625,4.0625,3.015625,-1.546875,4.25,-3.90625,5.40625,-3.28125,1.7265625,-3.265625,-6.15625,0.279296875,1.9296875,-5.5625,-4.09375,2.859375,0.216796875,5.78125,3.421875,-5.375,1.21875,-0.41796875,1.109375,2,0.30078125,-0.03759765625,-4.75,3.921875,4.1875,-2.40625,7.03125,-1.5703125,-1.6484375,-1.1171875,2.40625,-1.7734375,0.373046875,1.84375,0.287109375,-0.78125,-3.484375,0.96484375,0.5703125,-6.625,-7.21875,1.7265625,-1.7734375,7.0625,0.73046875,-0.859375,-3.15625,2,1.5546875,6.375,3.3125,3.765625,4.5,3.765625,-2.390625,2.671875,-3.6875,-6.09375,7,-6.53125,-1.8515625,1.015625,0.859375,-0.2578125,-1.0234375,-0.3515625,-0.71484375,-3.484375,-6.09375,-2.359375,-1.875,2.015625,-1.6484375,2.203125,0.57421875,-4.09375,-0.5703125,-1.6484375,-1.6875,-1.6640625,4.15625,-5.625,1.484375,5.71875,2.046875,-1.5234375,4.15625,3.09375,-0.47265625,-4.78125,0.7109375,-6.875,1.6015625,1.46875,-0.6015625,0.50390625,-8,2.03125,-2.4375,3.5,-0.671875,-0.05078125,-1.265625,-3.296875,-1.3984375,-0.91796875,-5.40625,-0.171875,1.6953125,1.125,-1.8359375,0.671875,3.078125,-0.52734375,0.384765625,-1.125,2.046875,0.40625,2.34375,-4.78125,-2.90625,1.28125,0.9140625,-2.03125,6.53125,0.91796875,0.79296875,3.546875,1.7265625,-5.5,-5.78125,3.921875,-2.8125,-1.796875,-3.25,2.421875,-1.359375,6.53125,-2.21875,-5.53125,-3.703125,1.6484375,3.15625,-2.609375,-3.09375,4.78125,1.8359375,2.765625,-2.15625,-7.5,1.609375,0.98828125,-0.146484375,-1.140625,8.625,-1.9296875,-0.4765625,-4.4375,-3.234375,2.046875,0.875,2.046875,-0.76171875,-1.2734375,0.69921875,0.4765625,-2.34375,-0.55078125,0.6015625,-2.546875,1.75,0.07177734375,4.875,-2.53125,0.3984375,-1.2734375,-0.50390625,-0.10009765625,4.3125,8.75,-1.765625,-0.96875,0.35546875,2.984375,-3.59375,6.6875,1.3515625,7.75,-1.1640625,0.25,1.03125,0.375,-2.171875,4.59375,-5.25,-2.84375,-1.890625,1.21875,-2.5625,0.671875,-3.984375,-0.498046875,4.40625,-0.455078125,-0.007568359375,2.609375,0.79296875,-0.201171875,-3.09375,-1.3125,-4.71875,-2.515625,-0.14453125,2.03125,-3.03125,-0.4921875,-0.33984375,5.84375,-0.357421875,-1.4453125,-2.59375,1.53125,1.859375,1.171875,-0.8046875,0.255859375,0.58984375,3.3125,-1.015625,-4.34375,-0.94921875,8.4375,4.21875,-6.875,1.5703125,-0.43359375,1.4453125,-4.8125,-1.4609375,-2.15625,-1.4921875,-4.1875,1.1328125,0.419921875,-3,-0.06494140625,4.5,-1.2890625,-0.15625,3.46875,4.0625,0.478515625,2.96875,-2.125,4.375,2.21875,-2.09375,-5.96875,-1.703125,0.48046875,-2.75,-1.4140625,2.03125,6.15625,0.55859375,2.625,-1.0625,2.28125,-1.6953125,3.78125,5.125,-4.59375,-2.703125,-2.3125,-9.5625,-4.03125,-1.7421875,-2.921875,-5.34375,-4.25,-0.86328125,-1.2421875,-8,0.0966796875,-2.234375,-3.265625,1.4453125,2.953125,1.7578125,-5.75,3.125,4.125,2.578125,2.546875,0.84765625,5.46875,-0.050537109375,-2.96875,1.4453125,-3.4375,4.15625,-1.03125,3.546875,6.25,-0.453125,-4.96875,4.78125,2.96875,5.53125,-7.375,-2.625,-0.337890625,-1.671875,-0.458984375,-1.7578125,2.546875,-4.5,-5.5,1.078125,-3.203125,1.2265625,4.6875,-0.8046875,6.78125,1.6328125,0.419921875,2.140625,2.71875,0.62109375,0.169921875,1.7421875,-5.9375,3.234375,-2.171875,3.265625,-0.296875,-1.5234375,2.734375,-0.7578125,-0.310546875,2.8125,2.734375,10.3125,0.515625,4,-2.3125,0.63671875,-1.7265625,-0.2392578125,2.25,2.015625,0.79296875,-1.4765625,0.7890625,-0.44921875,0.478515625,-0.4609375,-13.25,-1.9609375,-7.25,-1.9296875,7.0625,-2.1875,-1.9921875,1.4296875,2.6875,3.484375,5.125,-0.58984375,3.375,-0.60546875,0.80859375,5.96875,-4.25,1.03125,3.359375,2.546875,5.21875,0.154296875,-0.44921875,-3.203125,8,2.25,-1.4140625,0.8359375,2.796875,-1.3046875,-2.34375,3.09375,-3.171875,2.96875,-4.9375,0.5859375,4.15625,0.65625,-3.890625,-3.4375,-2,-0.62890625,1.3828125,1.375,-2.59375,0.18359375,0.94921875,-4.1875,3.328125,-0.59375,0.140625,-5.53125,1.03125,4.65625,0.703125,-0.109375,-1.8515625,1.4453125,-0.8984375,4.3125,2.78125,-2.734375,0.2734375,2.21875,1.7421875,-0.125,1.03125,1.1328125,2.921875,-3.09375,-0.353515625,-0.44140625,-1.625,1.4765625,-3.1875,1.6640625,3.203125,1.3984375,-3.984375,2.21875,0.79296875,-0.11669921875,2.96875,-5.125,-1.9921875,-1.1015625,-0.71484375,-4.0625,-0.9140625,-4.375,-0.1455078125,5.46875,-5,3.4375,-2.515625,8.1875,0.1298828125,-1.421875,1.2890625,-2.828125,2.59375,-3.390625,-1.234375,3.484375,-0.92578125,2.125,-3.546875,1.8984375,-2.078125,-0.46484375,6.09375,-3.953125,-1.9765625,0.7421875,3.21875,-5.0625,-3.296875,0.1611328125,0.8515625,0.009765625,-1.8984375,1.4765625,-2.03125,4.4375,-4.75,3.390625,-4.65625,-3.90625,0.28125,0.07568359375,7.90625,4.25,-3.796875,-3.421875,-0.6015625,-7.0625,-3.421875,-3.859375,6.65625,-0.52734375,0.96875,2.078125,2.390625,-0.01031494140625,1.46875,-2.96875,3.203125,5.28125,0.294921875,3.046875,2.1875,-1.125,-4.40625,0.3125,-3.171875,7.0625,3.0625,0.404296875,3,-1.8984375,1.484375,-1.03125,-1.0625,-2.828125,2.171875,1.71875,-2.5,-3.28125,1.046875,-3.859375,0.72265625,-5.40625,-2.578125,-5.3125,2.765625,2.3125,-0.81640625,-0.7578125,4.4375,0.318359375,3.328125,-5.53125,-3.890625,3.8125,0.9765625,0.333984375,2.84375,-0.6796875,-5.03125,-0.9375,0.201171875,1.9140625,-4.1875,-3.609375,3.328125,2.46875,0.283203125,-3.9375,-4.40625,-3.453125,2.390625,4.1875,-0.96484375,0.353515625,0.06005859375,-1.53125,2.171875,-2.65625,4.5,-3.109375,-4.15625,-0.47265625,0.734375,3.578125,-3.203125,-1.0703125,1.4296875,-3.4375,0.7578125,1.2734375,-0.11279296875,-1.9453125,3.171875,-2,-3.65625,-5.4375,5.78125,-2.0625,0.45703125,-3.875,-2.65625,-3.1875,-1.421875,-0.6640625,1.7421875,0.0703125,5.78125,-0.63671875,2.8125,0.478515625,-0.8828125,0.0712890625,3.453125,-0.271484375,-2.90625,1.8359375,-4.59375,-4.65625,0.7578125,-8.0625,-2.0625,2.90625,-2.40625,2.671875,-2.671875,2.375,-1.1015625,-2.21875,-1.8203125,-0.8203125,0.83984375,5.375,2.171875,0.2216796875,0.38671875,1.8984375,0.859375,-1.109375,-1.8515625,-0.25,5.34375,0.62109375,2.765625,-3.359375,-2.34375,4.46875,-0.59375,-3.75,0.8984375,-0.357421875,0.6640625,4.5625,0.9609375,-3.796875,-2.9375,-6.15625,4.03125,0.73828125,1.828125,-4.625,1.5,-3.0625,0.1748046875,2.03125,-6.5625,-2.546875,3.328125,2.828125,5.46875,1.328125,-2.421875,-4.53125,2.203125,-0.396484375,-1.6171875,-2.234375,-1.7265625,-0.96875,-3.765625,4.125,-2.515625,4.25,-1.3359375,-2.8125,-0.8671875,0.61328125,-0.203125,0.47265625,-0.353515625,-0.88671875,4.0625,-0.3515625,7,2.171875,-4.0625,4.59375,2.515625,0.412109375,-1.5625,3.75,-1.109375,-2.3125,3.921875,2.890625,-4.0625,4.96875,2.125,3.375,-3.46875,-2.1875,-0.9921875,4.5625,0.287109375,1.28125,-4.34375,0.1630859375,4.0625,-0.1884765625,0.8671875,-1.765625,0.3046875,0.65234375,0.52734375,2,1.921875,3.4375,-0.52734375,1,-0.92578125,-1.2265625,2.328125,-0.1328125,-0.703125,-1.8828125,3.21875,-1.6953125,-1.875,-6,1.2421875,-3.46875,2.21875,3.1875,2.875,2.234375,-2.828125,-1.625,-2.640625,-5.25,-3.140625,1.75,1.09375,-1.75,1.875,-0.1181640625,2.546875,5.84375,0.130859375,4.6875,-3.109375,2.5,1.140625,0.875,0.046630859375,4.3125,-1.8203125,-2.21875,3.640625,-4.46875,3.71875,-4.53125,-3.078125,-0.63671875,-0.10986328125,2.640625,6.625,-4.5625,-3.953125,5.21875,1.328125,4.59375,3.78125,-2.078125,-1.484375,0.79296875,1.3515625,5.46875,0.93359375,2.953125,-2.734375,6.9375,5.65625,0.90625,2.359375,0.166015625,-2.6875,-6.4375,5.125,1.3984375,1.984375,-2.375,1.6875,3.109375,0.1533203125,3.640625,-5.5,0.8671875,1.2109375,0.90625,0.5234375,-3.15625,0.103515625,2.640625,0.33203125,-1.6875,5.84375,0.97265625,4.125,-0.72265625,3.34375,2.328125,3.703125,-2.03125,1.5234375,-3.46875,3.578125,-1.3984375,2.15625,-5.5,1.0546875,3.640625,4.3125,-1.625,-3.5625,2.21875,0.275390625,-0.5,-4.46875,4.21875,3.59375,2.5625,-6.9375,-3.328125,-0.05029296875,0.2060546875,1.234375,-3.484375,1.171875,1.6796875,-4.625,-3.265625,1.296875,1.625,-5.65625,-6.0625,-3.203125,1.65625,1.3203125,3.1875,3.21875,-0.8203125,3.40625,-0.55078125,3.046875,4.28125,-1.1328125,1.5546875,0.9375,-2.75,4.125,-0.263671875,-2.671875,1.5546875,-0.50390625,-2.140625,0.50390625,-2.296875,-1.0703125,-4.21875,-0.85546875,2.328125,-1.09375,5.125,-3.96875,0.30078125,3.609375,-1.4375,-2.28125,-2.65625,0.5703125,-2.921875,-2.578125,-1.9140625,3.609375,2.984375,2.046875,0.58203125,-0.6015625,-3.265625,-6.40625,-5.65625,3.578125,-2.515625,2.859375,0.439453125,-4.25,2.078125,2.8125,1.78125,-0.1640625,-0.55859375,2.765625,4.59375,0.455078125,-1.7265625,-0.466796875,3.609375,-4.5625,-3.78125,0.515625,1,-3.171875,2.28125,-3.125,-1.8359375,0.79296875,4.5,-0.5078125,-2.859375,-1.75,-2.40625,-2.875,-3.03125,-2.859375,2.5625,1.859375,3.296875,0.1689453125,-0.421875,-5,3.71875,16.875,0.9375,-4.71875,2.421875,-3.140625,2.65625,3.171875,4.8125,-1.7109375,-1.96875,-2.1875,1.765625,0.01031494140625,1.4140625,-2.140625,1.7421875,1.9921875,-0.48828125,-4.125,-1.9765625,-1.328125,0.84765625,-0.7578125,2.96875,0.408203125,2.265625,-0.734375,-0.259765625,0.2333984375,-3.234375,-4.46875,-4.4375,2.265625,-1.7578125,4.75,-4.25,5.375,0.1845703125,-2.9375,-2.09375,-3.296875,-3.171875,1.0234375,-0.75,-1.9453125,4.34375,-0.72265625,1.09375,0.37890625,-0.337890625,-3.546875,-3.046875,-2.6875,7.25,0.62890625,-5.71875,-1.546875,-4.84375,-4.5625,0.58984375,2.796875,-2.328125,1.6328125,1.453125,-1.828125,-2.171875,-1.953125,0.85546875,3,-5.125,-5.625,0.13671875,1.5546875,3.359375,2.796875,-4.0625,1.5703125,5.3125,2.6875,0.69140625,-0.75,1.4453125,-1.3828125,-2.5,-0.91015625,1.4609375,-4.03125,1.109375,1.4453125,-4.875,11.25,-8.625,4.8125,4.0625,-4.75,-0.1865234375,2.796875,1.796875,-1.6796875,-0.169921875,2.953125,2.453125,3.359375,-0.306640625,6.09375,1.5234375,0.388671875,0.73828125,2.9375,3.578125,2.4375,2.9375,-0.828125,-1.9609375,1.3046875,1.7734375,-2.484375,-3.46875,-1.4609375,-4.4375,6,1.6171875,-2.765625,-1.2578125,-10.5,-3.421875,-2.328125,-5.84375,4.5,-2.65625,2.46875,3.421875,-0.609375,-1.078125,-2.53125,-5,2.296875,4.0625,0.208984375,-0.3984375,-6.0625,2.84375,3.546875,-3.984375,-2.09375,1.4453125,-3.265625,3.296875,-0.1923828125,4.9375,-3.578125,3.9375,2.03125,-2.546875,-5.8125,3.171875,-3.765625,-2.234375,-5.3125,-2.453125,-2.078125,-3.328125,-0.6171875,-0.35546875,-2.078125,-1.03125,1.6171875,-0.60546875,-3.15625,2.921875,2.96875,-4.375,-2.625,0.58203125,0.73046875,-4.28125,1.1875,5.1875,-0.54296875,1.5,0.55078125,0.078125,-0.3203125,-4.34375,0.81640625,1.71875,-4.03125,-0.71875,-1.359375,-2.828125,-2.4375,-2.78125,-3.375,3.875,3.59375,-5.0625,1.9609375,-0.34765625,0.014892578125,-1.4453125,-1.546875,6.4375,2.234375,-1.6484375,5.59375,1.03125,-4.15625,-2,-2.046875,-1.1484375,-1.2734375,6.3125,1.2578125,2.375,-5.90625,7.53125,2.453125,1.7265625,-0.43359375,2.34375,1.6796875,-3.71875,-5.40625,2.46875,2.75,3.84375,-4.59375,0.6328125,0.53515625,0.53125,-4.28125,1.90625,-0.259765625,0.482421875,-3.140625,-7.59375,-0.109375,0.90625,-1.8828125,1.5234375,4.25,-2.96875,1.3828125,0.95703125,-0.58984375,3.640625,3.28125,-2.828125,1.90625,-0.1904296875,2.625,-2.34375,1.4921875,-3.71875,-4.96875,-3.109375,-1.765625,1.8828125,-2.625,0.67578125,-0.357421875,-4.1875,2.109375,-2.25,1.125,1.09375,0.2578125,-6.25,3.984375,5.1875,-4.15625,4.4375,-5.53125,-2.4375,-1.640625,2.21875,-1.9140625,-6.46875,2.0625,4.5,-3.390625,2.203125,3.546875,-1.625,-0.4453125,-2.25,5.3125,-1.015625,4.78125,-0.6953125,3.953125,3.9375,-1.28125,-0.061279296875,-5.125,0.470703125,-2.28125,-3.84375,5.53125,-1.921875,2.46875,5.21875,4.9375,-9,-1.96875,0.54296875,-0.1845703125,3.578125,3.109375,-1.3671875,1.0234375,0.028076171875,-0.30859375,4.4375,-0.9296875,-1.46875,-3.65625,4.96875,-0.1728515625,-4.0625,2.984375,2.609375,-4.15625,4.34375,-2.75,-2.6875,-0.6875,-0.1396484375,-5.625,1.8046875,2.6875,-0.92578125,3.4375,3.109375,1.203125,3.59375,-2.640625,-10.0625,0.0703125,2.75,5.3125,1.7265625,2.3125,0.0859375,-1.0625,3.640625,-4.5625,0.46875,-1.484375,-9.5,0.255859375,-4.15625,-1.609375,-3.453125,-1.4921875,-1.9453125,3.90625,1.3984375,-0.8515625,3.5,2.921875,0.453125,4.15625,-0.361328125,-3.578125,1.2734375,1.75,-5.28125,-1.90625,4.8125,3.578125,-2.203125,-2.0625,3.84375,-4.28125,-0.70703125,4.3125,4.28125,2.15625,-0.828125,-3.234375,2.84375,-2.546875,-2.828125,1.703125,-3.421875,2.453125,-1.4375,2.578125,1.296875,-2.640625,-2.03125,-4.15625,-2.71875,3.484375,0.28515625,0.9765625,-2.265625,-1.1171875,3.234375,3.5625,-2.359375,-2.109375,2.796875,-1.3515625,-4.28125,-1.0859375,1.0859375,-5.90625,-2.609375,2.734375,3.4375,-2.5625,-3.5625,-2.125,1.6171875,1.3046875,-0.8984375,-0.1318359375,-3.53125,2.65625,5.0625,-2.9375,-3.75,-1.6171875,-0.486328125,-5.03125,-3.609375,-0.1767578125,1.140625,-0.73046875,3.890625,-1.40625,0.47265625,4.4375,-3.65625,-3.21875,3.96875,3.359375,-3.203125,-1.46875,2.25,-3.375,1.03125,5.4375,-2.390625,-2.234375,0.41796875,-2.171875,-4.28125,2.34375,1.2265625,-3.734375,-7.875,5.96875,1.0703125,4.34375,4.125,-3.90625,4.0625,-4.6875,1.8828125,-1.265625,1.015625,1.3828125,-5.65625,-1.1875,-2.5,-3.5,0.5390625,-1.734375,-3.5625,0.66015625,8.0625,-1.328125,-2.59375,-2.953125,-3.515625,3.3125,-4.15625,-7.625,0.1181640625,-7.34375,1.734375,-2.1875,1.75,-5.59375,1.9140625,-1.078125,1.734375,-2.984375,0.27734375,-0.384765625,1.21875,0.54296875,4.6875,1.2109375,1.984375,-0.1484375,2.71875,0.0791015625,1.875,-1.453125,-0.4921875,1.21875,-1.234375,0.33203125,0.69921875,-2.734375,0.1708984375,-1.7578125,-0.263671875,-1.015625,1.7578125,2.9375,-0.640625,-0.291015625,-1.6875,1.703125,-4.5,1.3125,-1.796875,0.859375,-0.78515625,-1.0078125,1.9609375,-2.328125,1.6640625,1.015625,1.640625,0.01068115234375,-1.5,2.234375,2.6875,-0.031982421875,-2.328125,-1.8046875,-0.55859375,-1.7421875,1.7421875,0.55078125,-2.0625,2.9375,-1.640625,-0.41015625,0.890625,1.7265625,0.44140625,-1.6484375,2.40625,-1.8671875,1.2890625,1.0859375,-1.5234375,2.609375,0.63671875,1.03125,1.2734375,0.9765625,-2,0.64453125,0.2578125,-1.4375,-0.291015625,3.484375,-1.7265625,0.31640625,-1.078125,-0.5625,1.0859375,-0.8671875,1.2109375,0.15625,-0.396484375,-2.75,2.640625,-2.125,-1.2578125,-0.42578125,0.29296875,-0.5703125,0.8984375,0.08935546875,1.2109375,-0.29296875,2.28125,-0.73828125,2.171875,-0.020263671875,-0.2060546875,1.3359375,3.421875,-1.984375,0.7421875,-2.0625,-1.1328125,1.3203125,-0.3046875,1.15625,-0.93359375,-2,1.2421875,1.1328125,-2.984375,-0.734375,2.265625,-0.189453125,-1.1328125,-0.609375,1.2265625,-0.75390625,-0.38671875,0.419921875,-0.89453125,2,3.265625,-1.0625,2.5,-1.453125,0.396484375,0.73046875,1.046875,2.3125,0.07958984375,-2.34375,-0.9296875,2.71875,-1.4375,0.37109375,0.890625,-1.53125,-0.1396484375,1.3359375,0.5703125,1.640625,-0.06982421875,-1.859375,-0.330078125,-0.6796875,1.609375,1.65625,-1.6875,0.68359375,-1.8359375,-0.53125,-1.015625,2.765625,-1.7578125,-2.140625,-0.78515625,-1.1015625,-0.83203125,-0.498046875,0.11962890625,-0.1298828125,0.60546875,1.125,1.5,0.4296875,-0.609375,1.4375,-0.08056640625,0.68359375,-1.1875,-1.5234375,1.484375,1.2421875,2.34375,-1.359375,1.34375,0.9296875,0.8828125,-1.1796875,1.9453125,-0.5234375,0.314453125,0.010986328125,-0.1181640625,1.40625,2.21875,0.318359375,0.5859375,-0.1328125,1.40625,0.69921875,1.375,-1.3046875,-2.203125,-1.0078125,-1.4296875,-2.125,0.361328125,-0.0615234375,-1.3046875,-0.1904296875,0.034912109375,-0.86328125,1.375,1.1796875,1.5390625,-0.828125,-0.58203125,0.1787109375,-0.328125,0.25390625,0.8828125,-0.8046875,-0.78125,-1.1171875,-2.0625,1.578125,0.88671875,-1.09375,-0.2890625,2.0625,-1.5,1.0078125,-2.78125,0.55078125,-1.828125,-0.341796875,0.0859375,-3.265625,0.34765625,-0.12451171875,-2.15625,-3.078125,-1.75,-0.85546875,-2.375,-0.3203125,4,-0.81640625,-1.21875,2.03125,0.08203125,-1.0078125,-0.94921875,1.7578125,2.84375,-0.8203125,3.859375,0.349609375,-0.16015625,-1.3984375,-1.265625,0.52734375,-1.2890625,0.294921875,-0.84765625,-0.8046875,-1.6796875,-3.109375,0.05859375,-4.1875,-2.125,0.1337890625,0.90625,1.890625,-0.08447265625,-0.7421875,-0.56640625,-0.96875,2.796875,-0.267578125,0.18359375,1.4375,0.27734375,0.46875,-1.4140625,0.92578125,-0.84375,2.953125,-1.171875,-0.50390625,-2.65625,-1.5546875,-4.1875,1.453125,2.484375,0.421875,2.96875,1.3671875,-0.5546875,-2.5625,0.07421875,0.00909423828125,-4.75,-0.373046875,-0.7265625,0.07275390625,-1.4140625,-0.7109375,-0.1318359375,-0.609375,-1.328125,-0.51953125,-1.828125,-0.271484375,-2.28125,2.984375,1.7890625,1.875,2.3125,0.3125,-0.31640625,1.1875,2.359375,1.1484375,0.6953125,0.255859375,0.408203125,-1.09375,2.09375,0.337890625,0.4609375,-1.2265625,0.2275390625,1.1875,2.5625,1.734375,-0.76171875,0.85546875,0.328125,-1.9140625,-1.40625,0.31640625,0.296875,1.140625,0.333984375,1.03125,-1.2890625,0.416015625,-0.6875,0.9453125,1.7578125,-1.953125,1.109375,-0.134765625,0.1787109375,-1.5,1.203125,1.15625,1.8203125,-0.48046875,2.140625,1.1640625,0.48828125,1.8515625,2.609375,-0.361328125,1.421875,-0.86328125,1.953125,0.51953125,-2.484375,3.15625,-0.34375,-0.47265625,-0.56640625,1.2890625,1.359375,-0.60546875,-0.25,-0.38671875,2.015625,0.52734375,0.14453125,1.8828125,0.67578125,-0.546875,-0.77734375,-0.6015625,-1.09375,-2.328125,-1.0078125,-3.0625,-0.37109375,-0.9375,1.765625,-0.828125,-1.484375,-0.142578125,1.390625,-0.02099609375,1.3203125,1.6171875,-1.0859375,2.09375,0.154296875,0.1962890625,0.89453125,-0.97265625,-1.2421875,1.15625,0.82421875,-0.59765625,4.625,0.1962890625,2.28125,-0.65625,-1.0390625,-0.78515625,3.59375,-0.44921875,-0.4375,-1.6953125,1.140625,-0.296875,-1.25,-0.76953125,-1.3984375,-0.9765625,1.78125,-0.87109375,-3.234375,-2.171875,0.330078125,-1.875,0.48828125,-1.859375,-1.0390625,2.40625,1.734375,-0.63671875,0.216796875,1.125,-1.0234375,0.58984375,-0.4296875,0.3515625,1.6015625,-1.2109375,1.765625,0.5859375,2.796875,-3.921875,-0.298828125,2.171875,1.578125,-0.458984375,-1.015625,-0.51171875,2.109375,0.369140625,-0.018798828125,-0.50390625,-4.46875,0.0135498046875,-0.043212890625,-3.21875,-0.09423828125,0.4921875,1.2421875,0.6640625,-3.15625,0.73046875,-1.5078125,-1.6328125,3.46875,-0.55078125,-0.41796875,0.58203125,1.1640625,-0.83203125,-0.84765625,1.53125,0.17578125,-3.484375,-1.1015625,-0.1591796875,-0.875,0.59765625,0.01373291015625,0.099609375,0.546875,-0.36328125,-1.171875,-1.1328125,-0.33984375,-0.08056640625,1.015625,4,1.1484375,1.265625,1.2109375,-2.125,4.5625,-2.515625,-0.96484375,1.1015625,1.3515625,-1.1796875,3.921875,1.109375,0.2265625,-2,0.55859375,2.96875,0.765625,0.9453125,0.671875,1.28125,1.7421875,1.78125,-1,-1.8671875,1.5,-0.35546875,-2.5,0.012451171875,0.2578125],"index":1,"object":"embedding"}],"model":"doubao-embedding-text-240715","object":"list","usage":{"prompt_tokens":7,"total_tokens":7}}, - "headers": { - "Content-Type": "application/json" - } - } - } -] - - - diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java index c38d4adc927..0e891710733 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/SeaTunnelClient.java @@ -170,24 +170,25 @@ public JobMetricsSummary getJobMetricsSummary(Long jobId) { public Map getClusterHealthMetrics() { Set members = hazelcastClient.getHazelcastInstance().getCluster().getMembers(); Map healthMetricsMap = new HashMap<>(); - members.forEach( - member -> { - String metrics = - hazelcastClient.requestAndDecodeResponse( - member.getUuid(), - SeaTunnelGetClusterHealthMetricsCodec.encodeRequest(), - SeaTunnelGetClusterHealthMetricsCodec::decodeResponse); - String[] split = metrics.split(","); - Map kvMap = new LinkedHashMap<>(); - Arrays.stream(split) - .forEach( - kv -> { - String[] kvArr = kv.split("="); - kvMap.put(kvArr[0], kvArr[1]); - }); - healthMetricsMap.put( - member.getAddress().toString(), JsonUtils.toJsonString(kvMap)); - }); + members.stream() + .forEach( + member -> { + String metrics = + hazelcastClient.requestAndDecodeResponse( + member.getUuid(), + SeaTunnelGetClusterHealthMetricsCodec.encodeRequest(), + SeaTunnelGetClusterHealthMetricsCodec::decodeResponse); + String[] split = metrics.split(","); + Map kvMap = new LinkedHashMap<>(); + Arrays.stream(split) + .forEach( + kv -> { + String[] kvArr = kv.split("="); + kvMap.put(kvArr[0], kvArr[1]); + }); + healthMetricsMap.put( + member.getAddress().toString(), JsonUtils.toJsonString(kvMap)); + }); return healthMetricsMap; } diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties b/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties index 8e1c93a35cc..c5b54e52eb9 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties +++ b/seatunnel-engine/seatunnel-engine-client/src/test/resources/log4j2-test.properties @@ -29,7 +29,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -39,7 +39,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java index 60551202e83..fdb21025814 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/Constant.java @@ -24,8 +24,6 @@ public class Constant { public static final String DEFAULT_SEATUNNEL_CLUSTER_NAME = "seatunnel"; - public static final String REST_SUBMIT_JOBS_PARAMS = "params"; - /** * The default port number for the cluster auto-discovery mechanism's multicast communication. */ diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java index 666021de1f8..a58e68d92d8 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/loader/SeaTunnelChildFirstClassLoader.java @@ -134,12 +134,10 @@ public Enumeration getResources(String name) throws IOException { return new Enumeration() { final Iterator iter = result.iterator(); - @Override public boolean hasMoreElements() { return iter.hasNext(); } - @Override public URL nextElement() { return iter.next(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java index 0e057907138..fe227df50de 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java @@ -21,8 +21,6 @@ import org.apache.seatunnel.api.common.metrics.RawJobMetrics; import org.apache.seatunnel.api.event.EventHandler; import org.apache.seatunnel.api.event.EventProcessor; -import org.apache.seatunnel.api.tracing.MDCExecutorService; -import org.apache.seatunnel.api.tracing.MDCTracer; import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.common.utils.StringFormatUtils; @@ -318,7 +316,7 @@ private void restoreAllRunningJobFromMasterNodeSwitch() { "restore job (%s) from master active switch finished", entry.getKey())); }, - MDCTracer.tracing(entry.getKey(), executorService))) + executorService)) .collect(Collectors.toList()); try { @@ -470,12 +468,11 @@ public PassiveCompletableFuture submitJob( return new PassiveCompletableFuture<>(jobSubmitFuture); } - MDCExecutorService mdcExecutorService = MDCTracer.tracing(jobId, executorService); JobMaster jobMaster = new JobMaster( jobImmutableInformation, this.nodeEngine, - mdcExecutorService, + executorService, getResourceManager(), getJobHistoryService(), runningJobStateIMap, @@ -485,7 +482,7 @@ public PassiveCompletableFuture submitJob( metricsImap, engineConfig, seaTunnelServer); - mdcExecutorService.submit( + executorService.submit( () -> { try { if (!isStartWithSavePoint @@ -574,11 +571,9 @@ public PassiveCompletableFuture waitForJobComplete(long jobId) { } CompletableFuture future = new CompletableFuture<>(); - if (jobState == null) { - future.complete(new JobResult(JobStatus.UNKNOWABLE, null)); - } else { + if (jobState == null) future.complete(new JobResult(JobStatus.UNKNOWABLE, null)); + else future.complete(new JobResult(jobState.getJobStatus(), jobState.getErrorMessage())); - } return new PassiveCompletableFuture<>(future); } else { return new PassiveCompletableFuture<>(runningJobMaster.getJobMasterCompleteFuture()); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java deleted file mode 100644 index 0c7b654b216..00000000000 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/EventService.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.engine.server; - -import org.apache.seatunnel.shade.com.google.common.util.concurrent.ThreadFactoryBuilder; - -import org.apache.seatunnel.api.event.Event; -import org.apache.seatunnel.common.utils.RetryUtils; -import org.apache.seatunnel.engine.server.event.JobEventReportOperation; -import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; - -import com.hazelcast.spi.impl.NodeEngineImpl; -import lombok.extern.slf4j.Slf4j; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -@Slf4j -public class EventService { - private final BlockingQueue eventBuffer; - - private ExecutorService eventForwardService; - - private final NodeEngineImpl nodeEngine; - - public EventService(NodeEngineImpl nodeEngine) { - eventBuffer = new ArrayBlockingQueue<>(2048); - initEventForwardService(); - this.nodeEngine = nodeEngine; - } - - private void initEventForwardService() { - eventForwardService = - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setNameFormat("event-forwarder-%d").build()); - eventForwardService.submit( - () -> { - List events = new ArrayList<>(); - RetryUtils.RetryMaterial retryMaterial = - new RetryUtils.RetryMaterial(2, true, e -> true); - while (!Thread.currentThread().isInterrupted()) { - try { - events.clear(); - - Event first = eventBuffer.take(); - events.add(first); - - eventBuffer.drainTo(events, 500); - JobEventReportOperation operation = new JobEventReportOperation(events); - - RetryUtils.retryWithException( - () -> - NodeEngineUtil.sendOperationToMasterNode( - nodeEngine, operation) - .join(), - retryMaterial); - - log.debug("Event forward success, events " + events.size()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - log.info("Event forward thread interrupted"); - } catch (Throwable t) { - log.warn("Event forward failed, discard events " + events.size(), t); - } - } - }); - } - - public void reportEvent(Event e) { - while (!eventBuffer.offer(e)) { - eventBuffer.poll(); - log.warn("Event buffer is full, discard the oldest event"); - } - } - - public void shutdownNow() { - if (eventForwardService != null) { - eventForwardService.shutdownNow(); - } - } -} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java index 67aac64aca2..aad8c3db7f7 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/LiteNodeDropOutTcpIpJoiner.java @@ -76,7 +76,6 @@ public LiteNodeDropOutTcpIpJoiner(Node node) { joinConfig = getActiveMemberNetworkConfig(config).getJoin(); } - @Override public boolean isClaimingMastership() { return claimingMastership; } @@ -199,7 +198,6 @@ private boolean tryClaimMastership(Collection
addresses) { return consensus; } - @Override protected Collection
getPossibleAddressesForInitialJoin() { return getPossibleAddresses(); } @@ -374,7 +372,6 @@ private Address getRequiredMemberAddress() { } @SuppressWarnings({"checkstyle:npathcomplexity", "checkstyle:cyclomaticcomplexity"}) - @Override protected Collection
getPossibleAddresses() { final Collection possibleMembers = getMembers(); final Set
possibleAddresses = new HashSet<>(); @@ -481,7 +478,6 @@ private boolean isLocalAddress(final Address address) throws UnknownHostExceptio return local; } - @Override protected Collection getMembers() { return getConfigurationMembers(config); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java index 99cd27d5642..b76af4c19a0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java @@ -76,8 +76,6 @@ public class SeaTunnelServer private volatile boolean isRunning = true; - @Getter private EventService eventService; - public SeaTunnelServer(@NonNull SeaTunnelConfig seaTunnelConfig) { this.liveOperationRegistry = new LiveOperationRegistry(); this.seaTunnelConfig = seaTunnelConfig; @@ -118,8 +116,6 @@ public void init(NodeEngine engine, Properties hzProperties) { new DefaultClassLoaderService( seaTunnelConfig.getEngineConfig().isClassloaderCacheMode()); - eventService = new EventService(nodeEngine); - if (EngineConfig.ClusterRole.MASTER_AND_WORKER.ordinal() == seaTunnelConfig.getEngineConfig().getClusterRole().ordinal()) { startWorker(); @@ -153,7 +149,7 @@ private void startMaster() { private void startWorker() { taskExecutionService = new TaskExecutionService( - classLoaderService, nodeEngine, nodeEngine.getProperties(), eventService); + classLoaderService, nodeEngine, nodeEngine.getProperties()); nodeEngine.getMetricsRegistry().registerDynamicMetricsProvider(taskExecutionService); taskExecutionService.start(); getSlotService(); @@ -180,10 +176,6 @@ public void shutdown(boolean terminate) { if (coordinatorService != null) { coordinatorService.shutdown(); } - - if (eventService != null) { - eventService.shutdownNow(); - } } @Override 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 b32dd7c6a97..a4717c6a81b 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 @@ -19,9 +19,8 @@ import org.apache.seatunnel.api.common.metrics.MetricTags; import org.apache.seatunnel.api.event.Event; -import org.apache.seatunnel.api.tracing.MDCExecutorService; -import org.apache.seatunnel.api.tracing.MDCTracer; import org.apache.seatunnel.common.utils.ExceptionUtils; +import org.apache.seatunnel.common.utils.RetryUtils; import org.apache.seatunnel.common.utils.StringFormatUtils; import org.apache.seatunnel.engine.common.Constant; import org.apache.seatunnel.engine.common.config.ConfigProvider; @@ -31,6 +30,7 @@ import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture; import org.apache.seatunnel.engine.core.classloader.ClassLoaderService; import org.apache.seatunnel.engine.core.job.ConnectorJarIdentifier; +import org.apache.seatunnel.engine.server.event.JobEventReportOperation; import org.apache.seatunnel.engine.server.exception.TaskGroupContextNotFoundException; import org.apache.seatunnel.engine.server.execution.ExecutionState; import org.apache.seatunnel.engine.server.execution.ProgressState; @@ -49,10 +49,12 @@ import org.apache.seatunnel.engine.server.task.SeaTunnelTask; import org.apache.seatunnel.engine.server.task.TaskGroupImmutableInformation; import org.apache.seatunnel.engine.server.task.operation.NotifyTaskStatusOperation; +import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; import org.apache.commons.collections4.CollectionUtils; 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; @@ -72,6 +74,7 @@ import java.io.IOException; import java.net.URL; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; @@ -79,6 +82,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; @@ -144,13 +148,13 @@ public class TaskExecutionService implements DynamicMetricsProvider { private final ServerConnectorPackageClient serverConnectorPackageClient; - private final EventService eventService; + private final BlockingQueue eventBuffer; + private final ExecutorService eventForwardService; public TaskExecutionService( ClassLoaderService classLoaderService, NodeEngineImpl nodeEngine, - HazelcastProperties properties, - EventService eventService) { + HazelcastProperties properties) { seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); this.hzInstanceName = nodeEngine.getHazelcastInstance().getName(); this.nodeEngine = nodeEngine; @@ -173,7 +177,42 @@ public TaskExecutionService( serverConnectorPackageClient = new ServerConnectorPackageClient(nodeEngine, seaTunnelConfig); - this.eventService = eventService; + eventBuffer = new ArrayBlockingQueue<>(2048); + eventForwardService = + Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setNameFormat("event-forwarder-%d").build()); + eventForwardService.submit( + () -> { + List events = new ArrayList<>(); + RetryUtils.RetryMaterial retryMaterial = + new RetryUtils.RetryMaterial(2, true, e -> true); + while (!Thread.currentThread().isInterrupted()) { + try { + events.clear(); + + Event first = eventBuffer.take(); + events.add(first); + + eventBuffer.drainTo(events, 500); + JobEventReportOperation operation = new JobEventReportOperation(events); + + RetryUtils.retryWithException( + () -> + NodeEngineUtil.sendOperationToMasterNode( + nodeEngine, operation) + .join(), + retryMaterial); + + logger.fine("Event forward success, events " + events.size()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.info("Event forward thread interrupted"); + } catch (Throwable t) { + logger.warning( + "Event forward failed, discard events " + events.size(), t); + } + } + }); } public void start() { @@ -184,6 +223,7 @@ public void shutdown() { isRunning = false; executorService.shutdownNow(); scheduledExecutorService.shutdown(); + eventForwardService.shutdownNow(); } public TaskGroupContext getExecutionContext(TaskGroupLocation taskGroupLocation) { @@ -236,7 +276,6 @@ private void submitThreadShareTask( private void submitBlockingTask( TaskGroupExecutionTracker taskGroupExecutionTracker, List tasks) { - MDCExecutorService mdcExecutorService = MDCTracer.tracing(executorService); CountDownLatch startedLatch = new CountDownLatch(tasks.size()); taskGroupExecutionTracker.blockingFutures = @@ -253,7 +292,7 @@ private void submitBlockingTask( "BlockingWorker-" + taskGroupExecutionTracker.taskGroup .getTaskGroupLocation())) - .map(mdcExecutorService::submit) + .map(executorService::submit) .collect(toList()); // Do not return from this method until all workers have started. Otherwise, @@ -376,15 +415,10 @@ public PassiveCompletableFuture deployLocalTask( seaTunnelConfig .getEngineConfig() .getTaskExecutionThreadShareMode(); - if (mode.equals(ThreadShareMode.ALL)) { - return true; - } - if (mode.equals(ThreadShareMode.OFF)) { - return false; - } - if (mode.equals(ThreadShareMode.PART)) { + if (mode.equals(ThreadShareMode.ALL)) return true; + if (mode.equals(ThreadShareMode.OFF)) return false; + if (mode.equals(ThreadShareMode.PART)) return t.isThreadsShare(); - } return true; })); executionContexts.put( @@ -425,7 +459,7 @@ public PassiveCompletableFuture deployLocalTask( r.getTaskGroupLocation(), r.getExecutionState())); notifyTaskStatusToMaster(taskGroup.getTaskGroupLocation(), r); }), - MDCTracer.tracing(executorService)); + executorService); return new PassiveCompletableFuture<>(resultFuture); } @@ -497,8 +531,7 @@ public void asyncExecuteFunction(TaskGroupLocation taskGroupLocation, Runnable t if (!taskAsyncFunctionFuture.containsKey(taskGroupLocation)) { taskAsyncFunctionFuture.put(taskGroupLocation, new ConcurrentHashMap<>()); } - CompletableFuture future = - CompletableFuture.runAsync(task, MDCTracer.tracing(executorService)); + CompletableFuture future = CompletableFuture.runAsync(task, executorService); taskAsyncFunctionFuture.get(taskGroupLocation).put(id, future); future.whenComplete( (r, e) -> { @@ -649,7 +682,10 @@ public void printTaskExecutionRuntimeInfo() { } public void reportEvent(Event e) { - eventService.reportEvent(e); + while (!eventBuffer.offer(e)) { + eventBuffer.poll(); + logger.warning("Event buffer is full, discard the oldest event"); + } } private final class BlockingWorker implements Runnable { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java index 515f3ad871d..8735048eac3 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.engine.server.checkpoint; -import org.apache.seatunnel.api.tracing.MDCTracer; import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.common.utils.RetryUtils; import org.apache.seatunnel.common.utils.SeaTunnelException; @@ -188,7 +187,6 @@ public CheckpointCoordinator( return thread; }); ((ScheduledThreadPoolExecutor) this.scheduler).setRemoveOnCancelPolicy(true); - this.scheduler = MDCTracer.tracing(scheduler); this.serializer = new ProtoStuffSerializer(); this.pipelineTasks = getPipelineTasks(plan.getPipelineSubtasks()); this.pipelineTaskStatus = new ConcurrentHashMap<>(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java index 63afe8c20fa..8ed7985fd68 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointBarrierTriggerOperation.java @@ -68,7 +68,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java index 4457b7eda2c..62ec42f0a6a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointEndOperation.java @@ -76,7 +76,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java index a02cb67a49d..967e1572311 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java @@ -40,7 +40,7 @@ public CheckpointErrorReportOperation(TaskLocation taskLocation, Throwable e) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService() .getJobMaster(taskLocation.getJobId()) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java index 1e97bd4b461..40569c987c5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointFinishedOperation.java @@ -76,7 +76,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java index 4cd83941c47..b04477370d9 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskRestoreOperation.java @@ -80,7 +80,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java index 0f0a3b5bddb..62224aea47a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/NotifyTaskStartOperation.java @@ -46,7 +46,7 @@ public int getClassId() { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java index 9262cb3c7d5..7355811613a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskAcknowledgeOperation.java @@ -22,11 +22,11 @@ import org.apache.seatunnel.engine.server.checkpoint.CheckpointBarrier; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -37,8 +37,7 @@ @Getter @AllArgsConstructor @Slf4j -public class TaskAcknowledgeOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class TaskAcknowledgeOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -60,7 +59,6 @@ public int getClassId() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeObject(taskLocation); out.writeObject(barrier); out.writeObject(states); @@ -68,14 +66,13 @@ protected void writeInternal(ObjectDataOutput out) throws IOException { @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); taskLocation = in.readObject(); barrier = in.readObject(); states = in.readObject(); } @Override - public void runInternal() { + public void run() { log.debug("TaskAcknowledgeOperation {}", taskLocation); ((SeaTunnelServer) getService()) .getCoordinatorService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java index 177b6f01394..960787fc5fe 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TaskReportStatusOperation.java @@ -23,12 +23,12 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import org.apache.seatunnel.engine.server.task.statemachine.SeaTunnelTaskState; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -38,8 +38,7 @@ @Getter @NoArgsConstructor @AllArgsConstructor -public class TaskReportStatusOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class TaskReportStatusOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation location; private SeaTunnelTaskState status; @@ -56,20 +55,18 @@ public int getClassId() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeObject(location); out.writeObject(status); } @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); location = in.readObject(TaskLocation.class); status = in.readObject(); } @Override - public void runInternal() throws Exception { + public void run() throws Exception { CoordinatorService coordinatorService = ((SeaTunnelServer) getService()).getCoordinatorService(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java index 5bd4ee5db4d..66be8cc9f30 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeAfterCheckpointOperation.java @@ -20,11 +20,11 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -36,7 +36,7 @@ @Getter @AllArgsConstructor @NoArgsConstructor -public class TriggerSchemaChangeAfterCheckpointOperation extends TracingOperation +public class TriggerSchemaChangeAfterCheckpointOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -53,18 +53,16 @@ public int getClassId() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); taskLocation = in.readObject(); } @Override - public void runInternal() { + public void run() { log.debug("call TriggerSchemaChangeAfterCheckpointOperation start {}", taskLocation); ((SeaTunnelServer) getService()) .getCoordinatorService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java index ba73ce64266..54daedd8c62 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/TriggerSchemaChangeBeforeCheckpointOperation.java @@ -20,11 +20,11 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -36,7 +36,7 @@ @Getter @AllArgsConstructor @NoArgsConstructor -public class TriggerSchemaChangeBeforeCheckpointOperation extends TracingOperation +public class TriggerSchemaChangeBeforeCheckpointOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -62,7 +62,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void runInternal() { + public void run() { log.debug("call TriggerSchemaChangeBeforeCheckpointOperation {}", taskLocation); ((SeaTunnelServer) getService()) .getCoordinatorService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java index 3755477f45a..8f5cb3f7761 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java @@ -150,10 +150,10 @@ private Set generateExecutionEdges(Set logicalEdges) Collections.sort( sortedLogicalEdges, (o1, o2) -> { - if (!o1.getInputVertexId().equals(o2.getInputVertexId())) { + if (o1.getInputVertexId() != o2.getInputVertexId()) { return o1.getInputVertexId() > o2.getInputVertexId() ? 1 : -1; } - if (!o1.getTargetVertexId().equals(o2.getTargetVertexId())) { + if (o1.getTargetVertexId() != o2.getTargetVertexId()) { return o1.getTargetVertexId() > o2.getTargetVertexId() ? 1 : -1; } return 0; diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java index 5a771c66e2f..d2b184e850f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/execution/Task.java @@ -62,6 +62,5 @@ default MetricsContext getMetricsContext() { return null; } - @Override default void provideDynamicMetrics(MetricDescriptor tagger, MetricsCollectionContext context) {} } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java index eb76a599800..2d2a0d04167 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/ReleaseSlotOperation.java @@ -23,17 +23,17 @@ import org.apache.seatunnel.engine.server.resourcemanager.worker.WorkerProfile; import org.apache.seatunnel.engine.server.serializable.ResourceDataSerializerHook; import org.apache.seatunnel.engine.server.service.slot.WrongTargetSlotException; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import lombok.extern.slf4j.Slf4j; import java.io.IOException; @Slf4j -public class ReleaseSlotOperation extends TracingOperation implements IdentifiedDataSerializable { +public class ReleaseSlotOperation extends Operation implements IdentifiedDataSerializable { private long jobID; private SlotProfile slotProfile; @@ -47,7 +47,7 @@ public ReleaseSlotOperation(long jobID, SlotProfile slotProfile) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); try { server.getSlotService().releaseSlot(jobID, slotProfile); @@ -68,14 +68,12 @@ public Object getResponse() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeObject(slotProfile); out.writeLong(jobID); } @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); slotProfile = in.readObject(); jobID = in.readLong(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java index e851172a648..d7fa05822d3 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/opeartion/RequestSlotOperation.java @@ -21,15 +21,15 @@ import org.apache.seatunnel.engine.server.resourcemanager.resource.ResourceProfile; import org.apache.seatunnel.engine.server.serializable.ResourceDataSerializerHook; import org.apache.seatunnel.engine.server.service.slot.SlotAndWorkerProfile; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class RequestSlotOperation extends TracingOperation implements IdentifiedDataSerializable { +public class RequestSlotOperation extends Operation implements IdentifiedDataSerializable { private ResourceProfile resourceProfile; private long jobID; @@ -43,14 +43,13 @@ public RequestSlotOperation(long jobID, ResourceProfile resourceProfile) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); result = server.getSlotService().requestSlot(jobID, resourceProfile); } @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeObject(resourceProfile); out.writeLong(jobID); } @@ -62,7 +61,6 @@ public Object getResponse() { @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); resourceProfile = in.readObject(); jobID = in.readLong(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java index f357a690da9..0d0f8c8054b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/resourcemanager/worker/WorkerProfile.java @@ -85,7 +85,6 @@ public void writeData(ObjectDataOutput out) throws IOException { out.writeObject(unassignedSlot); } out.writeBoolean(dynamicSlot); - out.writeObject(attributes); } @Override @@ -104,6 +103,5 @@ public void readData(ObjectDataInput in) throws IOException { unassignedSlots[i] = in.readObject(); } dynamicSlot = in.readBoolean(); - attributes = in.readObject(); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java index 1d26a0b5fe1..72487737033 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java @@ -50,6 +50,7 @@ public class RestConstant { @Deprecated public static final String RUNNING_JOB_URL = "/hazelcast/rest/maps/running-job"; public static final String JOB_INFO_URL = "/hazelcast/rest/maps/job-info"; public static final String FINISHED_JOBS_INFO = "/hazelcast/rest/maps/finished-jobs"; + public static final String SUBMIT_JOB_URL = "/hazelcast/rest/maps/submit-job"; public static final String ENCRYPT_CONFIG = "/hazelcast/rest/maps/encrypt-config"; // only for test use @@ -58,8 +59,5 @@ public class RestConstant { public static final String SYSTEM_MONITORING_INFORMATION = "/hazelcast/rest/maps/system-monitoring-information"; - public static final String SUBMIT_JOB_URL = "/hazelcast/rest/maps/submit-job"; - public static final String SUBMIT_JOBS_URL = "/hazelcast/rest/maps/submit-jobs"; public static final String STOP_JOB_URL = "/hazelcast/rest/maps/stop-job"; - public static final String STOP_JOBS_URL = "/hazelcast/rest/maps/stop-jobs"; } 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 6827a2cfb03..fec77708b68 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 @@ -474,9 +474,7 @@ private void processMetric( String tableName, JsonNode metricNode, Map[] tableMetricsMaps) { - if (metricNode == null) { - return; - } + if (metricNode == null) return; // Define index constant final int SOURCE_COUNT_IDX = 0, diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java index 9b8f0f8bca4..150aae54c1e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java @@ -44,23 +44,17 @@ import com.hazelcast.internal.ascii.rest.HttpCommandProcessor; import com.hazelcast.internal.ascii.rest.HttpPostCommand; import com.hazelcast.internal.json.Json; -import com.hazelcast.internal.json.JsonArray; import com.hazelcast.internal.json.JsonObject; import com.hazelcast.internal.serialization.Data; -import scala.Tuple2; import java.io.IOException; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_400; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_500; import static org.apache.seatunnel.engine.server.rest.RestConstant.ENCRYPT_CONFIG; -import static org.apache.seatunnel.engine.server.rest.RestConstant.STOP_JOBS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.STOP_JOB_URL; -import static org.apache.seatunnel.engine.server.rest.RestConstant.SUBMIT_JOBS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.SUBMIT_JOB_URL; public class RestHttpPostCommandProcessor extends HttpCommandProcessor { @@ -84,14 +78,10 @@ protected RestHttpPostCommandProcessor( public void handle(HttpPostCommand httpPostCommand) { String uri = httpPostCommand.getURI(); try { - if (uri.startsWith(SUBMIT_JOBS_URL)) { - handleSubmitJobs(httpPostCommand); - } else if (uri.startsWith(SUBMIT_JOB_URL)) { + if (uri.startsWith(SUBMIT_JOB_URL)) { handleSubmitJob(httpPostCommand, uri); - } else if (uri.startsWith(STOP_JOBS_URL)) { - handleStopJobs(httpPostCommand); } else if (uri.startsWith(STOP_JOB_URL)) { - handleStopJob(httpPostCommand); + handleStopJob(httpPostCommand, uri); } else if (uri.startsWith(ENCRYPT_CONFIG)) { handleEncrypt(httpPostCommand); } else { @@ -103,6 +93,7 @@ public void handle(HttpPostCommand httpPostCommand) { logger.warning("An error occurred while handling request " + httpPostCommand, e); prepareResponse(SC_500, httpPostCommand, exceptionResponse(e)); } + this.textCommandService.sendResponse(httpPostCommand); } @@ -112,41 +103,11 @@ private SeaTunnelServer getSeaTunnelServer() { return (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); } - private void handleSubmitJobs(HttpPostCommand httpPostCommand) throws IllegalArgumentException { - List, Config>> configTuples = - RestUtil.buildConfigList(requestHandle(httpPostCommand), false); - - JsonArray jsonArray = - configTuples.stream() - .map( - tuple -> { - String urlParams = mapToUrlParams(tuple._1); - Map requestParams = new HashMap<>(); - RestUtil.buildRequestParams(requestParams, urlParams); - return submitJobInternal(tuple._2, requestParams); - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - - prepareResponse(httpPostCommand, jsonArray); - } - - private String mapToUrlParams(Map params) { - return params.entrySet().stream() - .map(entry -> entry.getKey() + "=" + entry.getValue()) - .collect(Collectors.joining("&", "?", "")); - } - private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri) throws IllegalArgumentException { Map requestParams = new HashMap<>(); RestUtil.buildRequestParams(requestParams, uri); Config config = RestUtil.buildConfig(requestHandle(httpPostCommand), false); - - JsonObject jsonObject = submitJobInternal(config, requestParams); - this.prepareResponse(httpPostCommand, jsonObject); - } - - private JsonObject submitJobInternal(Config config, Map requestParams) { ReadonlyConfig envOptions = ReadonlyConfig.fromConfig(config.getConfig("env")); String jobName = envOptions.get(EnvCommonOptions.JOB_NAME); @@ -185,35 +146,15 @@ private JsonObject submitJobInternal(Config config, Map requestP submitJob(seaTunnelServer, jobImmutableInformation, jobConfig); } - return new JsonObject() - .add(RestConstant.JOB_ID, String.valueOf(jobId)) - .add(RestConstant.JOB_NAME, jobConfig.getName()); - } - - private void handleStopJobs(HttpPostCommand command) { - List jobList = JsonUtils.toList(requestHandle(command).toString(), Map.class); - JsonArray jsonResponse = new JsonArray(); - - jobList.forEach( - job -> { - handleStopJob(job); - jsonResponse.add( - new JsonObject() - .add(RestConstant.JOB_ID, (Long) job.get(RestConstant.JOB_ID))); - }); - - this.prepareResponse(command, jsonResponse); - } - - private void handleStopJob(HttpPostCommand httpPostCommand) { - Map map = JsonUtils.toMap(requestHandle(httpPostCommand)); - handleStopJob(map); this.prepareResponse( httpPostCommand, - new JsonObject().add(RestConstant.JOB_ID, map.get(RestConstant.JOB_ID).toString())); + new JsonObject() + .add(RestConstant.JOB_ID, String.valueOf(jobId)) + .add(RestConstant.JOB_NAME, jobConfig.getName())); } - private void handleStopJob(Map map) { + private void handleStopJob(HttpPostCommand httpPostCommand, String uri) { + Map map = JsonUtils.toMap(requestHandle(httpPostCommand)); boolean isStopWithSavePoint = false; if (map.get(RestConstant.JOB_ID) == null) { throw new IllegalArgumentException("jobId cannot be empty."); @@ -245,7 +186,10 @@ private void handleStopJob(Map map) { coordinatorService.cancelJob(jobId); } } - logger.info("Stop job with jobId: " + jobId); + + this.prepareResponse( + httpPostCommand, + new JsonObject().add(RestConstant.JOB_ID, map.get(RestConstant.JOB_ID).toString())); } private void handleEncrypt(HttpPostCommand httpPostCommand) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java index 3a5e27171c1..7c0ae38bfd8 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/service/slot/DefaultSlotService.java @@ -149,7 +149,6 @@ public synchronized SlotAndWorkerProfile requestSlot( return new SlotAndWorkerProfile(getWorkerProfile(), profile); } - @Override public SlotContext getSlotContext(SlotProfile slotProfile) { if (!contexts.containsKey(slotProfile.getSlotID())) { throw new WrongTargetSlotException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java index 8004068ce68..d1fc333ade2 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SourceSplitEnumeratorTask.java @@ -22,8 +22,6 @@ import org.apache.seatunnel.api.source.SourceEvent; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; -import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; import org.apache.seatunnel.engine.core.dag.actions.SourceAction; import org.apache.seatunnel.engine.core.job.ConnectorJarIdentifier; import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; @@ -123,7 +121,6 @@ public void close() throws IOException { super.close(); if (enumerator != null) { enumerator.close(); - enumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); } progress.done(); } @@ -312,7 +309,6 @@ private void stateProcess() throws Exception { if (startCalled && readerRegisterComplete) { currState = STARTING; enumerator.open(); - enumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); } else { Thread.sleep(100); } 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 3234560fe4b..cacaa75aaef 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 @@ -24,7 +24,6 @@ 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.event.WriterCloseEvent; import org.apache.seatunnel.api.sink.multitablesink.MultiTableSink; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.event.SchemaChangeEvent; @@ -70,7 +69,6 @@ public class SinkFlowLifeCycle sinkAction; private SinkWriter writer; - private SinkWriter.Context writerContext; private transient Optional> commitInfoSerializer; private transient Optional> writerStateSerializer; @@ -152,7 +150,6 @@ private Address getCommitterTaskAddress() throws ExecutionException, Interrupted public void close() throws IOException { super.close(); writer.close(); - writerContext.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); @@ -286,11 +283,19 @@ public void restoreState(List actionStateList) throws Except .deserialize(bytes))) .collect(Collectors.toList()); } - this.writerContext = new SinkWriterContext(indexID, metricsContext, eventListener); if (states.isEmpty()) { - this.writer = sinkAction.getSink().createWriter(writerContext); + this.writer = + sinkAction + .getSink() + .createWriter( + new SinkWriterContext(indexID, metricsContext, eventListener)); } else { - this.writer = sinkAction.getSink().restoreWriter(writerContext, states); + this.writer = + sinkAction + .getSink() + .restoreWriter( + new SinkWriterContext(indexID, metricsContext, eventListener), + states); } if (this.writer instanceof SupportResourceShare) { resourceManager = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java index 6c596da0c33..ca137b3e069 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java @@ -23,8 +23,6 @@ import org.apache.seatunnel.api.source.SourceEvent; import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.source.event.ReaderCloseEvent; -import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.engine.core.checkpoint.CheckpointType; import org.apache.seatunnel.engine.core.checkpoint.InternalCheckpointListener; @@ -85,7 +83,6 @@ public class SourceFlowLifeCycle extends ActionFl private final MetricsContext metricsContext; private final EventListener eventListener; - private SourceReader.Context context; private final AtomicReference schemaChangePhase = new AtomicReference<>(); @@ -114,20 +111,21 @@ public void setCollector(SeaTunnelSourceCollector collector) { @Override public void init() throws Exception { this.splitSerializer = sourceAction.getSource().getSplitSerializer(); - this.context = - new SourceReaderContext( - indexID, - sourceAction.getSource().getBoundedness(), - this, - metricsContext, - eventListener); - this.reader = sourceAction.getSource().createReader(context); + this.reader = + sourceAction + .getSource() + .createReader( + new SourceReaderContext( + indexID, + sourceAction.getSource().getBoundedness(), + this, + metricsContext, + eventListener)); this.enumeratorTaskAddress = getEnumeratorTaskAddress(); } @Override public void open() throws Exception { - context.getEventListener().onEvent(new ReaderOpenEvent()); reader.open(); register(); } @@ -142,7 +140,6 @@ private Address getEnumeratorTaskAddress() throws ExecutionException, Interrupte @Override public void close() throws IOException { - context.getEventListener().onEvent(new ReaderCloseEvent()); reader.close(); super.close(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java index f2b01bf0404..a6c5f7b087d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CancelTaskOperation.java @@ -24,6 +24,7 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; @@ -33,7 +34,7 @@ * is cancelled, the {@link org.apache.seatunnel.engine.server.TaskExecutionService} will notified * JobMaster */ -public class CancelTaskOperation extends TracingOperation implements IdentifiedDataSerializable { +public class CancelTaskOperation extends Operation implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; public CancelTaskOperation() {} @@ -53,7 +54,7 @@ public int getClassId() { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); server.getTaskExecutionService().cancelTaskGroup(taskGroupLocation); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java index e48e1fcac98..d4e158abdbe 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CheckTaskGroupIsExecutingOperation.java @@ -25,10 +25,11 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CheckTaskGroupIsExecutingOperation extends TracingOperation +public class CheckTaskGroupIsExecutingOperation extends Operation implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; @@ -41,7 +42,7 @@ public CheckTaskGroupIsExecutingOperation(TaskGroupLocation taskGroupLocation) { } @Override - public void runInternal() { + public void run() { SeaTunnelServer server = getService(); try { response = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java index 19099f563fa..42d83131382 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/CleanTaskGroupContextOperation.java @@ -24,10 +24,11 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CleanTaskGroupContextOperation extends TracingOperation +public class CleanTaskGroupContextOperation extends Operation implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; @@ -39,7 +40,7 @@ public CleanTaskGroupContextOperation(TaskGroupLocation taskGroupLocation) { } @Override - public void runInternal() { + public void run() { // remove TaskGroupContext for TaskExecutionService SeaTunnelServer service = getService(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java index e684e188c6e..296d5f0c9bc 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/DeployTaskOperation.java @@ -27,11 +27,12 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import lombok.NonNull; import java.io.IOException; -public class DeployTaskOperation extends TracingOperation implements IdentifiedDataSerializable { +public class DeployTaskOperation extends Operation implements IdentifiedDataSerializable { private Data taskImmutableInformation; private SlotProfile slotProfile; @@ -46,7 +47,7 @@ public DeployTaskOperation( } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); state = server.getSlotService() diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java index 20415e86635..ecb94e85c0f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/GetTaskGroupAddressOperation.java @@ -27,12 +27,12 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; import java.util.Objects; -public class GetTaskGroupAddressOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class GetTaskGroupAddressOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation taskLocation; @@ -45,7 +45,7 @@ public GetTaskGroupAddressOperation(TaskLocation taskLocation) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); response = RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java index 52fc15418b5..26f947bca8a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/NotifyTaskStatusOperation.java @@ -25,11 +25,11 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class NotifyTaskStatusOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class NotifyTaskStatusOperation extends Operation implements IdentifiedDataSerializable { private TaskGroupLocation taskGroupLocation; private TaskExecutionState taskExecutionState; @@ -68,7 +68,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService().updateTaskExecutionState(taskExecutionState); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java index 986fe3131e7..729d14dc072 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TaskOperation.java @@ -23,6 +23,7 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.NoArgsConstructor; @@ -32,7 +33,7 @@ @Getter @NoArgsConstructor @AllArgsConstructor -public abstract class TaskOperation extends TracingOperation implements IdentifiedDataSerializable { +public abstract class TaskOperation extends Operation implements IdentifiedDataSerializable { protected TaskLocation taskLocation; @@ -43,13 +44,11 @@ public String getServiceName() { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); taskLocation = in.readObject(TaskLocation.class); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java deleted file mode 100644 index 08d859a4b14..00000000000 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/TracingOperation.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.engine.server.task.operation; - -import org.apache.seatunnel.api.tracing.MDCContext; - -import com.hazelcast.nio.ObjectDataInput; -import com.hazelcast.nio.ObjectDataOutput; -import com.hazelcast.spi.impl.operationservice.Operation; - -import java.io.IOException; - -public abstract class TracingOperation extends Operation { - private MDCContext context; - - public TracingOperation() { - this(MDCContext.current()); - } - - public TracingOperation(MDCContext context) { - this.context = context; - } - - @Override - public final void run() throws Exception { - try { - context.put(); - - runInternal(); - } finally { - context.clear(); - } - } - - public abstract void runInternal() throws Exception; - - @Override - protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); - out.writeString(context.toString()); - } - - @Override - protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); - context = MDCContext.valueOf(in.readString()); - } -} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java index e105f8bde04..89d3b3805e5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/BarrierFlowOperation.java @@ -69,7 +69,7 @@ protected void readInternal(ObjectDataInput in) throws IOException { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java index e8eaaf1fd1a..f12361bdfb1 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/checkpoint/CloseRequestOperation.java @@ -24,15 +24,15 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSeaTunnelTask; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CloseRequestOperation extends TracingOperation implements IdentifiedDataSerializable { +public class CloseRequestOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation readerLocation; @@ -43,7 +43,7 @@ public CloseRequestOperation(TaskLocation readerLocation) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java index db3ae487a31..5ed6f81a7aa 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkPrepareCommitOperation.java @@ -69,7 +69,7 @@ public int getClassId() { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { TaskExecutionService taskExecutionService = ((SeaTunnelServer) getService()).getTaskExecutionService(); SinkAggregatedCommitterTask committerTask = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java index 32ba9145e74..33c781309e9 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/sink/SinkRegisterOperation.java @@ -24,7 +24,6 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SinkAggregatedCommitterTask; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.cluster.Address; import com.hazelcast.logging.ILogger; @@ -32,10 +31,11 @@ import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class SinkRegisterOperation extends TracingOperation implements IdentifiedDataSerializable { +public class SinkRegisterOperation extends Operation implements IdentifiedDataSerializable { private static final ILogger LOGGER = Logger.getLogger(SinkRegisterOperation.class); private TaskLocation writerTaskID; @@ -49,7 +49,7 @@ public SinkRegisterOperation(TaskLocation writerTaskID, TaskLocation committerTa } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); Address readerAddress = getCallerAddress(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java index b080d2f1bad..b21111e18fd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/AssignSplitOperation.java @@ -25,17 +25,17 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSeaTunnelTask; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; import java.util.ArrayList; import java.util.List; -public class AssignSplitOperation extends TracingOperation +public class AssignSplitOperation extends Operation implements IdentifiedDataSerializable { private List splits; @@ -49,7 +49,7 @@ public AssignSplitOperation(TaskLocation taskID, List splits) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { @@ -84,7 +84,6 @@ public void runInternal() throws Exception { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeInt(splits.size()); for (byte[] split : splits) { out.writeByteArray(split); @@ -94,7 +93,6 @@ protected void writeInternal(ObjectDataOutput out) throws IOException { @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); int splitCount = in.readInt(); splits = new ArrayList<>(splitCount); for (int i = 0; i < splitCount; i++) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java index b6c722203a5..abedf1a4993 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/CloseIdleReaderOperation.java @@ -20,16 +20,15 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class CloseIdleReaderOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class CloseIdleReaderOperation extends Operation implements IdentifiedDataSerializable { private long jobId; private TaskLocation taskLocation; @@ -41,7 +40,7 @@ public CloseIdleReaderOperation(long jobId, TaskLocation taskLocation) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService() .getJobMaster(jobId) @@ -51,14 +50,12 @@ public void runInternal() throws Exception { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeLong(jobId); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); jobId = in.readLong(); taskLocation = in.readObject(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java index d2391017d21..1309b92b451 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/LastCheckpointNotifyOperation.java @@ -20,16 +20,15 @@ import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class LastCheckpointNotifyOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class LastCheckpointNotifyOperation extends Operation implements IdentifiedDataSerializable { private long jobId; private TaskLocation taskLocation; @@ -42,7 +41,7 @@ public LastCheckpointNotifyOperation(long jobId, TaskLocation taskLocation) { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); server.getCoordinatorService() .getJobMaster(jobId) @@ -52,14 +51,12 @@ public void runInternal() throws Exception { @Override protected void writeInternal(ObjectDataOutput out) throws IOException { - super.writeInternal(out); out.writeLong(jobId); out.writeObject(taskLocation); } @Override protected void readInternal(ObjectDataInput in) throws IOException { - super.readInternal(in); jobId = in.readLong(); taskLocation = in.readObject(); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java index 1a4b7469db5..5750046d0c0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RequestSplitOperation.java @@ -24,15 +24,15 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSplitEnumeratorTask; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class RequestSplitOperation extends TracingOperation implements IdentifiedDataSerializable { +public class RequestSplitOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation enumeratorTaskID; @@ -46,7 +46,7 @@ public RequestSplitOperation(TaskLocation taskID, TaskLocation enumeratorTaskID) } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java index a6ee60f6f12..e3f6c908159 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/RestoredSplitOperation.java @@ -81,7 +81,7 @@ public int getClassId() { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); TaskExecutionService taskExecutionService = server.getTaskExecutionService(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java index 35667bf9b57..60b67c0c83b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceNoMoreElementOperation.java @@ -24,16 +24,15 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSplitEnumeratorTask; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; -public class SourceNoMoreElementOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class SourceNoMoreElementOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation currentTaskID; private TaskLocation enumeratorTaskID; @@ -46,7 +45,7 @@ public SourceNoMoreElementOperation(TaskLocation currentTaskID, TaskLocation enu } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java index 56a376dc5dc..8c3f4e0133d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceReaderEventOperation.java @@ -45,7 +45,7 @@ public int getClassId() { } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); RetryUtils.retryWithException( () -> { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java index 12685ae8b44..faf44b4c40d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/operation/source/SourceRegisterOperation.java @@ -25,12 +25,12 @@ import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook; import org.apache.seatunnel.engine.server.task.SourceSplitEnumeratorTask; -import org.apache.seatunnel.engine.server.task.operation.TracingOperation; import com.hazelcast.cluster.Address; import com.hazelcast.nio.ObjectDataInput; import com.hazelcast.nio.ObjectDataOutput; import com.hazelcast.nio.serialization.IdentifiedDataSerializable; +import com.hazelcast.spi.impl.operationservice.Operation; import java.io.IOException; @@ -38,8 +38,7 @@ * For {@link org.apache.seatunnel.api.source.SourceReader} to register with the {@link * org.apache.seatunnel.api.source.SourceSplitEnumerator} */ -public class SourceRegisterOperation extends TracingOperation - implements IdentifiedDataSerializable { +public class SourceRegisterOperation extends Operation implements IdentifiedDataSerializable { private TaskLocation readerTaskID; private TaskLocation enumeratorTaskID; @@ -52,7 +51,7 @@ public SourceRegisterOperation(TaskLocation readerTaskID, TaskLocation enumerato } @Override - public void runInternal() throws Exception { + public void run() throws Exception { SeaTunnelServer server = getService(); Address readerAddress = getCallerAddress(); RetryUtils.retryWithException( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java index b114073aadb..9aaa8cd5951 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/utils/RestUtil.java @@ -26,17 +26,11 @@ import org.apache.seatunnel.engine.server.rest.RestConstant; import com.hazelcast.internal.util.StringUtil; -import scala.Tuple2; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URLDecoder; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import static org.apache.seatunnel.engine.common.Constant.REST_SUBMIT_JOBS_PARAMS; public class RestUtil { private RestUtil() {} @@ -75,19 +69,4 @@ public static Config buildConfig(JsonNode jsonNode, boolean isEncrypt) { Map objectMap = JsonUtils.toMap(jsonNode); return ConfigBuilder.of(objectMap, isEncrypt, true); } - - public static List, Config>> buildConfigList( - JsonNode jsonNode, boolean isEncrypt) { - return StreamSupport.stream(jsonNode.spliterator(), false) - .filter(JsonNode::isObject) - .map( - node -> { - Map nodeMap = JsonUtils.toMap(node); - Map params = - (Map) nodeMap.remove(REST_SUBMIT_JOBS_PARAMS); - Config config = ConfigBuilder.of(nodeMap, isEncrypt, true); - return new Tuple2<>(params, config); - }) - .collect(Collectors.toList()); - } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java index 53cc68b9423..b3df40f6aaa 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/resourcemanager/FixSlotResourceTest.java @@ -35,32 +35,19 @@ public class FixSlotResourceTest extends AbstractSeaTunnelServerTest { - private final int totalSlots = 3; - @Override public SeaTunnelConfig loadSeaTunnelConfig() { SeaTunnelConfig seaTunnelConfig = super.loadSeaTunnelConfig(); SlotServiceConfig slotServiceConfig = seaTunnelConfig.getEngineConfig().getSlotServiceConfig(); slotServiceConfig.setDynamicSlot(false); - slotServiceConfig.setSlotNum(totalSlots); + slotServiceConfig.setSlotNum(3); seaTunnelConfig.getEngineConfig().setSlotServiceConfig(slotServiceConfig); return seaTunnelConfig; } @Test public void testEnoughResource() throws ExecutionException, InterruptedException { - // wait all slot ready - await().atMost(20000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assertions.assertEquals( - totalSlots, - server.getCoordinatorService() - .getResourceManager() - .getUnassignedSlots(null) - .size()); - }); long jobId = System.currentTimeMillis(); List resourceProfiles = new ArrayList<>(); resourceProfiles.add(new ResourceProfile()); @@ -96,7 +83,7 @@ public void testNotEnoughResource() throws ExecutionException, InterruptedExcept .untilAsserted( () -> { Assertions.assertEquals( - totalSlots, + 3, server.getCoordinatorService() .getResourceManager() .getUnassignedSlots(null) diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties b/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties index d40ba6afd72..153d4d97c68 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties +++ b/seatunnel-engine/seatunnel-engine-server/src/test/resources/log4j2-test.properties @@ -25,7 +25,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -35,7 +35,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%-30.30c{1.}] [%t] - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java index eec10aeabe1..213f458b9ee 100644 --- a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java +++ b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/main/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/HdfsStorageFactory.java @@ -58,6 +58,7 @@ * fs.oss.accessKeySecret = "your script key" * fs.oss.endpoint = "such as: oss-cn-hangzhou.aliyuncs.com" * oss.bucket= "oss://your bucket" + * fs.oss.credentials.provider = "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider" * */ @AutoService(CheckpointStorageFactory.class) diff --git a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java index 9e6ef2409b9..3d7299c266f 100644 --- a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java +++ b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-hdfs/src/test/java/org/apache/seatunnel/engine/checkpoint/storage/hdfs/OssFileCheckpointTest.java @@ -40,6 +40,9 @@ public static void setup() throws CheckpointStorageException { config.put("fs.oss.accessKeySecret", "your access key secret"); config.put("fs.oss.endpoint", "oss-cn-hangzhou.aliyuncs.com"); config.put("oss.bucket", "oss://seatunnel-test/"); + config.put( + "fs.oss.credentials.provider", + "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider"); STORAGE = new HdfsStorage(config); initStorageData(); } diff --git a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties index 0bf06d2b773..2dc1b8ca501 100644 --- a/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties +++ b/seatunnel-engine/seatunnel-engine-storage/checkpoint-storage-plugins/checkpoint-storage-local-file/src/test/resources/log4j2-test.properties @@ -25,7 +25,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -35,7 +35,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java b/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java index c642670016c..dd92b65e762 100644 --- a/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java +++ b/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/src/test/java/org/apache/seatunnel/engine/imap/storage/file/IMapFileOSSStorageTest.java @@ -68,6 +68,9 @@ public class IMapFileOSSStorageTest { CONF.set("fs.oss.accessKeyId", OSS_ACCESS_KEY_ID); CONF.set("fs.oss.accessKeySecret", OSS_ACCESS_KEY_SECRET); CONF.set("fs.oss.impl", "org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem"); + CONF.set( + "fs.oss.credentials.provider", + "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider"); STORAGE = new IMapFileStorage(); Map properties = new HashMap<>(); @@ -78,6 +81,9 @@ public class IMapFileOSSStorageTest { properties.put("fs.oss.accessKeyId", OSS_ACCESS_KEY_ID); properties.put("fs.oss.accessKeySecret", OSS_ACCESS_KEY_SECRET); properties.put("fs.oss.impl", "org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem"); + properties.put( + "fs.oss.credentials.provider", + "org.apache.hadoop.fs.aliyun.oss.AliyunCredentialsProvider"); properties.put(FileConstants.FileInitProperties.BUSINESS_KEY, BUSINESS); properties.put(FileConstants.FileInitProperties.NAMESPACE_KEY, NAMESPACE); properties.put(FileConstants.FileInitProperties.CLUSTER_NAME, CLUSTER_NAME); diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties index 77985ea23a9..db3dcb88e90 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/resources/log4j2.properties @@ -28,7 +28,7 @@ appender.consoleStdout.name = consoleStdoutAppender appender.consoleStdout.type = CONSOLE appender.consoleStdout.target = SYSTEM_OUT appender.consoleStdout.layout.type = PatternLayout -appender.consoleStdout.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStdout.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStdout.filter.acceptLtWarn.type = ThresholdFilter appender.consoleStdout.filter.acceptLtWarn.level = WARN appender.consoleStdout.filter.acceptLtWarn.onMatch = DENY @@ -38,7 +38,7 @@ appender.consoleStderr.name = consoleStderrAppender appender.consoleStderr.type = CONSOLE appender.consoleStderr.target = SYSTEM_ERR appender.consoleStderr.layout.type = PatternLayout -appender.consoleStderr.layout.pattern = [%X{ST-JID}] %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n +appender.consoleStderr.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %c - %m%n appender.consoleStderr.filter.acceptGteWarn.type = ThresholdFilter appender.consoleStderr.filter.acceptGteWarn.level = WARN appender.consoleStderr.filter.acceptGteWarn.onMatch = ACCEPT diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml index 99c75d324a8..ef801bdb9c0 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml @@ -112,6 +112,12 @@ ${flink.1.15.3.version} + + org.apache.flink + flink-runtime-web + ${flink.1.15.3.version} + + com.squareup.okhttp3 mockwebserver diff --git a/seatunnel-formats/pom.xml b/seatunnel-formats/pom.xml index c66c6c8eafa..a330e9b0e05 100644 --- a/seatunnel-formats/pom.xml +++ b/seatunnel-formats/pom.xml @@ -32,7 +32,6 @@ seatunnel-format-compatible-debezium-json seatunnel-format-compatible-connect-json seatunnel-format-avro - seatunnel-format-protobuf diff --git a/seatunnel-formats/seatunnel-format-protobuf/pom.xml b/seatunnel-formats/seatunnel-format-protobuf/pom.xml deleted file mode 100644 index 8f8f7030f64..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/pom.xml +++ /dev/null @@ -1,59 +0,0 @@ - - - - 4.0.0 - - org.apache.seatunnel - seatunnel-formats - ${revision} - - - seatunnel-format-protobuf - SeaTunnel : Formats : Protobuf - - - 8 - 8 - UTF-8 - 3.25.3 - 3.11.4 - - - - - org.apache.seatunnel - seatunnel-api - ${project.version} - provided - - - com.google.protobuf - protobuf-java - ${protobuf.version} - - - com.google.protobuf - protobuf-java-util - ${protobuf.version} - - - com.github.os72 - protoc-jar - ${protoc.jar.version} - - - diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java deleted file mode 100644 index 17a14bc84f7..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/CompileDescriptor.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.format.protobuf; - -import org.apache.seatunnel.common.utils.FileUtils; -import org.apache.seatunnel.format.protobuf.exception.ProtobufFormatErrorCode; -import org.apache.seatunnel.format.protobuf.exception.SeaTunnelProtobufFormatException; - -import com.github.os72.protocjar.Protoc; -import com.google.protobuf.DescriptorProtos; -import com.google.protobuf.Descriptors; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.util.List; - -public class CompileDescriptor { - - public static Descriptors.Descriptor compileDescriptorTempFile( - String protoContent, String messageName) - throws IOException, InterruptedException, Descriptors.DescriptorValidationException { - // Because Protobuf can only be dynamically parsed through the descriptor file, the file - // needs to be compiled and generated. The following method is used here to solve the - // problem: generate a temporary directory and compile .proto into a descriptor temporary - // file. The temporary file and directory are deleted after the JVM runs. - File tmpDir = createTempDirectory(); - File protoFile = createProtoFile(tmpDir, protoContent); - String targetDescPath = compileProtoToDescriptor(tmpDir, protoFile); - - try (FileInputStream fis = new FileInputStream(targetDescPath)) { - DescriptorProtos.FileDescriptorSet descriptorSet = - DescriptorProtos.FileDescriptorSet.parseFrom(fis); - Descriptors.FileDescriptor[] descriptorsArray = buildFileDescriptors(descriptorSet); - return descriptorsArray[0].findMessageTypeByName(messageName); - } finally { - tmpDir.delete(); - protoFile.delete(); - new File(targetDescPath).delete(); - } - } - - private static File createTempDirectory() throws IOException { - File tmpDir = File.createTempFile("tmp_protobuf_", "_proto"); - tmpDir.delete(); - tmpDir.mkdirs(); - tmpDir.deleteOnExit(); - return tmpDir; - } - - private static File createProtoFile(File tmpDir, String protoContent) throws IOException { - File protoFile = new File(tmpDir, ".proto"); - protoFile.deleteOnExit(); - FileUtils.writeStringToFile(protoFile.getPath(), protoContent); - return protoFile; - } - - private static String compileProtoToDescriptor(File tmpDir, File protoFile) - throws IOException, InterruptedException { - String targetDesc = tmpDir + "/.desc"; - new File(targetDesc).deleteOnExit(); - - int exitCode = - Protoc.runProtoc( - new String[] { - "--proto_path=" + protoFile.getParent(), - "--descriptor_set_out=" + targetDesc, - protoFile.getPath() - }); - - if (exitCode != 0) { - throw new SeaTunnelProtobufFormatException( - ProtobufFormatErrorCode.DESCRIPTOR_CONVERT_FAILED, - "Protoc compile error, exit code: " + exitCode); - } - return targetDesc; - } - - private static Descriptors.FileDescriptor[] buildFileDescriptors( - DescriptorProtos.FileDescriptorSet descriptorSet) - throws Descriptors.DescriptorValidationException { - List fileDescriptors = descriptorSet.getFileList(); - Descriptors.FileDescriptor[] descriptorsArray = - new Descriptors.FileDescriptor[fileDescriptors.size()]; - for (int i = 0; i < fileDescriptors.size(); i++) { - descriptorsArray[i] = - Descriptors.FileDescriptor.buildFrom( - fileDescriptors.get(i), new Descriptors.FileDescriptor[] {}); - } - return descriptorsArray; - } -} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java deleted file mode 100644 index 34e79bc7ff1..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufDeserializationSchema.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.format.protobuf; - -import org.apache.seatunnel.api.serialization.DeserializationSchema; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.TablePath; -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 com.google.protobuf.Descriptors; -import com.google.protobuf.DynamicMessage; - -import java.io.IOException; -import java.util.Optional; - -public class ProtobufDeserializationSchema implements DeserializationSchema { - private static final long serialVersionUID = -7907358485475741366L; - - private final SeaTunnelRowType rowType; - private final ProtobufToRowConverter converter; - private final CatalogTable catalogTable; - private final String protoContent; - private final String messageName; - - public ProtobufDeserializationSchema(CatalogTable catalogTable) { - this.catalogTable = catalogTable; - this.rowType = catalogTable.getSeaTunnelRowType(); - this.messageName = catalogTable.getOptions().get("protobuf_message_name"); - this.protoContent = catalogTable.getOptions().get("protobuf_schema"); - this.converter = new ProtobufToRowConverter(protoContent, messageName); - } - - @Override - public SeaTunnelRow deserialize(byte[] message) throws IOException { - Descriptors.Descriptor descriptor = this.converter.getDescriptor(); - DynamicMessage dynamicMessage = DynamicMessage.parseFrom(descriptor, message); - SeaTunnelRow seaTunnelRow = this.converter.converter(descriptor, dynamicMessage, rowType); - Optional tablePath = - Optional.ofNullable(catalogTable).map(CatalogTable::getTablePath); - if (tablePath.isPresent()) { - seaTunnelRow.setTableId(tablePath.toString()); - } - return seaTunnelRow; - } - - @Override - public SeaTunnelDataType getProducedType() { - return this.rowType; - } -} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.java deleted file mode 100644 index b733c542dc2..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufSerializationSchema.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.format.protobuf; - -import org.apache.seatunnel.api.serialization.SerializationSchema; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; - -import com.google.protobuf.Descriptors; - -import java.io.IOException; - -public class ProtobufSerializationSchema implements SerializationSchema { - - private static final long serialVersionUID = 4438784443025715370L; - - private final RowToProtobufConverter converter; - - public ProtobufSerializationSchema( - SeaTunnelRowType rowType, String protobufMessageName, String protobufSchema) { - try { - Descriptors.Descriptor descriptor = - CompileDescriptor.compileDescriptorTempFile( - protobufSchema, protobufMessageName); - this.converter = new RowToProtobufConverter(rowType, descriptor); - } catch (IOException | InterruptedException | Descriptors.DescriptorValidationException e) { - throw new RuntimeException(e); - } - } - - @Override - public byte[] serialize(SeaTunnelRow element) { - return converter.convertRowToGenericRecord(element); - } -} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java deleted file mode 100644 index 3f7d01b3ff6..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/ProtobufToRowConverter.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.format.protobuf; - -import org.apache.seatunnel.api.table.type.ArrayType; -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 com.google.protobuf.ByteString; -import com.google.protobuf.Descriptors; -import com.google.protobuf.DynamicMessage; - -import java.io.IOException; -import java.io.Serializable; -import java.lang.reflect.Array; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class ProtobufToRowConverter implements Serializable { - private static final long serialVersionUID = 8177020083886379563L; - - private Descriptors.Descriptor descriptor = null; - private String protoContent; - private String messageName; - - public ProtobufToRowConverter(String protoContent, String messageName) { - this.protoContent = protoContent; - this.messageName = messageName; - } - - public Descriptors.Descriptor getDescriptor() { - if (descriptor == null) { - try { - descriptor = createDescriptor(); - } catch (IOException - | Descriptors.DescriptorValidationException - | InterruptedException e) { - throw new RuntimeException(e); - } - } - return descriptor; - } - - private Descriptors.Descriptor createDescriptor() - throws IOException, InterruptedException, Descriptors.DescriptorValidationException { - - return CompileDescriptor.compileDescriptorTempFile(protoContent, messageName); - } - - public SeaTunnelRow converter( - Descriptors.Descriptor descriptor, - DynamicMessage dynamicMessage, - SeaTunnelRowType rowType) { - String[] fieldNames = rowType.getFieldNames(); - Object[] values = new Object[fieldNames.length]; - for (int i = 0; i < fieldNames.length; i++) { - Descriptors.FieldDescriptor fieldByName = descriptor.findFieldByName(fieldNames[i]); - if (fieldByName == null && descriptor.findNestedTypeByName(fieldNames[i]) == null) { - values[i] = null; - } else { - values[i] = - convertField( - descriptor, - dynamicMessage, - rowType.getFieldType(i), - fieldByName == null ? null : dynamicMessage.getField(fieldByName), - fieldNames[i]); - } - } - return new SeaTunnelRow(values); - } - - private Object convertField( - Descriptors.Descriptor descriptor, - DynamicMessage dynamicMessage, - SeaTunnelDataType dataType, - Object val, - String fieldName) { - switch (dataType.getSqlType()) { - case STRING: - return val.toString(); - case BOOLEAN: - case INT: - case BIGINT: - case FLOAT: - case DOUBLE: - case NULL: - case DATE: - case DECIMAL: - case TIMESTAMP: - return val; - case BYTES: - return ((ByteString) val).toByteArray(); - case SMALLINT: - return ((Integer) val).shortValue(); - case TINYINT: - Class typeClass = dataType.getTypeClass(); - if (typeClass == Byte.class) { - Integer integer = (Integer) val; - return integer.byteValue(); - } - return val; - case MAP: - MapType mapType = (MapType) dataType; - Map res = - ((List) val) - .stream() - .collect( - Collectors.toMap( - dm -> - convertField( - descriptor, - dm, - mapType.getKeyType(), - getFieldValue(dm, "key"), - null), - dm -> - convertField( - descriptor, - dm, - mapType.getValueType(), - getFieldValue(dm, "value"), - null))); - - return res; - case ROW: - Descriptors.Descriptor nestedTypeByName = - descriptor.findNestedTypeByName(fieldName); - DynamicMessage s = - (DynamicMessage) - dynamicMessage.getField( - descriptor.findFieldByName(fieldName.toLowerCase())); - return converter(nestedTypeByName, s, (SeaTunnelRowType) dataType); - case ARRAY: - SeaTunnelDataType basicType = ((ArrayType) dataType).getElementType(); - List list = (List) val; - return convertArray(list, basicType); - default: - String errorMsg = - String.format( - "SeaTunnel avro format is not supported for this data type [%s]", - dataType.getSqlType()); - throw new RuntimeException(errorMsg); - } - } - - private Object getFieldValue(DynamicMessage dm, String fieldName) { - return dm.getAllFields().entrySet().stream() - .filter(entry -> entry.getKey().getName().equals(fieldName)) - .map(Map.Entry::getValue) - .findFirst() - .orElse(null); - } - - protected Object convertArray(List val, SeaTunnelDataType dataType) { - if (val == null) { - return null; - } - int length = val.size(); - Object instance = Array.newInstance(dataType.getTypeClass(), length); - for (int i = 0; i < val.size(); i++) { - Array.set(instance, i, convertField(null, null, dataType, val.get(i), null)); - } - return instance; - } -} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java deleted file mode 100644 index a6a75aaee3f..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/RowToProtobufConverter.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.format.protobuf; - -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.format.protobuf.exception.ProtobufFormatErrorCode; -import org.apache.seatunnel.format.protobuf.exception.SeaTunnelProtobufFormatException; - -import com.google.protobuf.ByteString; -import com.google.protobuf.Descriptors; -import com.google.protobuf.DynamicMessage; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.Map; - -public class RowToProtobufConverter implements Serializable { - - private static final long serialVersionUID = -576124379280229724L; - private final Descriptors.Descriptor descriptor; - private final SeaTunnelRowType rowType; - - public RowToProtobufConverter(SeaTunnelRowType rowType, Descriptors.Descriptor descriptor) { - this.rowType = rowType; - this.descriptor = descriptor; - } - - public byte[] convertRowToGenericRecord(SeaTunnelRow element) { - DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor); - String[] fieldNames = rowType.getFieldNames(); - - for (int i = 0; i < fieldNames.length; i++) { - String fieldName = rowType.getFieldName(i); - Object value = element.getField(i); - Object resolvedValue = - resolveObject(fieldName, value, rowType.getFieldType(i), builder); - if (resolvedValue != null) { - if (resolvedValue instanceof byte[]) { - resolvedValue = ByteString.copyFrom((byte[]) resolvedValue); - } - builder.setField( - descriptor.findFieldByName(fieldName.toLowerCase()), resolvedValue); - } - } - - return builder.build().toByteArray(); - } - - private Object resolveObject( - String fieldName, - Object data, - SeaTunnelDataType seaTunnelDataType, - DynamicMessage.Builder builder) { - if (data == null) { - return null; - } - - switch (seaTunnelDataType.getSqlType()) { - case STRING: - case SMALLINT: - case INT: - case BIGINT: - case FLOAT: - case DOUBLE: - case BOOLEAN: - case DECIMAL: - case DATE: - case TIMESTAMP: - case BYTES: - return data; - case TINYINT: - if (data instanceof Byte) { - return Byte.toUnsignedInt((Byte) data); - } - return data; - case MAP: - return handleMapType(fieldName, data, seaTunnelDataType, builder); - case ARRAY: - return Arrays.asList((Object[]) data); - case ROW: - return handleRowType(fieldName, data, seaTunnelDataType); - default: - throw new SeaTunnelProtobufFormatException( - ProtobufFormatErrorCode.UNSUPPORTED_DATA_TYPE, - String.format( - "SeaTunnel protobuf format is not supported for this data type [%s]", - seaTunnelDataType.getSqlType())); - } - } - - private Object handleMapType( - String fieldName, - Object data, - SeaTunnelDataType seaTunnelDataType, - DynamicMessage.Builder builder) { - Descriptors.Descriptor mapEntryDescriptor = - descriptor.findFieldByName(fieldName).getMessageType(); - - if (data instanceof Map) { - Map mapData = (Map) data; - mapData.forEach( - (key, value) -> { - DynamicMessage mapEntry = - DynamicMessage.newBuilder(mapEntryDescriptor) - .setField(mapEntryDescriptor.findFieldByName("key"), key) - .setField( - mapEntryDescriptor.findFieldByName("value"), value) - .build(); - builder.addRepeatedField(descriptor.findFieldByName(fieldName), mapEntry); - }); - } - - return null; - } - - private Object handleRowType( - String fieldName, Object data, SeaTunnelDataType seaTunnelDataType) { - SeaTunnelRow seaTunnelRow = (SeaTunnelRow) data; - SeaTunnelDataType[] fieldTypes = ((SeaTunnelRowType) seaTunnelDataType).getFieldTypes(); - String[] fieldNames = ((SeaTunnelRowType) seaTunnelDataType).getFieldNames(); - Descriptors.Descriptor nestedTypeDescriptor = descriptor.findNestedTypeByName(fieldName); - DynamicMessage.Builder nestedBuilder = DynamicMessage.newBuilder(nestedTypeDescriptor); - - for (int i = 0; i < fieldNames.length; i++) { - Object resolvedValue = - resolveObject( - fieldNames[i], seaTunnelRow.getField(i), fieldTypes[i], nestedBuilder); - nestedBuilder.setField( - nestedTypeDescriptor.findFieldByName(fieldNames[i]), resolvedValue); - } - - return nestedBuilder.build(); - } -} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java deleted file mode 100644 index e43125a41a1..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/ProtobufFormatErrorCode.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.format.protobuf.exception; - -import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; - -public enum ProtobufFormatErrorCode implements SeaTunnelErrorCode { - DESCRIPTOR_CONVERT_FAILED("PROTOBUF-01", "Protobuf descriptor conversion failed."), - UNSUPPORTED_DATA_TYPE("PROTOBUF-02", "Unsupported data type."); - - private final String code; - private final String description; - - ProtobufFormatErrorCode(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-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.java b/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.java deleted file mode 100644 index 30f6dfd0312..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/main/java/org/apache/seatunnel/format/protobuf/exception/SeaTunnelProtobufFormatException.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.format.protobuf.exception; - -import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; -import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; - -public class SeaTunnelProtobufFormatException extends SeaTunnelRuntimeException { - - public SeaTunnelProtobufFormatException( - SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { - super(seaTunnelErrorCode, errorMessage); - } -} diff --git a/seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java b/seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java deleted file mode 100644 index e3d6dd35a2d..00000000000 --- a/seatunnel-formats/seatunnel-format-protobuf/src/test/java/org/apache/seatunnel/format/protobuf/ProtobufConverterTest.java +++ /dev/null @@ -1,155 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.seatunnel.format.protobuf; - -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.MapType; -import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import com.google.protobuf.Descriptors; -import com.google.protobuf.DynamicMessage; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -class ProtobufConverterTest { - - private SeaTunnelRow buildSeaTunnelRow() { - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(10); - - Map attributesMap = new HashMap<>(); - attributesMap.put("k1", 0.1F); - attributesMap.put("k2", 2.3F); - - String[] phoneNumbers = {"1", "2"}; - byte[] byteVal = {1, 2, 3}; - - SeaTunnelRow address = new SeaTunnelRow(3); - address.setField(0, "city_value"); - address.setField(1, "state_value"); - address.setField(2, "street_value"); - - seaTunnelRow.setField(0, 123); - seaTunnelRow.setField(1, 123123123123L); - seaTunnelRow.setField(2, 0.123f); - seaTunnelRow.setField(3, 0.123d); - seaTunnelRow.setField(4, false); - seaTunnelRow.setField(5, "test data"); - seaTunnelRow.setField(6, byteVal); - seaTunnelRow.setField(7, address); - seaTunnelRow.setField(8, attributesMap); - seaTunnelRow.setField(9, phoneNumbers); - - return seaTunnelRow; - } - - private SeaTunnelRowType buildSeaTunnelRowType() { - SeaTunnelRowType addressType = - new SeaTunnelRowType( - new String[] {"city", "state", "street"}, - new SeaTunnelDataType[] { - BasicType.STRING_TYPE, BasicType.STRING_TYPE, BasicType.STRING_TYPE - }); - - return new SeaTunnelRowType( - new String[] { - "c_int32", - "c_int64", - "c_float", - "c_double", - "c_bool", - "c_string", - "c_bytes", - "Address", - "attributes", - "phone_numbers" - }, - new SeaTunnelDataType[] { - BasicType.INT_TYPE, - BasicType.LONG_TYPE, - BasicType.FLOAT_TYPE, - BasicType.DOUBLE_TYPE, - BasicType.BOOLEAN_TYPE, - BasicType.STRING_TYPE, - PrimitiveByteArrayType.INSTANCE, - addressType, - new MapType<>(BasicType.STRING_TYPE, BasicType.FLOAT_TYPE), - ArrayType.STRING_ARRAY_TYPE - }); - } - - @Test - public void testConverter() - throws Descriptors.DescriptorValidationException, IOException, InterruptedException { - SeaTunnelRowType rowType = buildSeaTunnelRowType(); - SeaTunnelRow originalRow = buildSeaTunnelRow(); - - String protoContent = - "syntax = \"proto3\";\n" - + "\n" - + "package org.apache.seatunnel.format.protobuf;\n" - + "\n" - + "option java_outer_classname = \"ProtobufE2E\";\n" - + "\n" - + "message Person {\n" - + " int32 c_int32 = 1;\n" - + " int64 c_int64 = 2;\n" - + " float c_float = 3;\n" - + " double c_double = 4;\n" - + " bool c_bool = 5;\n" - + " string c_string = 6;\n" - + " bytes c_bytes = 7;\n" - + "\n" - + " message Address {\n" - + " string street = 1;\n" - + " string city = 2;\n" - + " string state = 3;\n" - + " string zip = 4;\n" - + " }\n" - + "\n" - + " Address address = 8;\n" - + "\n" - + " map attributes = 9;\n" - + "\n" - + " repeated string phone_numbers = 10;\n" - + "}"; - - String messageName = "Person"; - Descriptors.Descriptor descriptor = - CompileDescriptor.compileDescriptorTempFile(protoContent, messageName); - - RowToProtobufConverter rowToProtobufConverter = - new RowToProtobufConverter(rowType, descriptor); - byte[] protobufMessage = rowToProtobufConverter.convertRowToGenericRecord(originalRow); - - ProtobufToRowConverter protobufToRowConverter = - new ProtobufToRowConverter(protoContent, messageName); - DynamicMessage dynamicMessage = DynamicMessage.parseFrom(descriptor, protobufMessage); - SeaTunnelRow convertedRow = - protobufToRowConverter.converter(descriptor, dynamicMessage, rowType); - - Assertions.assertEquals(originalRow, convertedRow); - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java similarity index 57% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java index 92db061ccca..d19960044f1 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransform.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.nlpmodel.llm; +package org.apache.seatunnel.transform.llm; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; @@ -26,11 +26,8 @@ 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.nlpmodel.ModelProvider; -import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; -import org.apache.seatunnel.transform.nlpmodel.llm.remote.Model; -import org.apache.seatunnel.transform.nlpmodel.llm.remote.custom.CustomModel; -import org.apache.seatunnel.transform.nlpmodel.llm.remote.openai.OpenAIModel; +import org.apache.seatunnel.transform.llm.model.Model; +import org.apache.seatunnel.transform.llm.model.openai.OpenAIModel; import lombok.NonNull; import lombok.SneakyThrows; @@ -64,47 +61,18 @@ public String getPluginName() { @Override public void open() { - ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); - switch (provider) { - case CUSTOM: - // load custom_config from the configuration - ReadonlyConfig customConfig = - config.getOptional(ModelTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) - .map(ReadonlyConfig::fromMap) - .orElseThrow( - () -> - new IllegalArgumentException( - "Custom config can't be null")); - model = - new CustomModel( - inputCatalogTable.getSeaTunnelRowType(), - outputDataType.getSqlType(), - config.get(LLMTransformConfig.PROMPT), - config.get(LLMTransformConfig.MODEL), - provider.usedLLMPath(config.get(LLMTransformConfig.API_PATH)), - customConfig.get( - LLMTransformConfig.CustomRequestConfig - .CUSTOM_REQUEST_HEADERS), - customConfig.get( - LLMTransformConfig.CustomRequestConfig.CUSTOM_REQUEST_BODY), - customConfig.get( - LLMTransformConfig.CustomRequestConfig - .CUSTOM_RESPONSE_PARSE)); - break; - case OPENAI: - case DOUBAO: - model = - new OpenAIModel( - inputCatalogTable.getSeaTunnelRowType(), - outputDataType.getSqlType(), - config.get(LLMTransformConfig.PROMPT), - config.get(LLMTransformConfig.MODEL), - config.get(LLMTransformConfig.API_KEY), - provider.usedLLMPath(config.get(LLMTransformConfig.API_PATH))); - break; - case QIANFAN: - default: - throw new IllegalArgumentException("Unsupported model provider: " + provider); + 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); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java similarity index 55% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java index 8800f061db7..ca3da7e6706 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformConfig.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformConfig.java @@ -15,13 +15,27 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.nlpmodel.llm; +package org.apache.seatunnel.transform.llm; import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; -import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; +import org.apache.seatunnel.api.table.type.SqlType; -public class LLMTransformConfig extends ModelTransformConfig { +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") @@ -29,6 +43,19 @@ public class LLMTransformConfig extends ModelTransformConfig { .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() @@ -40,5 +67,5 @@ public class LLMTransformConfig extends ModelTransformConfig { Options.key("openai.api_path") .stringType() .defaultValue("https://api.openai.com/v1/chat/completions") - .withDescription("The API path of OpenAI LLM"); + .withDescription("The API path of OpenAI"); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java similarity index 67% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformFactory.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java index eda57e1275f..6fe5d53fe5b 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransformFactory.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/LLMTransformFactory.java @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.nlpmodel.llm; - -import org.apache.seatunnel.shade.com.google.common.collect.Lists; +package org.apache.seatunnel.transform.llm; import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.table.catalog.CatalogTable; @@ -25,8 +23,6 @@ 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 org.apache.seatunnel.transform.nlpmodel.ModelProvider; -import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; import com.google.auto.service.AutoService; @@ -43,25 +39,15 @@ public OptionRule optionRule() { .required( LLMTransformConfig.MODEL_PROVIDER, LLMTransformConfig.MODEL, - LLMTransformConfig.PROMPT) + LLMTransformConfig.PROMPT, + LLMTransformConfig.API_KEY) .optional( - LLMTransformConfig.API_PATH, LLMTransformConfig.OUTPUT_DATA_TYPE, - LLMTransformConfig.PROCESS_BATCH_SIZE) - .conditional( - LLMTransformConfig.MODEL_PROVIDER, - Lists.newArrayList(ModelProvider.OPENAI, ModelProvider.DOUBAO), - LLMTransformConfig.API_KEY) - .conditional( - LLMTransformConfig.MODEL_PROVIDER, - ModelProvider.QIANFAN, - LLMTransformConfig.API_KEY, - LLMTransformConfig.SECRET_KEY, - ModelTransformConfig.OAUTH_PATH) + LLMTransformConfig.INFERENCE_BATCH_SIZE) .conditional( LLMTransformConfig.MODEL_PROVIDER, - ModelProvider.CUSTOM, - LLMTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) + ModelProvider.OPENAI, + LLMTransformConfig.OPENAI_API_PATH) .build(); } diff --git a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java similarity index 83% rename from seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java index 99f341d78cc..a55d706c099 100644 --- a/seatunnel-connectors-v2/connector-typesense/src/main/java/org/apache/seatunnel/connectors/seatunnel/typesense/state/TypesenseSinkState.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/ModelProvider.java @@ -15,8 +15,8 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.typesense.state; +package org.apache.seatunnel.transform.llm; -import java.io.Serializable; - -public class TypesenseSinkState implements Serializable {} +public enum ModelProvider { + OPENAI +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/AbstractModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java similarity index 98% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/AbstractModel.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java index e658e514597..51d674c0ad8 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/AbstractModel.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/AbstractModel.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.nlpmodel.llm.remote; +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; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/Model.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java similarity index 94% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/Model.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java index d055ac8b9e9..77a8da63281 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/Model.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/Model.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.nlpmodel.llm.remote; +package org.apache.seatunnel.transform.llm.model; import org.apache.seatunnel.api.table.type.SeaTunnelRow; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/openai/OpenAIModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java similarity index 96% rename from seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/openai/OpenAIModel.java rename to seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java index 1424eed9e4c..9477b873202 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/openai/OpenAIModel.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/llm/model/openai/OpenAIModel.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.transform.nlpmodel.llm.remote.openai; +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; @@ -24,7 +24,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.api.table.type.SqlType; -import org.apache.seatunnel.transform.nlpmodel.llm.remote.AbstractModel; +import org.apache.seatunnel.transform.llm.model.AbstractModel; import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.CloseableHttpResponse; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.java deleted file mode 100644 index 287dd903524..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/CustomConfigPlaceholder.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.seatunnel.transform.nlpmodel; - -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -public class CustomConfigPlaceholder { - - // Placeholder ${model} - public static final String REPLACE_PLACEHOLDER_MODEL = "model"; - // Placeholder ${input} - public static final String REPLACE_PLACEHOLDER_INPUT = "input"; - // Placeholder ${prompt} - public static final String REPLACE_PLACEHOLDER_PROMPT = "prompt"; - - public static String replacePlaceholders( - String input, String placeholderName, String value, String defaultValue) { - String placeholderRegex = "\\$\\{" + Pattern.quote(placeholderName) + "(:[^}]*)?\\}"; - Pattern pattern = Pattern.compile(placeholderRegex); - Matcher matcher = pattern.matcher(input); - - StringBuffer result = new StringBuffer(); - while (matcher.find()) { - String replacement = - value != null && !value.isEmpty() - ? value - : (matcher.group(1) != null - ? matcher.group(1).substring(1).trim() - : defaultValue); - if (replacement == null) { - continue; - } - matcher.appendReplacement(result, Matcher.quoteReplacement(replacement)); - } - matcher.appendTail(result); - return result.toString(); - } - - public static Boolean findPlaceholder(String input, String placeholderName) { - String placeholderRegex = "\\$\\{" + Pattern.quote(placeholderName) + "(:[^}]*)?\\}"; - Pattern pattern = Pattern.compile(placeholderRegex); - Matcher matcher = pattern.matcher(input); - return matcher.find(); - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java deleted file mode 100644 index c14877816fc..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelProvider.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel; - -import org.apache.commons.lang3.StringUtils; - -public enum ModelProvider { - OPENAI("https://api.openai.com/v1/chat/completions", "https://api.openai.com/v1/embeddings"), - DOUBAO( - "https://ark.cn-beijing.volces.com/api/v3/chat/completions", - "https://ark.cn-beijing.volces.com/api/v3/embeddings"), - QIANFAN("", "https://aip.baidubce.com/rpc/2.0/ai_custom/v1/wenxinworkshop/embeddings"), - CUSTOM("", ""), - LOCAL("", ""); - - private final String LLMProviderPath; - private final String EmbeddingProviderPath; - - ModelProvider(String llmProviderPath, String embeddingProviderPath) { - LLMProviderPath = llmProviderPath; - EmbeddingProviderPath = embeddingProviderPath; - } - - public String usedLLMPath(String path) { - if (StringUtils.isBlank(path)) { - return LLMProviderPath; - } - return path; - } - - public String usedEmbeddingPath(String path) { - if (StringUtils.isBlank(path)) { - return EmbeddingProviderPath; - } - return path; - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java deleted file mode 100644 index b1234597502..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/ModelTransformConfig.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; -import org.apache.seatunnel.api.table.type.SqlType; - -import java.io.Serializable; -import java.util.Map; - -public class ModelTransformConfig implements Serializable { - - public static final Option MODEL_PROVIDER = - Options.key("model_provider") - .enumType(ModelProvider.class) - .noDefaultValue() - .withDescription("The model provider of LLM/Embedding"); - - 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 MODEL = - Options.key("model") - .stringType() - .noDefaultValue() - .withDescription( - "The model of LLM/Embedding, eg: if the model provider is OpenAI LLM, 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/Embedding"); - - public static final Option SECRET_KEY = - Options.key("secret_key") - .stringType() - .noDefaultValue() - .withDescription("The Secret key of LLM/Embedding"); - - public static final Option API_PATH = - Options.key("api_path") - .stringType() - .noDefaultValue() - .withFallbackKeys("openai.api_path") - .withDescription("The API of LLM/Embedding"); - - public static final Option OAUTH_PATH = - Options.key("oauth_path") - .stringType() - .noDefaultValue() - .withDescription("The Oauth path of LLM/Embedding"); - - public static final Option PROCESS_BATCH_SIZE = - Options.key("process_batch_size") - .intType() - .defaultValue(100) - .withFallbackKeys("inference_batch_size") - .withDescription("The row batch size of each process"); - - public static class CustomRequestConfig { - - // Custom response parsing - public static final Option> CUSTOM_CONFIG = - Options.key("custom_config") - .type(new TypeReference>() {}) - .noDefaultValue() - .withDescription("The custom config of the custom model."); - - public static final Option CUSTOM_RESPONSE_PARSE = - Options.key("custom_response_parse") - .stringType() - .noDefaultValue() - .withDescription( - "The response parse of the custom model. You can use Jsonpath to parse the return object you want to parse. eg: $.choices[*].message.content"); - - public static final Option> CUSTOM_REQUEST_HEADERS = - Options.key("custom_request_headers") - .mapType() - .noDefaultValue() - .withDescription("The custom request headers of the custom model."); - - public static final Option> CUSTOM_REQUEST_BODY = - Options.key("custom_request_body") - .type(new TypeReference>() {}) - .noDefaultValue() - .withDescription( - "The custom request body of the custom model." - + "1. ${model} placeholder for selecting model name." - + "2. ${input} placeholder for Determine input type. eg: [\"${input}\"]" - + "3. ${prompt} placeholder for LLM model " - + "4. ..."); - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java deleted file mode 100644 index 9e77043f0a7..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java +++ /dev/null @@ -1,212 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding; - -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.type.SeaTunnelRowType; -import org.apache.seatunnel.api.table.type.VectorType; -import org.apache.seatunnel.transform.common.MultipleFieldOutputTransform; -import org.apache.seatunnel.transform.common.SeaTunnelRowAccessor; -import org.apache.seatunnel.transform.exception.TransformCommonError; -import org.apache.seatunnel.transform.nlpmodel.ModelProvider; -import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.Model; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.custom.CustomModel; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.doubao.DoubaoModel; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.openai.OpenAIModel; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.qianfan.QianfanModel; -import org.apache.seatunnel.transform.nlpmodel.llm.LLMTransformConfig; - -import lombok.NonNull; -import lombok.SneakyThrows; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class EmbeddingTransform extends MultipleFieldOutputTransform { - - private final ReadonlyConfig config; - private List fieldNames; - private List fieldOriginalIndexes; - private Model model; - private Integer dimension; - - public EmbeddingTransform( - @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); - this.config = config; - initOutputFields( - inputCatalogTable.getTableSchema().toPhysicalRowDataType(), - config.get(EmbeddingTransformConfig.VECTORIZATION_FIELDS)); - } - - private void tryOpen() { - if (model == null) { - open(); - } - } - - @Override - public void open() { - // Initialize model - ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); - try { - switch (provider) { - case CUSTOM: - // load custom_config from the configuration - ReadonlyConfig customConfig = - config.getOptional( - ModelTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) - .map(ReadonlyConfig::fromMap) - .orElseThrow( - () -> - new IllegalArgumentException( - "Custom config can't be null")); - model = - new CustomModel( - config.get(ModelTransformConfig.MODEL), - provider.usedEmbeddingPath( - config.get(ModelTransformConfig.API_PATH)), - customConfig.get( - LLMTransformConfig.CustomRequestConfig - .CUSTOM_REQUEST_HEADERS), - customConfig.get( - ModelTransformConfig.CustomRequestConfig - .CUSTOM_REQUEST_BODY), - customConfig.get( - LLMTransformConfig.CustomRequestConfig - .CUSTOM_RESPONSE_PARSE), - config.get( - EmbeddingTransformConfig - .SINGLE_VECTORIZED_INPUT_NUMBER)); - break; - case OPENAI: - model = - new OpenAIModel( - config.get(ModelTransformConfig.API_KEY), - config.get(ModelTransformConfig.MODEL), - provider.usedEmbeddingPath( - config.get(ModelTransformConfig.API_PATH)), - config.get( - EmbeddingTransformConfig - .SINGLE_VECTORIZED_INPUT_NUMBER)); - break; - case DOUBAO: - model = - new DoubaoModel( - config.get(ModelTransformConfig.API_KEY), - config.get(ModelTransformConfig.MODEL), - provider.usedEmbeddingPath( - config.get(ModelTransformConfig.API_PATH)), - config.get( - EmbeddingTransformConfig - .SINGLE_VECTORIZED_INPUT_NUMBER)); - break; - case QIANFAN: - model = - new QianfanModel( - config.get(ModelTransformConfig.API_KEY), - config.get(ModelTransformConfig.SECRET_KEY), - config.get(ModelTransformConfig.MODEL), - provider.usedEmbeddingPath( - config.get(ModelTransformConfig.API_PATH)), - config.get(ModelTransformConfig.OAUTH_PATH), - config.get( - EmbeddingTransformConfig - .SINGLE_VECTORIZED_INPUT_NUMBER)); - break; - case LOCAL: - default: - throw new IllegalArgumentException("Unsupported model provider: " + provider); - } - // Initialize dimension - dimension = model.dimension(); - } catch (IOException e) { - throw new RuntimeException("Failed to initialize model", e); - } - } - - private void initOutputFields(SeaTunnelRowType inputRowType, Map fields) { - List fieldNames = new ArrayList<>(); - List fieldOriginalIndexes = new ArrayList<>(); - for (Map.Entry field : fields.entrySet()) { - String srcField = field.getValue(); - int srcFieldIndex; - try { - srcFieldIndex = inputRowType.indexOf(srcField); - } catch (IllegalArgumentException e) { - throw TransformCommonError.cannotFindInputFieldError(getPluginName(), srcField); - } - fieldNames.add(field.getKey()); - fieldOriginalIndexes.add(srcFieldIndex); - } - this.fieldNames = fieldNames; - this.fieldOriginalIndexes = fieldOriginalIndexes; - } - - @Override - protected Object[] getOutputFieldValues(SeaTunnelRowAccessor inputRow) { - tryOpen(); - try { - Object[] fieldArray = new Object[fieldOriginalIndexes.size()]; - for (int i = 0; i < fieldOriginalIndexes.size(); i++) { - fieldArray[i] = inputRow.getField(fieldOriginalIndexes.get(i)); - } - List vectorization = model.vectorization(fieldArray); - return vectorization.toArray(); - } catch (Exception e) { - throw new RuntimeException("Failed to data vectorization", e); - } - } - - @Override - protected Column[] getOutputColumns() { - Column[] columns = new Column[fieldNames.size()]; - for (int i = 0; i < fieldNames.size(); i++) { - columns[i] = - PhysicalColumn.of( - fieldNames.get(i), - VectorType.VECTOR_FLOAT_TYPE, - null, - dimension, - true, - "", - ""); - } - return columns; - } - - @Override - public String getPluginName() { - return "Embedding"; - } - - @SneakyThrows - @Override - public void close() { - if (model != null) { - model.close(); - } - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java deleted file mode 100644 index 0309363abd8..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformConfig.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; -import org.apache.seatunnel.transform.nlpmodel.ModelTransformConfig; - -import java.util.Map; - -public class EmbeddingTransformConfig extends ModelTransformConfig { - - public static final Option SINGLE_VECTORIZED_INPUT_NUMBER = - Options.key("single_vectorized_input_number") - .intType() - .defaultValue(1) - .withDescription( - "The number of single vectorized inputs, default is 1 , which means 1 inputs will be vectorized in one request , eg: qianfan only allows a maximum of 16 simultaneous messages, depending on your own settings, etc"); - - public static final Option> VECTORIZATION_FIELDS = - Options.key("vectorization_fields") - .mapType() - .noDefaultValue() - .withDescription( - "Specify the field vectorization relationship between input and output"); -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java deleted file mode 100644 index db3464e9291..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransformFactory.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding; - -import org.apache.seatunnel.shade.com.google.common.collect.Lists; - -import org.apache.seatunnel.api.configuration.util.OptionRule; -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 org.apache.seatunnel.transform.nlpmodel.ModelProvider; -import org.apache.seatunnel.transform.nlpmodel.llm.LLMTransformConfig; - -import com.google.auto.service.AutoService; - -@AutoService(Factory.class) -public class EmbeddingTransformFactory implements TableTransformFactory { - @Override - public String factoryIdentifier() { - return "Embedding"; - } - - @Override - public OptionRule optionRule() { - return OptionRule.builder() - .required( - EmbeddingTransformConfig.MODEL_PROVIDER, - EmbeddingTransformConfig.MODEL, - EmbeddingTransformConfig.VECTORIZATION_FIELDS) - .optional( - EmbeddingTransformConfig.API_PATH, - EmbeddingTransformConfig.SINGLE_VECTORIZED_INPUT_NUMBER, - EmbeddingTransformConfig.PROCESS_BATCH_SIZE) - .conditional( - EmbeddingTransformConfig.MODEL_PROVIDER, - Lists.newArrayList(ModelProvider.OPENAI, ModelProvider.DOUBAO), - EmbeddingTransformConfig.API_KEY) - .conditional( - EmbeddingTransformConfig.MODEL_PROVIDER, - ModelProvider.QIANFAN, - EmbeddingTransformConfig.API_KEY, - EmbeddingTransformConfig.SECRET_KEY, - EmbeddingTransformConfig.OAUTH_PATH) - .conditional( - LLMTransformConfig.MODEL_PROVIDER, - ModelProvider.CUSTOM, - LLMTransformConfig.CustomRequestConfig.CUSTOM_CONFIG) - .build(); - } - - @Override - public TableTransform createTransform(TableTransformFactoryContext context) { - return () -> - new EmbeddingTransform(context.getOptions(), context.getCatalogTables().get(0)); - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java deleted file mode 100644 index 2e8665559b0..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/AbstractModel.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding.remote; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; - -import org.apache.seatunnel.common.utils.BufferUtils; - -import org.apache.commons.lang3.ArrayUtils; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; - -public abstract class AbstractModel implements Model { - - protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - protected static final String DIMENSION_EXAMPLE = "dimension example"; - private final Integer singleVectorizedInputNumber; - - protected AbstractModel(Integer singleVectorizedInputNumber) { - this.singleVectorizedInputNumber = singleVectorizedInputNumber; - } - - @Override - public List vectorization(Object[] fields) throws IOException { - List result = new ArrayList<>(); - - List> vectors = batchProcess(fields, singleVectorizedInputNumber); - for (List vector : vectors) { - result.add(BufferUtils.toByteBuffer(vector.toArray(new Float[0]))); - } - return result; - } - - protected abstract List> vector(Object[] fields) throws IOException; - - public List> batchProcess(Object[] array, int batchSize) throws IOException { - List> merged = new ArrayList<>(); - if (array == null || array.length == 0) { - return merged; - } - for (int i = 0; i < array.length; i += batchSize) { - Object[] batch = ArrayUtils.subarray(array, i, i + batchSize); - List> vector = vector(batch); - merged.addAll(vector); - } - if (array.length != merged.size()) { - throw new RuntimeException( - "The number of vectors is not equal to the number of inputs, Please verify the configuration of the input field and the result returned."); - } - return merged; - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java deleted file mode 100644 index d02dec7718e..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/Model.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding.remote; - -import java.io.Closeable; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; - -public interface Model extends Closeable { - - List vectorization(Object[] fields) throws IOException; - - Integer dimension() throws IOException; -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java deleted file mode 100644 index 532d89bbd69..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/custom/CustomModel.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding.remote.custom; - -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.transform.nlpmodel.CustomConfigPlaceholder; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.AbstractModel; - -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 com.jayway.jsonpath.JsonPath; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -public class CustomModel extends AbstractModel { - - private final CloseableHttpClient client; - private final String model; - private final String apiPath; - private final Map header; - private final Map body; - private final String parse; - - public CustomModel( - String model, - String apiPath, - Map header, - Map body, - String parse, - Integer vectorizedNumber) { - super(vectorizedNumber); - this.apiPath = apiPath; - this.model = model; - this.header = header; - this.body = body; - this.parse = parse; - this.client = HttpClients.createDefault(); - } - - @Override - protected List> vector(Object[] fields) throws IOException { - return vectorGeneration(fields); - } - - @Override - public Integer dimension() throws IOException { - return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).size(); - } - - private List> vectorGeneration(Object[] fields) throws IOException { - HttpPost post = new HttpPost(apiPath); - // Construct a request with custom parameters - for (Map.Entry entry : header.entrySet()) { - post.setHeader(entry.getKey(), entry.getValue()); - } - - post.setEntity( - new StringEntity( - OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); - - CloseableHttpResponse response = client.execute(post); - - String responseStr = EntityUtils.toString(response.getEntity()); - - if (response.getStatusLine().getStatusCode() != 200) { - throw new IOException("Failed to get vector from custom, response: " + responseStr); - } - - return OBJECT_MAPPER.convertValue( - parseResponse(responseStr), new TypeReference>>() {}); - } - - @VisibleForTesting - public Object parseResponse(String responseStr) { - return JsonPath.parse(responseStr).read(parse); - } - - @VisibleForTesting - public ObjectNode createJsonNodeFromData(Object[] fields) throws IOException { - JsonNode rootNode = OBJECT_MAPPER.readTree(OBJECT_MAPPER.writeValueAsString(body)); - Iterator> bodyFields = rootNode.fields(); - while (bodyFields.hasNext()) { - Map.Entry field = bodyFields.next(); - String fieldName = field.getKey(); - JsonNode fieldValue = field.getValue(); - if (fieldValue.isTextual()) { - String value = fieldValue.asText(); - if (CustomConfigPlaceholder.findPlaceholder( - value, CustomConfigPlaceholder.REPLACE_PLACEHOLDER_MODEL)) { - ((ObjectNode) rootNode) - .put( - fieldName, - CustomConfigPlaceholder.replacePlaceholders( - value, - CustomConfigPlaceholder.REPLACE_PLACEHOLDER_MODEL, - model, - null)); - } else if (CustomConfigPlaceholder.findPlaceholder( - value, CustomConfigPlaceholder.REPLACE_PLACEHOLDER_INPUT)) { - ((ObjectNode) rootNode) - .put( - fieldName, - CustomConfigPlaceholder.replacePlaceholders( - value, - CustomConfigPlaceholder.REPLACE_PLACEHOLDER_INPUT, - fields[0].toString(), - null)); - } - } else if (fieldValue.isArray()) { - ArrayNode arrayNode = OBJECT_MAPPER.valueToTree(Arrays.asList(fields)); - ((ObjectNode) rootNode).set(fieldName, arrayNode); - } - } - return ((ObjectNode) rootNode); - } - - @Override - public void close() throws IOException { - if (client != null) { - client.close(); - } - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java deleted file mode 100644 index 945767d8324..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/doubao/DoubaoModel.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding.remote.doubao; - -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.transform.nlpmodel.embadding.remote.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 java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class DoubaoModel extends AbstractModel { - - private final CloseableHttpClient client; - private final String apiKey; - private final String model; - private final String apiPath; - - public DoubaoModel(String apiKey, String model, String apiPath, Integer vectorizedNumber) { - super(vectorizedNumber); - this.apiKey = apiKey; - this.model = model; - this.apiPath = apiPath; - this.client = HttpClients.createDefault(); - } - - @Override - protected List> vector(Object[] fields) throws IOException { - return vectorGeneration(fields); - } - - @Override - public Integer dimension() throws IOException { - return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).size(); - } - - private List> vectorGeneration(Object[] fields) throws IOException { - HttpPost post = new HttpPost(apiPath); - post.setHeader("Authorization", "Bearer " + apiKey); - post.setHeader("Content-Type", "application/json"); - post.setConfig( - RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); - - post.setEntity( - new StringEntity( - OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); - - CloseableHttpResponse response = client.execute(post); - String responseStr = EntityUtils.toString(response.getEntity()); - - if (response.getStatusLine().getStatusCode() != 200) { - throw new IOException("Failed to get vector from doubao, response: " + responseStr); - } - - JsonNode data = OBJECT_MAPPER.readTree(responseStr).get("data"); - List> embeddings = new ArrayList<>(); - - if (data.isArray()) { - for (JsonNode node : data) { - JsonNode embeddingNode = node.get("embedding"); - List embedding = - OBJECT_MAPPER.readValue( - embeddingNode.traverse(), new TypeReference>() {}); - embeddings.add(embedding); - } - } - return embeddings; - } - - @VisibleForTesting - public ObjectNode createJsonNodeFromData(Object[] fields) { - ArrayNode arrayNode = OBJECT_MAPPER.valueToTree(Arrays.asList(fields)); - return OBJECT_MAPPER.createObjectNode().put("model", model).set("input", arrayNode); - } - - @Override - public void close() throws IOException { - if (client != null) { - client.close(); - } - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java deleted file mode 100644 index 62a81cc097b..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/openai/OpenAIModel.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding.remote.openai; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; - -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.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 java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -public class OpenAIModel extends AbstractModel { - - private final CloseableHttpClient client; - private final String apiKey; - private final String model; - private final String apiPath; - - public OpenAIModel(String apiKey, String model, String apiPath, Integer vectorizedNumber) { - super(vectorizedNumber); - this.apiKey = apiKey; - this.model = model; - this.apiPath = apiPath; - this.client = HttpClients.createDefault(); - } - - @Override - protected List> vector(Object[] fields) throws IOException { - if (fields.length > 1) { - throw new IllegalArgumentException("OpenAI model only supports single input"); - } - return vectorGeneration(fields); - } - - @Override - public Integer dimension() throws IOException { - return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).size(); - } - - private List> vectorGeneration(Object[] fields) throws IOException { - HttpPost post = new HttpPost(apiPath); - post.setHeader("Authorization", "Bearer " + apiKey); - post.setHeader("Content-Type", "application/json"); - post.setConfig( - RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); - - post.setEntity( - new StringEntity( - OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); - - CloseableHttpResponse response = client.execute(post); - String responseStr = EntityUtils.toString(response.getEntity()); - - if (response.getStatusLine().getStatusCode() != 200) { - throw new IOException("Failed to get vector from openai, response: " + responseStr); - } - - JsonNode data = OBJECT_MAPPER.readTree(responseStr).get("data"); - List> embeddings = new ArrayList<>(); - - if (data.isArray()) { - for (JsonNode node : data) { - JsonNode embeddingNode = node.get("embedding"); - List embedding = - OBJECT_MAPPER.readValue( - embeddingNode.traverse(), new TypeReference>() {}); - embeddings.add(embedding); - } - } - return embeddings; - } - - @VisibleForTesting - public ObjectNode createJsonNodeFromData(Object[] data) throws JsonProcessingException { - ObjectNode objectNode = OBJECT_MAPPER.createObjectNode(); - objectNode.put("model", model); - objectNode.put("input", data[0].toString()); - return objectNode; - } - - @Override - public void close() throws IOException { - if (client != null) { - client.close(); - } - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java deleted file mode 100644 index 081377e3def..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/remote/qianfan/QianfanModel.java +++ /dev/null @@ -1,173 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.embadding.remote.qianfan; - -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.transform.nlpmodel.embadding.remote.AbstractModel; - -import org.apache.http.client.config.RequestConfig; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpGet; -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 java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class QianfanModel extends AbstractModel { - - private final CloseableHttpClient client; - private final String apiKey; - private final String secretKey; - private final String model; - private final String apiPath; - private final String oauthPath; - private final String oauthSuffixPath = - "?grant_type=client_credentials&client_id=%s&client_secret=%s"; - private String accessToken; - - public QianfanModel( - String apiKey, - String secretKey, - String model, - String apiPath, - String oauthPath, - Integer vectorizedNumber) - throws IOException { - super(vectorizedNumber); - this.apiKey = apiKey; - this.secretKey = secretKey; - this.model = model; - this.apiPath = apiPath; - this.oauthPath = oauthPath; - this.client = HttpClients.createDefault(); - this.accessToken = getAccessToken(); - } - - public QianfanModel( - String apiKey, - String secretKey, - String model, - String apiPath, - Integer vectorizedNumber, - String oauthPath, - String accessToken) - throws IOException { - super(vectorizedNumber); - this.apiKey = apiKey; - this.secretKey = secretKey; - this.model = model; - this.apiPath = apiPath; - this.oauthPath = oauthPath; - this.client = HttpClients.createDefault(); - this.accessToken = accessToken; - } - - private String getAccessToken() throws IOException { - HttpGet get = new HttpGet(String.format(oauthPath + oauthSuffixPath, apiKey, secretKey)); - CloseableHttpResponse response = client.execute(get); - String responseStr = EntityUtils.toString(response.getEntity()); - if (response.getStatusLine().getStatusCode() != 200) { - throw new IOException("Failed to Oauth for qianfan, response: " + responseStr); - } - JsonNode result = OBJECT_MAPPER.readTree(responseStr); - return result.get("access_token").asText(); - } - - @Override - public List> vector(Object[] fields) throws IOException { - return vectorGeneration(fields); - } - - @Override - public Integer dimension() throws IOException { - return vectorGeneration(new Object[] {DIMENSION_EXAMPLE}).get(0).size(); - } - - private List> vectorGeneration(Object[] fields) throws IOException { - String formattedApiPath = - String.format( - (apiPath.endsWith("/") ? apiPath : apiPath + "/") + "%s?access_token=%s", - model, - accessToken); - HttpPost post = new HttpPost(formattedApiPath); - post.setHeader("Content-Type", "application/json"); - post.setConfig( - RequestConfig.custom().setConnectTimeout(20000).setSocketTimeout(20000).build()); - - post.setEntity( - new StringEntity( - OBJECT_MAPPER.writeValueAsString(createJsonNodeFromData(fields)), "UTF-8")); - - CloseableHttpResponse response = client.execute(post); - String responseStr = EntityUtils.toString(response.getEntity()); - - if (response.getStatusLine().getStatusCode() != 200) { - throw new IOException("Failed to get vector from qianfan, response: " + responseStr); - } - - JsonNode result = OBJECT_MAPPER.readTree(responseStr); - JsonNode errorCode = result.get("error_code"); - - if (errorCode != null) { - // Handle access token expiration - if (errorCode.asInt() == 110) { - this.accessToken = getAccessToken(); - } - throw new IOException( - "Failed to get vector from qianfan, response: " + result.get("error_msg")); - } - - List> embeddings = new ArrayList<>(); - JsonNode data = result.get("data"); - if (data.isArray()) { - for (JsonNode node : data) { - List embedding = - OBJECT_MAPPER.readValue( - node.get("embedding").traverse(), - new TypeReference>() {}); - embeddings.add(embedding); - } - } - return embeddings; - } - - @VisibleForTesting - public ObjectNode createJsonNodeFromData(Object[] data) { - ArrayNode arrayNode = OBJECT_MAPPER.valueToTree(Arrays.asList(data)); - return OBJECT_MAPPER.createObjectNode().set("input", arrayNode); - } - - @Override - public void close() throws IOException { - if (client != null) { - client.close(); - } - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java deleted file mode 100644 index af893e92ddc..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/remote/custom/CustomModel.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.nlpmodel.llm.remote.custom; - -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.shade.com.fasterxml.jackson.databind.node.TextNode; - -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.api.table.type.SqlType; -import org.apache.seatunnel.transform.nlpmodel.CustomConfigPlaceholder; -import org.apache.seatunnel.transform.nlpmodel.llm.remote.AbstractModel; - -import org.apache.groovy.util.Maps; -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 com.jayway.jsonpath.JsonPath; - -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -public class CustomModel extends AbstractModel { - - private final CloseableHttpClient client; - private final String model; - private final String apiPath; - private final Map header; - private final Map body; - private final String parse; - - public CustomModel( - SeaTunnelRowType rowType, - SqlType outputType, - String prompt, - String model, - String apiPath, - Map header, - Map body, - String parse) { - super(rowType, outputType, prompt); - this.apiPath = apiPath; - this.model = model; - this.header = header; - this.body = body; - this.parse = parse; - this.client = HttpClients.createDefault(); - } - - @Override - protected List chatWithModel(String promptWithLimit, String rowsJson) - throws IOException { - HttpPost post = new HttpPost(apiPath); - // Construct a request with custom parameters - for (Map.Entry entry : header.entrySet()) { - post.setHeader(entry.getKey(), entry.getValue()); - } - - post.setEntity( - new StringEntity( - OBJECT_MAPPER.writeValueAsString( - createJsonNodeFromData(promptWithLimit, rowsJson)), - "UTF-8")); - - CloseableHttpResponse response = client.execute(post); - - String responseStr = EntityUtils.toString(response.getEntity()); - - if (response.getStatusLine().getStatusCode() != 200) { - throw new IOException("Failed to get vector from custom, response: " + responseStr); - } - return OBJECT_MAPPER.convertValue( - parseResponse(responseStr), new TypeReference>() {}); - } - - @VisibleForTesting - public Object parseResponse(String responseStr) { - return JsonPath.parse(responseStr).read(parse); - } - - @VisibleForTesting - public ObjectNode createJsonNodeFromData(String prompt, String data) throws IOException { - JsonNode jsonNode = OBJECT_MAPPER.readTree(OBJECT_MAPPER.writeValueAsString(body)); - Map placeholderValues = - Maps.of( - CustomConfigPlaceholder.REPLACE_PLACEHOLDER_INPUT, data, - CustomConfigPlaceholder.REPLACE_PLACEHOLDER_PROMPT, prompt, - CustomConfigPlaceholder.REPLACE_PLACEHOLDER_MODEL, model); - - return (ObjectNode) replacePlaceholders(jsonNode, placeholderValues); - } - - private static JsonNode replacePlaceholders( - JsonNode node, Map placeholderValues) { - if (node.isObject()) { - ObjectNode objectNode = (ObjectNode) node; - Iterator> fields = objectNode.fields(); - while (fields.hasNext()) { - Map.Entry field = fields.next(); - objectNode.set( - field.getKey(), replacePlaceholders(field.getValue(), placeholderValues)); - } - } else if (node.isArray()) { - ArrayNode arrayNode = (ArrayNode) node; - for (int i = 0; i < arrayNode.size(); i++) { - arrayNode.set(i, replacePlaceholders(arrayNode.get(i), placeholderValues)); - } - } else if (node.isTextual()) { - String textValue = node.asText(); - for (Map.Entry entry : placeholderValues.entrySet()) { - if (CustomConfigPlaceholder.findPlaceholder(textValue, entry.getKey())) { - textValue = - CustomConfigPlaceholder.replacePlaceholders( - textValue, entry.getKey(), entry.getValue(), null); - } - } - return new TextNode(textValue); - } - return node; - } - - @Override - public void close() throws IOException { - if (client != null) { - client.close(); - } - } -} diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.java deleted file mode 100644 index 4d054329069..00000000000 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/EmbeddingTransformFactoryTest.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform; - -import org.apache.seatunnel.transform.nlpmodel.embadding.EmbeddingTransformFactory; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class EmbeddingTransformFactoryTest { - - @Test - public void testOptionRule() throws Exception { - EmbeddingTransformFactory embeddingTransformFactory = new EmbeddingTransformFactory(); - Assertions.assertNotNull(embeddingTransformFactory.optionRule()); - } -} 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 index 85cad8e7995..39b27694805 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.seatunnel.transform; -import org.apache.seatunnel.transform.nlpmodel.llm.LLMTransformFactory; +import org.apache.seatunnel.transform.llm.LLMTransformFactory; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java deleted file mode 100644 index 56e37f50212..00000000000 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/embedding/EmbeddingRequestJsonTest.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.transform.embedding; - -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.type.TypeReference; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ObjectNode; - -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.custom.CustomModel; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.doubao.DoubaoModel; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.openai.OpenAIModel; -import org.apache.seatunnel.transform.nlpmodel.embadding.remote.qianfan.QianfanModel; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import com.google.common.collect.Lists; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class EmbeddingRequestJsonTest { - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - @Test - void testOpenAIRequestJson() throws IOException { - OpenAIModel model = - new OpenAIModel( - "apikey", "modelName", "https://api.openai.com/v1/chat/completions", 1); - ObjectNode node = - model.createJsonNodeFromData( - new Object[] { - "Determine whether someone is Chinese or American by their name" - }); - Assertions.assertEquals( - "{\"model\":\"modelName\",\"input\":\"Determine whether someone is Chinese or American by their name\"}", - OBJECT_MAPPER.writeValueAsString(node)); - model.close(); - } - - @Test - void testDoubaoRequestJson() throws IOException { - DoubaoModel model = - new DoubaoModel( - "apikey", "modelName", "https://api.doubao.io/v1/chat/completions", 1); - ObjectNode node = - model.createJsonNodeFromData( - new Object[] { - "Determine whether someone is Chinese or American by their name" - }); - Assertions.assertEquals( - "{\"model\":\"modelName\",\"input\":[\"Determine whether someone is Chinese or American by their name\"]}", - OBJECT_MAPPER.writeValueAsString(node)); - model.close(); - } - - @Test - void testQianfanRequestJson() throws IOException { - QianfanModel model = - new QianfanModel( - "apikey", - "secretKey", - "modelName", - "https://api.qianfan.io/v1/chat/completions", - 1, - "xxxx", - "xxxxxxx"); - ObjectNode node = - model.createJsonNodeFromData( - new Object[] { - "Determine whether someone is Chinese or American by their name" - }); - Assertions.assertEquals( - "{\"input\":[\"Determine whether someone is Chinese or American by their name\"]}", - OBJECT_MAPPER.writeValueAsString(node)); - model.close(); - } - - @Test - void testCustomRequestJson() throws IOException { - Map header = new HashMap<>(); - header.put("Content-Type", "application/json"); - header.put("Authorization", "Bearer " + "apikey"); - Map body = new HashMap<>(); - body.put("model1", "${model}"); - body.put("input1", Lists.newArrayList("${input}", "${input}")); - - CustomModel model = - new CustomModel( - "modelName", - "https://api.custom.com/v1/chat/completions", - header, - body, - "$.data[*].embedding", - 1); - ObjectNode node = - model.createJsonNodeFromData( - new Object[] { - "Determine whether someone is Chinese or American by their name" - }); - Assertions.assertEquals( - "{\"model1\":\"modelName\",\"input1\":[\"Determine whether someone is Chinese or American by their name\"]}", - OBJECT_MAPPER.writeValueAsString(node)); - model.close(); - } - - @Test - void testCustomParseResponse() { - CustomModel customModel = - new CustomModel( - "modelName", - "https://api.custom.com/v1/chat/completions", - new HashMap<>(), - new HashMap<>(), - "$.data[*].embedding", - 1); - List> lists = - OBJECT_MAPPER.convertValue( - customModel.parseResponse( - "{\"created\":1725001256,\"id\":\"02172500125677376580aba8475a41c550bbf05104842f0405ef5\",\"data\":[{\"embedding\":[-1.625,0.07958984375,-1.5703125,-3.03125,-1.4609375,3.46875,-0.73046875,-2.578125,-0.66796875,1.71875,0.361328125,2,5.125,2.25,4.6875,1.4921875,-0.77734375,-0.466796875,0.0439453125,-2.46875,3.59375,4.96875,2.34375,-5.34375,0.11083984375,-5.875,3.0625,4.09375,3.4375,0.2265625,9,-1.9296875,2.25,0.765625,3.671875,-2.484375,-1.171875,-1.6171875,4.1875,2.390625,-6.90625,0.369140625,0.259765625,3.671875,-2.9375,-1.9140625,-0.71875,-1.6640625,0.29296875,0.396484375,-4.625,-1.9921875,5.15625,-1.3984375,3.015625,-3.203125,-1.453125,4,-8.75,-5.625,1.0546875,-3.28125,-1.2265625,0.287109375,2.09375,4.6875,0.1572265625,0.42578125,0.79296875,3.234375,-0.169921875,0.9296875,7.40625,-3.296875,5.53125,3.890625,0.62109375,1.1171875,-0.373046875,4.125,-2.78125,0.333984375,3.9375,4.59375,6,1.53125,-0.373046875,1.109375,-4.0625,1.96875,1.421875,0.57421875,-0.56640625,-2.390625,0.734375,1.1875,-2.859375,-6.09375,-5.96875,1.8359375,-3,0.80859375,-0.130859375,-5.3125,-2.859375,1.484375,-4.53125,-6.90625,-2.25,0.7734375,-1.2734375,1.1484375,3.421875,-3.484375,2.65625,1.3359375,1.1484375,-4.09375,-5.625,2.625,-0.283203125,-3.46875,2.3125,-0.220703125,4.21875,3.75,-0.37109375,0.9609375,7.25,-0.87890625,7.03125,2.34375,4.5,-1.609375,-6.46875,-6.125,-2.59375,2.234375,3.78125,1.3046875,-5.5,1.953125,-3.421875,-5.9375,3.25,-3.4375,-8.3125,-2.546875,3.640625,0.267578125,-0.220703125,0.294921875,-0.4140625,2.515625,-1.0546875,-5.21875,6.6875,3.640625,0.2314453125,-2.5,1,1.6640625,0.59765625,2.75,1.1328125,1.1328125,-4.96875,4.53125,-0.349609375,3.390625,-0.193359375,7.625,2.921875,-3.484375,4.1875,8.5,-1.9140625,6.3125,2.5625,3.0625,0.40234375,0.76953125,-4.78125,3.53125,-2.765625,0.1591796875,-0.1025390625,-3.875,2.203125,0.03076171875,1.765625,1.859375,2.15625,-1.2578125,-4.40625,-0.62890625,4.4375,-1.78125,2.671875,2.765625,-1.7890625,-8.3125,-0.02197265625,1.640625,-3.96875,-3.15625,2.796875,1.1875,2,1.15625,2.359375,1.3984375,4.21875,-2.953125,8.5,3.46875,3.578125,0.90625,-1.8828125,2.15625,3.921875,4.125,-0.9609375,-2.171875,2.328125,2.921875,1.9765625,1.0703125,4.03125,6.28125,-3.59375,-0.94921875,5.6875,-1.9140625,-5.1875,-4.25,-7.71875,1.7109375,-1.59375,3.765625,-5.3125,-3.9375,-3.796875,2.90625,2.859375,-2.203125,-1.78125,-3.796875,0.1708984375,-5.15625,0.298828125,1.828125,-7.1875,1.6953125,6.125,2.671875,0.1728515625,3.375,0.609375,-4.78125,0.40625,-3.875,-6.4375,0.6953125,1.171875,-2.140625,5.8125,-1.640625,5.90625,-0.1650390625,4.9375,-2.421875,1.609375,-3.171875,-4.71875,7.6875,-1.09375,-1.9296875,0.033447265625,-3.46875,-2.671875,-8.875,2.4375,-1.1015625,4.40625,-3.53125,1.546875,2.359375,-3.15625,10.625,7.46875,-3.0625,-0.044677734375,0.90234375,-5.28125,-3,-1.2890625,0.59375,-6.34375,-1.8203125,5.40625,5.78125,-1.578125,2.46875,-2.171875,-1.71875,-0.38671875,-2.21875,-0.150390625,4.65625,-3.46875,1.5625,4.4375,-2.609375,1.6875,-2.828125,-6.03125,5.15625,-2.296875,-1.65625,-2.3125,-4.75,-3.3125,-3.703125,-1.9296875,-6.59375,3.640625,-0.62890625,4.8125,0.11279296875,2.515625,0.9921875,-3.03125,-5.40625,7.5625,-1.765625,4.4375,4.25,-0.140625,3.671875,-2.984375,-2.734375,2.203125,-6.96875,-1.1640625,2.390625,1.3515625,-1.75,2.96875,-3.75,-0.109375,2.5,0.796875,5.21875,7.8125,-4,1.171875,0.435546875,1.2734375,-3.015625,1.90625,-1.21875,5.9375,-0.31640625,-4.03125,-3.59375,1.09375,4.65625,-0.81640625,-2.046875,0.80859375,-5.375,2,-2.265625,5.34375,-0.46875,-1.3359375,-2.953125,-4.875,-0.53515625,-3,1.8203125,-2.59375,-1.4765625,6.28125,2.09375,0.1318359375,2.40625,-0.09130859375,-2.421875,-1.78125,1.59375,0.48828125,-0.310546875,-0.2353515625,0.1748046875,0.4453125,2.078125,-1.046875,1.46875,0.6953125,-0.52734375,-0.19140625,-2.28125,-0.515625,0.47265625,-1.2421875,-8.3125,1.1875,2.015625,-4.46875,3.734375,1.453125,-2.8125,-2.78125,5.875,-0.38671875,1.171875,-6.5,1.8046875,-2.15625,4,3.375,-0.78125,0.87890625,-1.796875,-1.265625,2.59375,3.96875,1.7421875,2.296875,2.78125,-5.8125,-2.046875,-0.1201171875,-4.1875,3.96875,-3.484375,-4.125,1.21875,3.484375,0.3828125,3.8125,1.90625,-8.3125,-2.15625,2.578125,2.578125,-1.34375,-3.359375,4.71875,-1.640625,-3.484375,2.046875,3.0625,-1.03125,-2.96875,6.96875,3.703125,-0.29296875,-0.423828125,2.640625,-1.265625,3.9375,-0.314453125,-4.15625,-2.171875,0.2734375,6.375,-6.21875,-6.3125,4.6875,-0.053466796875,0.045166015625,2.765625,2.953125,1.078125,-0.453125,1.96875,-6.71875,-3.375,-4.1875,2.515625,-0.5390625,-1.9296875,-4.03125,1.1953125,8.1875,1.0078125,0.80859375,-1.15625,-1.53125,2.875,-3.921875,1.953125,4.09375,6.59375,-4.5625,-1.2109375,3.5,-8.1875,0.294921875,-3.453125,-0.9921875,-2.015625,1.5,0.6328125,-4.90625,-2.765625,1.0546875,4.25,-2.390625,-5.96875,7.15625,-5.4375,-3.953125,1.15625,-0.017822265625,2.90625,2.78125,-2.21875,3.390625,1.9453125,2.171875,1.8671875,-1.125,-3.65625,-1.359375,0.96484375,2.5625,-2.9375,1.2734375,4.15625,-6,-0.2021484375,-1.8515625,-0.56640625,-1.671875,1.546875,5.8125,-0.640625,1.140625,-5.71875,-0.40625,0.5390625,-1.6640625,0.3203125,-2.375,4.9375,-2.453125,-1.59375,0.1669921875,1.6796875,-0.81640625,1.765625,-3.125,-1.234375,0.84375,-0.96484375,0.232421875,-0.01300048828125,-6.03125,4.25,5.625,0.65625,-1.6015625,1.390625,5.65625,3.0625,0.287109375,-0.08203125,4.15625,-1.5703125,-0.609375,-6.34375,2.203125,-3.84375,-2.53125,-3.390625,3.15625,4.59375,-4.46875,5.0625,-3.09375,3.328125,-0.65625,1.8515625,-9.375,1.609375,-1.515625,-2.5625,-2.953125,0.734375,2.375,1.3515625,0.390625,1.8671875,0.07080078125,1.328125,3.6875,0.2421875,0.73828125,3.1875,1.65625,2.75,2.859375,-2.8125,-7.75,1.53125,-1.1015625,-1.6875,6.3125,3.03125,-2.46875,0.77734375,-0.34765625,-1.78125,-1.4453125,3.40625,3.140625,-3.96875,3.984375,-3.21875,5.375,-2.890625,2.90625,-2.375,-6.1875,-2.4375,0.34375,-4.46875,-2.421875,3.40625,-1.2578125,4.59375,4.125,-6,0.003936767578125,1.1484375,2.359375,4.09375,0.5703125,-1.328125,-6.03125,4.5,3.234375,-2.140625,5.03125,-2.640625,0.041748046875,-0.90234375,4.375,-2.125,-0.1630859375,2.421875,-2.078125,1.1328125,-3.53125,1.0234375,-0.2734375,-9.125,-6.03125,0.73828125,-0.87109375,6.59375,-0.65625,-2.109375,-3.359375,2.40625,-0.0157470703125,5.96875,2.390625,3.078125,5.65625,5.09375,-1.5859375,1.78125,-0.921875,-8.0625,7.0625,-5.71875,-2.375,2.359375,2.65625,-1.453125,-1.2265625,1.984375,-2.125,-5.46875,-5.25,-1.78125,-4.28125,3.375,-2.09375,1.984375,-0.75,-5.0625,1.46875,-1.8671875,-2.875,-1.859375,2.609375,-5.5,2.484375,5.65625,1.875,-0.94921875,3.890625,4.125,0.8984375,-2.796875,0.95703125,-7.9375,1.7890625,3.453125,-1.9296875,-0.69140625,-5.84375,2.171875,-3.4375,2.921875,0.890625,-2.203125,-2.375,-1.6328125,-2.65625,0.8515625,-7.28125,2.484375,1.6484375,-0.8359375,-0.859375,0.232421875,1.921875,0.73046875,-0.30078125,1.515625,4.9375,0.7109375,-0.43359375,-3.140625,-2.796875,-0.2431640625,2.265625,-2.53125,6.875,-0.54296875,-1.5625,3.96875,0.44921875,-3.640625,-4.25,4.375,-1.875,0.45703125,-1.2265625,5.65625,0.298828125,3.921875,-1.703125,-2.8125,-3.328125,1.7578125,3.3125,-1.6875,-3.234375,2.09375,2.375,5.40625,-3.234375,-7.09375,1.984375,4.125,-0.8046875,-2.71875,8.6875,-1.296875,-2.625,-3,-3.78125,1.359375,1.515625,2.875,0.11279296875,-1.5859375,1.078125,3.46875,-1.390625,0.6328125,0.24609375,-3.765625,3.515625,0.380859375,2.609375,-0.80078125,-2.484375,-2.15625,-1.3203125,0.02490234375,4.03125,8.25,-1.5234375,-1.1953125,1.2109375,0.3125,-1.7421875,5.625,-0.76953125,5.90625,1.15625,0.1640625,1.390625,0.82421875,-0.322265625,3.21875,-4.65625,-4.5,-1.765625,3.171875,-4.3125,-1.4375,-2.546875,-0.9140625,4.28125,0.609375,-3.171875,3.671875,0.48046875,-0.9140625,-4,-2.4375,-5.34375,-1.96875,0.828125,1.953125,-2.140625,-2.59375,-0.353515625,4.78125,-4.09375,-3.921875,0.03173828125,1.8359375,1.3984375,-0.65234375,-1.15625,0.1611328125,0.50390625,2.90625,-1.875,-3.40625,0.498046875,8.75,3.90625,-4.53125,0.67578125,-0.765625,1.8359375,-5.3125,-2.15625,-0.6796875,-1.8984375,-3.046875,-1.7734375,-1.390625,-2.71875,-2.015625,5.84375,-3.28125,0.55859375,0.8046875,3.984375,0.99609375,3.015625,0.458984375,5.3125,3.1875,-1.2421875,-5.84375,-1.3828125,-0.04052734375,-5.75,-1.8828125,3.234375,6,3.171875,1.5703125,-2.828125,0.033203125,-0.953125,0.640625,5.3125,-5.75,-3.78125,-1.984375,-7.9375,-6.84375,-3.859375,-2.65625,-3.15625,-6.84375,-0.9765625,-1.375,-7.1875,-1.1328125,-2.109375,-1.546875,-1,0.640625,4.625,-4.65625,2.3125,3.703125,2.6875,3.0625,-2.28125,3.34375,0.474609375,-1.46875,0.34765625,-2.03125,5.25,-1.4609375,5.875,3.984375,-0.87890625,-3.8125,4.46875,4.40625,5.90625,-4.875,-3.53125,-2.53125,-1.8125,-0.39453125,-1.2421875,2.203125,-3.828125,-3.59375,-1.0859375,-3.453125,0.1845703125,5.625,0.421875,5.3125,-1.3671875,0.30859375,1.5234375,2.953125,0.1064453125,2.59375,1.5546875,-4.46875,3.609375,-0.81640625,1.390625,0.8359375,-2.78125,2.125,-1.6875,0.365234375,2.234375,3.875,10.4375,1.15625,2.328125,-0.09326171875,-0.76171875,-2.609375,-2.96875,2.40625,1.6796875,1.4921875,-3.65625,0.74609375,-0.8828125,2.03125,-0.306640625,-16.875,-3.328125,-5.53125,-2.109375,4.625,-1.0546875,-1.984375,1.0625,3.6875,2.671875,7.09375,-1.484375,4.03125,-1.078125,-0.7265625,2.515625,-4.3125,1.578125,3.6875,1.890625,4.625,1.7734375,-1.8125,-2.828125,6.9375,5.0625,-4.5,0.193359375,5.09375,-1.3515625,-1.140625,4.40625,-2.96875,2.078125,-4.75,3.078125,7.09375,2.75,-2.953125,-4.125,-2.375,-2.0625,1.0234375,3.046875,-2.578125,1.578125,2.921875,-5.65625,2.28125,2.28125,-0.259765625,-3.484375,-0.37109375,2.71875,1.625,-0.158203125,-4.5,2.5625,0.98828125,3.84375,4.8125,-2.796875,-2.140625,2.34375,2.90625,2.1875,1.5546875,2.578125,2.8125,-1.8515625,-2.984375,0.310546875,-1.328125,-0.0234375,-1.9765625,0.83984375,3.65625,2.046875,-4.5625,2.171875,2.234375,-2.109375,-0.0439453125,-4.0625,-3.5,2.09375,-2.21875,-2.5,0.703125,-2.953125,-1.28125,3.234375,-4.6875,4.1875,-2.484375,8.75,-0.53125,-1.8203125,1.171875,-3.0625,4.78125,-2.484375,-3.453125,3.765625,-2.6875,1.5625,-3.828125,1.9296875,-1.765625,1.2421875,5.0625,-4.65625,-2.0625,0.53125,3.265625,-2.875,-2.296875,0.29296875,3.859375,0.123046875,-4.46875,4.09375,-2.796875,3.96875,-3.890625,1.875,-4.46875,-0.5078125,2.140625,0.3203125,4.84375,5.03125,-5.34375,-4.96875,-1.3203125,-5.03125,-4.875,-4.5625,5.03125,-2.625,-0.75,1.046875,2.109375,-0.130859375,1.890625,-1.8125,2.53125,6.53125,-2.09375,0.87890625,-0.41015625,-0.412109375,-4.09375,-2.421875,-4.46875,6.40625,0.43359375,1.2578125,3.734375,-1.7109375,2.953125,1.8125,-1.1171875,-1.7109375,2.15625,1.859375,-2.015625,-2.25,1.7734375,-3.578125,4.15625,-3.328125,-3.28125,-4.71875,2.953125,1.40625,-0.287109375,1.5703125,3.53125,1.578125,3.171875,-4.34375,-3.125,5.78125,3.453125,-2.046875,4.3125,-1.2265625,-1.84375,0.640625,2.625,0.12890625,-3.25,-4.6875,5.28125,2.65625,2.015625,-4.4375,-5.75,-3.625,4.0625,4.59375,-0.78125,-2.484375,-2.03125,-3.75,1.6875,-4.15625,2.734375,-1.65625,-3.453125,-0.89453125,3.71875,2.453125,-4.15625,2.09375,0.82421875,-2.03125,0.052978515625,4.4375,1.734375,-3.71875,1.375,-0.349609375,-1.75,-7,3.59375,-2.625,-0.427734375,-4.40625,-3.84375,-3.265625,-3.796875,0.74609375,2.65625,1.6171875,3.609375,-0.7890625,3.890625,2.796875,-0.8671875,-0.43359375,2.15625,-1.7578125,-3.640625,2.375,-4.65625,-3.5,1.3984375,-7.1875,-1.5,5.0625,-2.625,4.0625,-1.171875,3.796875,-1.453125,-2.9375,-4,-1.3046875,0.91796875,6.59375,0.64453125,-0.91796875,0.64453125,2.703125,2.1875,-2.296875,-1.015625,-1.9921875,5,-0.298828125,2.953125,-5.125,-5.03125,5.375,-1.1328125,-4.46875,-0.5546875,-3.09375,1.5703125,5.34375,0.765625,-4.46875,-2.421875,-6.75,2.8125,-1.6171875,3.109375,-5.59375,0.87109375,-4.875,2.53125,4.46875,-7.21875,-3.1875,2.4375,3,5.1875,1.84375,-2.625,-6.21875,2.21875,0.306640625,-1.90625,-4.09375,-2.34375,-1.3046875,-3.875,4.4375,-2.328125,2.546875,-3.875,-2.40625,0.80078125,0.34765625,1,0.828125,1.4453125,-0.859375,3.03125,1.109375,5.15625,1.1953125,-3.8125,2.734375,4.21875,0.345703125,-1.2109375,2.0625,-0.79296875,-2.8125,2.109375,2.96875,-2.90625,5.15625,3.359375,4.3125,-5.53125,-2.875,1.515625,3.515625,-2.75,1.7109375,-4.9375,0.7265625,3.71875,-0.4765625,1.34375,0.049560546875,2.796875,-1.421875,-1.7890625,1.5,2.3125,4.21875,1.6875,3.015625,3.3125,-1.1640625,3.546875,-0.375,-1.2265625,-1.59375,3.609375,-3.015625,-2.546875,-4.625,1.046875,-1.796875,4.75,2.515625,1.1484375,0.8984375,-1.4140625,-2.328125,0.037841796875,-5.78125,-1.5859375,0.11669921875,3.015625,-0.83984375,0.84375,-0.82421875,0.96484375,4.0625,0.0400390625,4.25,-2.28125,1.3515625,1,1.5625,-2.8125,3.15625,-2.609375,-0.142578125,1.578125,-2.875,3.75,-4.3125,-1.359375,-2.578125,-0.69140625,2.84375,3.75,-4.75,-5.5625,0.84765625,0.380859375,5.125,3.0625,-3.140625,-0.93359375,0.73046875,0.0303955078125,4.3125,0.85546875,2.703125,-4.28125,5.625,5.90625,0.4296875,0.76953125,-0.9140625,-1.71875,-4.5,3.828125,-0.4609375,2.21875,-1.9453125,2.5,4.15625,1.8984375,3.984375,-5.75,2.953125,0.2734375,3.890625,-0.76171875,-3.90625,0.337890625,1.96875,0.69140625,-0.70703125,3.578125,0.046142578125,0.765625,-2.734375,4.28125,4.3125,2.578125,-4.40625,1.921875,-2.90625,1.7734375,-3.90625,1.1484375,-5.625,1.65625,2.703125,5.34375,-1.9296875,-6.1875,4.5,1.5625,-0.9140625,-3.953125,4.65625,4.5625,2.484375,-5.15625,-2.375,1.625,-1.328125,-0.26171875,-5.25,3.328125,2.0625,-3.609375,-3.71875,1.6171875,1.046875,-3.1875,-3.71875,-3.34375,1.9609375,2.5625,3.609375,1.59375,-2.484375,4.125,-0.80078125,1.9140625,4.78125,-1.09375,0.140625,3.171875,-3.578125,2.640625,-0.6640625,-2.65625,-1.4375,0.47265625,-2.46875,2.6875,-2.515625,-1.0234375,-2.09375,-0.138671875,-0.5078125,1.5,4.15625,-3.09375,0.158203125,4.4375,-1.96875,-3,-1.9609375,2.09375,-1.7734375,-1.09375,-1.8984375,3.3125,1.9765625,0.8671875,0.2890625,0.66796875,-1.9765625,-3.640625,-4.90625,2.0625,-4.0625,3.59375,-0.84765625,-6.21875,1.515625,3.890625,3.640625,-0.2734375,-2.046875,0.875,3.78125,0.07470703125,-1.078125,-1.4921875,3.671875,-2.796875,-3.6875,2.75,2.78125,-5.40625,1.7890625,-4.28125,-2.265625,-0.98046875,4.46875,0.173828125,-2.25,-2.875,-3.84375,-1.7421875,-1.6171875,-3.21875,1.9140625,1.7421875,2.671875,1.09375,1.4375,-3.5,2.59375,19.125,0.0101318359375,-8.4375,1.3515625,-3.625,4.4375,4.65625,1.8125,0.423828125,-1.5,0.62890625,4.21875,0.609375,0.5390625,-2.390625,0.984375,-0.79296875,2.078125,-3.703125,-3.109375,-2.265625,-1.0234375,-0.328125,1.9765625,2.5,2.375,0.8046875,-2.265625,1.2734375,-3.390625,-4.375,-4.71875,3.765625,-2.921875,3.125,-3.171875,4.65625,-0.7890625,-3.3125,-2.984375,-3.296875,-2.796875,2.375,-0.12255859375,-3.21875,5.21875,0.1982421875,0.2138671875,-1.1796875,-0.130859375,-4.34375,-1.4453125,-2.5,6.3125,1.0625,-6.15625,-0.5703125,-3.203125,-3.546875,-1.375,2.9375,-0.53515625,1.7578125,2.71875,-1.9453125,-2.640625,-3.046875,0.49609375,1.0078125,-3,-4.84375,0.2119140625,1.2265625,1.3515625,1.609375,-4.84375,2.46875,2.140625,2.171875,1.75,0.67578125,-0.60546875,-2.46875,-2.234375,-0.9453125,1.2421875,-3.15625,0.006744384765625,3.359375,-1.765625,8.375,-8.3125,5.8125,5.15625,-2.0625,-0.470703125,1.5,-0.30859375,-2.421875,-0.2294921875,0.95703125,1.8828125,4.84375,-0.68359375,4.625,1.359375,0.373046875,0.83203125,2.640625,4.34375,0.7578125,3.109375,-0.412109375,-2,2.15625,-0.08349609375,-3.140625,-3,-3.703125,-2.5625,3.6875,1.7890625,-3.296875,0.89453125,-7.5,-5.40625,-2.25,-7.625,4.34375,-1.34375,-0.14453125,3.515625,-2.46875,-1.2109375,-2.46875,-3.921875,1.265625,3.65625,1.4375,-1.46875,-5.03125,2.59375,3.890625,-2.765625,-2.4375,0.353515625,-4.21875,4.4375,-0.376953125,3.9375,-2.09375,3.96875,3.234375,-2.203125,-6.875,5.15625,-3.6875,-4.34375,-6.625,-2.90625,-4.9375,-3.34375,0.412109375,-0.9453125,-0.5703125,-1.3046875,3.21875,-0.65234375,-1.6796875,3.171875,3.453125,-4.4375,-1.2578125,0.828125,1.1796875,-4.375,0.1787109375,4,0.53515625,1.328125,-0.546875,0.271484375,-0.5546875,-3.859375,-0.2216796875,0.86328125,-4.53125,-1.3828125,-0.60546875,-5.46875,-1.3515625,-1.2890625,-3.734375,2.9375,2.40625,-3.984375,0.875,-2.953125,-0.9765625,-1.6328125,-1.25,3.96875,1.6953125,0.0072021484375,5.875,-0.921875,-3.46875,-3.140625,-0.26953125,0.2265625,-2.09375,7.0625,-1.09375,0.30078125,-6.03125,5.34375,2.359375,1.6640625,-0.99609375,4.625,4.25,-2.484375,-4,0.89453125,3.0625,4.1875,-4.28125,3.953125,0.6328125,-0.74609375,-1.53125,2.015625,-1.1796875,1.03125,-1.6484375,-5.4375,0.3671875,1.8125,-0.326171875,1.546875,4.03125,-3.34375,0.484375,2.5,-1.4140625,3.34375,4.25,-1.7890625,1.09375,2.171875,5.34375,-1.5625,0.98828125,-5.09375,-3.625,-2.640625,-2.46875,3.109375,-2.515625,0.09033203125,0.21484375,-3.921875,3.125,-4.1875,1.2109375,1.3671875,1.1875,-5.4375,4.59375,3.890625,-2.8125,3.328125,-5.125,-1.9765625,-1.4296875,2.34375,-2.71875,-5.875,3.125,3.453125,-1.515625,3.546875,2.265625,-0.52734375,1.9375,-2.859375,2.703125,-3.359375,4.75,1.2734375,3.09375,3.65625,-0.255859375,-0.1044921875,-5.75,-0.3359375,-0.77734375,-2.234375,6.1875,-3.84375,0.19921875,4.25,6.4375,-10.5,-1.5078125,0.7265625,0.2890625,3.921875,5.0625,0.09814453125,0.68359375,3.109375,1.015625,2.671875,0.0257568359375,-0.4765625,-4,5.15625,0.2314453125,-4.6875,3.1875,3.984375,-2.609375,3.4375,-2.375,-3.734375,-0.07568359375,2.75,-5.3125,1.9296875,4.625,-1.6484375,2.875,3.734375,-1.34375,3.875,-1.9921875,-11.3125,-1.53125,3.296875,5.71875,0.80859375,1.7578125,0.48046875,-2.015625,1.4765625,-0.5546875,0.71484375,-0.7578125,-11.1875,0.9765625,-3,-0.09765625,-1.9453125,-3.8125,-2.5,4.375,1.65625,1.1015625,3.328125,2.84375,0.84375,4.5625,0.11279296875,-5.84375,1.1484375,1.7578125,-4.8125,-0.59765625,3.234375,1.125,-1.859375,-2.515625,3.78125,-1.7421875,-0.69921875,5.8125,3.765625,1.578125,-1.84375,-5.03125,0.984375,-3.375,-1.9140625,1.1953125,-0.384765625,2.8125,-2.203125,2.828125,1.1171875,-3.75,-4.15625,-2.25,-3.5625,1.5,2.671875,2.171875,-2.609375,-1.7265625,2.8125,2.5,-0.455078125,-1.546875,2.1875,-0.1884765625,-2.984375,-1.4765625,2.0625,-4.46875,-2.90625,4.0625,1.8359375,0.443359375,-0.7734375,-3.140625,2.171875,1.734375,-1.8515625,-1.84375,-1.234375,2.15625,5.34375,-2.484375,-5.6875,-1.2734375,0.1806640625,-4.375,-3.5625,0.89453125,-1.15625,0.75,3.09375,-2.25,1.1875,4.6875,-1.3359375,-3.875,3.53125,4.4375,-2.671875,-0.75,-0.458984375,-2.53125,3.8125,5,-1.2421875,-2.109375,-0.50390625,-2.734375,-4.90625,1.0234375,2.421875,-3.34375,-10.125,6.46875,3.671875,5.40625,1.546875,-2.59375,3.8125,-1.6953125,3.703125,-0.423828125,0.82421875,1.515625,-7.59375,-2.40625,-2.0625,-5.0625,0.59375,-0.345703125,-4.75,1.4921875,6.25,-2.15625,-1.8671875,-2.703125,-3.9375,4.28125,-3.484375,-5.9375,1.984375,-7.4375,1.4609375,-1.9609375,3.265625,-5.875,1.8359375,-0.017333984375,2.046875,-0.5859375,-0.671875,-2.328125,1.1953125,-2.65625,3.625,0.7890625,3.9375,-0.365234375,2.90625,-1.2421875,0.314453125,-3.265625,1.6640625,1.7109375,0.60546875,0.384765625,2.296875,-2.28125,-0.8046875,-1.0546875,1.046875,2.796875,0.61328125,-0.625,0.10693359375,4.21875,-0.6484375,2.03125,-2.3125,-0.173828125,-1.015625,-0.224609375,0.74609375,-0.86328125,0.0145263671875,0.1318359375,1.7109375,1.421875,0.486328125,-0.19921875,0.140625,1.2734375,1.015625,1.5625,-1.65625,-0.45703125,-0.435546875,-0.0206298828125,1.828125,1.734375,-2.734375,1.65625,-2.09375,-0.6875,-0.2421875,2.125,1.1015625,0.1064453125,1.59375,-1.875,1.828125,0.15234375,-1.2421875,1.25,-0.765625,-2.265625,2.34375,-2.109375,-0.921875,0.6640625,-1.2734375,-1.4765625,-0.73828125,2.21875,-0.84375,1.328125,-1.171875,-0.181640625,0.306640625,-1.171875,0.279296875,0.94140625,1.171875,-3.921875,3.15625,1.2421875,0.52734375,-0.1630859375,1.0390625,-1.46875,-0.08447265625,1.0390625,-0.37109375,0.921875,1.859375,-1.8046875,0.54296875,-0.8203125,-1.09375,1.1640625,1.515625,0.54296875,-1.65625,-1,1.5234375,1.4453125,-1.1953125,0.359375,-0.062255859375,-2.09375,3.03125,1.21875,-3.15625,-0.357421875,-0.169921875,0.546875,-0.73828125,-0.126953125,1.046875,-2.75,-0.2314453125,0.2421875,0.306640625,-1.1328125,1.8984375,0.00469970703125,3.9375,0.8515625,1.1328125,1.1875,1.3984375,2.046875,-1.3515625,0.25390625,-0.9921875,3.234375,-0.373046875,0.8828125,1.3828125,-1.921875,-0.484375,-0.81640625,0.61328125,1.4296875,-0.70703125,-0.404296875,2.53125,1.625,0.494140625,2.375,-2.03125,0.33984375,0.291015625,-0.68359375,-1.625,1.625,-0.478515625,0.349609375,-2.0625,-1.25,-0.1484375,-0.44140625,0.67578125,0.3671875,0.4921875,0.236328125,1.1953125,0.5078125,-2.375,1.3671875,-0.341796875,0.6328125,-1.7265625,-1.328125,0.84375,-0.08935546875,1.0625,0.90625,1.984375,2.828125,1.109375,-1.3671875,1.03125,1.0625,1.75,0.263671875,-1.234375,-0.09228515625,-0.13671875,0.271484375,0.58203125,-0.9375,-1.28125,0.4609375,-0.95703125,-0.1552734375,-1.5703125,3.375,-0.9609375,-1.1796875,-0.419921875,-1.5,0.58984375,-1.3125,1,-1.578125,2.484375,1.34375,3.34375,1.4296875,-0.671875,-0.984375,0.30859375,0.72265625,-0.337890625,-0.06982421875,-1.125,-0.44921875,-0.62890625,5.40625,0.263671875,1.0390625,-2.03125,3.296875,0.68359375,-0.10986328125,-1.078125,-0.2412109375,-2.078125,-0.13671875,-1.4375,-1.390625,0.29296875,-1.1484375,-4.0625,-2.703125,-0.302734375,0.77734375,-1.640625,-0.0390625,3.890625,0.375,1.2890625,1.5,2.640625,0.19140625,-1.78125,-0.5859375,1.6328125,-1.234375,2,0.8125,-1.9453125,-2.78125,-0.3671875,-2.328125,-1.9453125,-0.59375,-0.8046875,1.9921875,-0.265625,-0.03515625,-1.3125,-1.5234375,-3.03125,-0.458984375,-0.1279296875,2.375,1.53125,0.67578125,-0.55078125,-0.4296875,0.515625,-1.75,0.6640625,-1.65625,4.25,-0.326171875,-1.4296875,2.53125,0.396484375,3.140625,0.859375,-1.3671875,-1.8828125,-0.828125,0.45703125,0.7109375,3.0625,-0.2578125,0.6328125,0.57421875,-0.85546875,0.5625,1.0234375,-0.296875,-4.84375,-1.578125,-0.486328125,2.59375,-1.2109375,0.09765625,2.59375,-0.87109375,-0.7890625,-1.7421875,-2.34375,-0.2490234375,-0.82421875,0.8046875,2.078125,-0.7265625,-0.10400390625,-0.703125,-1.046875,0.46875,-1.7734375,1.09375,-0.30859375,0.0181884765625,0.2734375,-2.703125,-0.470703125,0.67578125,-1.921875,-1.0078125,1.6328125,0.2021484375,1.359375,1.6796875,-1.6015625,1.5703125,0.6484375,-2.859375,-0.63671875,-0.8359375,1.34375,0.0556640625,0.4375,1.765625,-1.1484375,-1.90625,-1.453125,0.57421875,0.84375,-0.349609375,0.251953125,-0.0927734375,0.416015625,-0.40625,-2.71875,-0.48046875,0.4140625,-0.2109375,0.96484375,1.0859375,1.453125,1.15625,1.375,-0.478515625,1.375,-1.8828125,1.6484375,0.9921875,-2.171875,0.5859375,2.03125,-2.125,0.314453125,1.1796875,-0.4921875,-0.72265625,-0.80078125,0.5546875,-0.52734375,0.58203125,-0.52734375,1.9453125,1.71875,-0.328125,1.453125,-2.203125,-2.09375,-2.625,0.2177734375,-0.82421875,0.3359375,-2.203125,1.375,-1.7578125,-0.072265625,-0.4765625,-0.38671875,-1.9453125,1.5625,1.7578125,0.4453125,0.640625,0.0255126953125,-0.5703125,3.796875,-1.0703125,-0.1201171875,0.93359375,1.15625,-2.078125,3.484375,0.5234375,2.109375,0.0037078857421875,1.3359375,-0.796875,1.25,0.1455078125,0.86328125,0.478515625,1.828125,0.31640625,-0.296875,-0.154296875,-1.53125,-1.1640625,0.6484375,1.0703125,-5.375,0.86328125,0.890625,0.48828125,0.84765625,-2.828125,1.1015625,0.4765625,3.296875,-0.00408935546875,-0.40234375,3.421875,0.61328125,-1.46875,1.1875,0.953125,0.0771484375,-2.78125,-1.171875,-0.86328125,2.9375,-1.0703125,0.1015625,-0.279296875,-0.90625,3.046875,0.6796875,-1.6640625,1.453125,0.443359375,-0.439453125,-1.453125,-3.40625,-0.1689453125,1.71875,-0.9453125,2.234375,0.158203125,0.87109375,0.66796875,-1.640625,1,0.265625,0.267578125,-0.90625,1.75,-0.2041015625,-1.59375,1.65625,-1.1484375,-1.78125,2.421875,1.6953125,-2.328125,0.027587890625,-0.494140625,-0.3203125,-0.01953125,0.58203125,-2.28125,0.546875,0.62109375,0.90625,-0.921875,-1.53125,2.484375,1.890625,2.953125,2.359375,-0.90234375,0.171875,-2.234375,0.33984375,-0.45703125,-0.87109375,0.08251953125,1.8671875,-1.0078125,1.5703125,-0.30078125,0.921875,-1.8046875,1.609375,2.703125,0.92578125,0.40625,-0.26171875,-0.322265625,-1.8671875,-0.5,-2.296875,0.62109375,0.6953125,1.1640625,0.1376953125,-1.4296875,1.5390625],\"index\":0,\"object\":\"embedding\"},{\"embedding\":[-2.28125,-0.7734375,-0.8359375,-2.3125,3.046875,4.125,-1.0390625,-2.890625,0.0103759765625,1.9296875,0.1015625,1.75,2.4375,2.015625,5.09375,1.203125,-2.140625,-2.828125,-1.328125,-4.6875,1.0078125,6.8125,0.578125,-4.71875,-0.80859375,-6.25,1.578125,4.25,4.46875,-1.0078125,8,-2.3125,2.546875,-0.00555419921875,1.5625,-1.8671875,-2.375,-2.53125,5.25,-0.69140625,-2.96875,-0.68359375,1.6171875,2.96875,-3.015625,-1.734375,0.4140625,-2.9375,2.53125,-1.6640625,-4.5625,-1.9296875,3.234375,-2.734375,2.359375,-4.125,-3.046875,4.5,-5.875,-2.984375,-1.8515625,-2.8125,-0.7734375,0.46484375,1.3984375,5.28125,0.68359375,-1.3359375,0.51171875,8.625,-0.055908203125,3.578125,6.5,-2.390625,6.34375,5.5625,0.7265625,1.578125,-2.921875,4.90625,-2.953125,-0.62890625,2.453125,3.46875,4.5625,2.671875,-1.9140625,0.859375,-3.03125,1.703125,1.96875,0.59375,-1.4140625,-3.140625,-1.2109375,1.2890625,-3.21875,-6.5625,-6.78125,2.765625,-0.78515625,-0.3515625,1.8125,-4.53125,-5.03125,2.171875,-1.8515625,-5.46875,-1.78125,0.380859375,2.640625,1.65625,3.640625,-2.140625,2.46875,1.21875,4.28125,-2.796875,-4.40625,2.796875,-2.0625,-1.9765625,4.28125,-0.6796875,4.4375,4.28125,-4.03125,-0.01416015625,5.53125,-1.4609375,7.25,3.578125,3.6875,-2.375,-8.0625,-4.71875,-1.9453125,3.71875,4.3125,4.40625,-5.03125,3.21875,-3.734375,-6.625,4.1875,-3.4375,-6.4375,-3.15625,3.859375,-1.9140625,-1.78125,1.8046875,0.5,2.3125,-1.2421875,-4.375,4.0625,3.875,0.1259765625,-1.0546875,2.015625,3.328125,1.1484375,1.7265625,1.8046875,-0.462890625,-5.625,3.6875,-1.0390625,2.5625,0.90625,10.4375,4.28125,-4.5625,1.9765625,8.625,-1.328125,8.625,1.4609375,2.203125,0.81640625,-0.640625,-2.90625,4.53125,-2.15625,1.5,0.12255859375,-5.6875,3.140625,1.2890625,1.578125,1.5625,2.71875,-1,-4.84375,-1.8671875,3.484375,-2.578125,3.4375,0.1025390625,-1.40625,-7.375,1.4921875,1.5546875,-4.71875,-3.765625,2.703125,-1.71875,3.078125,-0.380859375,2.265625,0.24609375,3.21875,-2.0625,7.65625,2.640625,2.734375,2.046875,1.8359375,2.46875,4.53125,3.484375,1.8359375,-2.078125,-0.83984375,2.03125,5.8125,0.439453125,3.75,8.6875,0.251953125,0.408203125,6.84375,-2.515625,-1.78125,-3.578125,-3.78125,1.6015625,-0.279296875,2.671875,-5.65625,-4.0625,-2.328125,2.984375,3.515625,-3.359375,-2.34375,-2.703125,-0.51171875,-6.4375,1.484375,3.671875,-9.0625,1.8828125,5.625,3.96875,1.984375,1.265625,-0.33203125,-4.125,0.333984375,-2.4375,-5.875,-0.58203125,1.890625,-2.390625,5.09375,-1.5546875,3.515625,-0.7421875,5.1875,-2.28125,-0.0927734375,-3.046875,-4.3125,8.8125,-0.232421875,-1.90625,1.0703125,-3.078125,-3.5625,-10.25,2.5,1.1171875,4.96875,-2.921875,1.40625,0.40234375,-3.640625,12.75,3.90625,-1.8203125,1.9921875,-0.63671875,-6.03125,-1.984375,-2.046875,2.046875,-5.59375,1.84375,3.6875,4.5,-1.9296875,3.4375,-1.7421875,-0.9296875,-1.109375,-4.5625,-1.9375,2.671875,-3.765625,2.34375,9.625,-4.75,2.03125,-2.109375,-6.1875,4.75,-0.03662109375,-0.11376953125,-2.140625,-5.125,-1.9921875,-2.78125,-1.4296875,-6.65625,4.96875,-0.984375,5.375,0.97265625,3,3.296875,-4.1875,-5.03125,8.4375,-1.5,3.296875,5.71875,0.55078125,0.68359375,-3.515625,-4.6875,2.46875,-5.46875,0.953125,5.71875,3.328125,-1.640625,1.0234375,-6.21875,2.40625,2.328125,-0.68359375,6.53125,6.90625,-2.265625,2.78125,1.9140625,-0.71484375,-2.28125,-0.2294921875,-1.078125,6.34375,1.1875,-3.890625,-3.796875,-0.5859375,5.03125,-2.375,0.7734375,-1.21875,-4.15625,2.59375,-1.15625,3.6875,0.91796875,0.90625,-1.8046875,-5.125,0.087890625,-2.625,0.29296875,-1.7734375,-3.28125,4.25,1.515625,-0.484375,1.59375,0.67578125,-3.53125,-0.46484375,0.59765625,-1.15625,0.65625,2.5625,-0.5703125,-0.984375,1.5546875,-0.3828125,-2.21875,1.0546875,-1.2734375,2.40625,-6.9375,-0.6484375,-0.2490234375,-2.125,-8.375,-0.4765625,1.0703125,-3.78125,2.71875,1.96875,-1.2578125,-3.0625,4.4375,1.421875,1.8671875,-6.90625,2.15625,-1.8828125,3.328125,2.140625,-1.7421875,0.59375,-1.4296875,-2.765625,4.375,3.546875,-0.69921875,3.453125,0.68359375,-3.265625,-3.625,0.1630859375,-4.90625,4.75,-0.236328125,-1.859375,5.21875,2.203125,-1.5,1.625,0.98828125,-6.28125,-4.78125,2.96875,3.171875,-3.078125,-3.96875,0.470703125,-1.4296875,-4.4375,3.078125,3.84375,-1.1171875,-2.8125,3.40625,4.375,-2.203125,0.0830078125,1.1171875,0.52734375,2.703125,-1.9375,-3.140625,-0.1103515625,0.130859375,4.71875,-5.8125,-6.84375,3.015625,-2.875,0.2001953125,1.15625,4.5625,0.46875,-1.8984375,-1.9296875,-3.0625,-3.46875,-2.828125,3.53125,-1.078125,-2.53125,-2.90625,0.29296875,8.3125,1.90625,0.369140625,-2.375,-0.11572265625,2.453125,-1.71875,0.50390625,4.4375,7.90625,-4.03125,-0.63671875,3.53125,-8.125,0.94921875,-1.375,-1.15625,-0.94921875,2.3125,2.1875,-6.25,-0.7890625,0.0115966796875,5.03125,-3.453125,-3.828125,5.15625,-4.8125,-3.09375,1.859375,-0.6875,4.0625,1.296875,-1.34375,2.875,2.984375,2.65625,1.8203125,-2.53125,-3.640625,-3.3125,1.2890625,2.265625,-2.234375,2.296875,4,-5.4375,0.90234375,-2.25,-0.6953125,-0.212890625,-0.515625,5.90625,2.125,2.25,-6.09375,1.2578125,0.50390625,-0.416015625,-0.7421875,-1.1484375,6.71875,-0.5,-0.2294921875,0.94921875,2.09375,-1.1953125,1.640625,-3.796875,-2.453125,-3.109375,-1.796875,-1.0234375,-4.03125,-5.5,4.4375,6,-1.234375,-1.6796875,2.171875,5.5,3.984375,-0.84375,1.515625,3.421875,-2.5,0.23828125,-5.40625,2.609375,-7.84375,-2.53125,-1.6875,2.921875,3.75,-4.15625,3.765625,-2.578125,2.4375,-1.4375,4.4375,-10.5625,2.046875,-2.15625,-2.796875,-2.28125,-0.57421875,3.171875,-0.44921875,2.109375,1.3671875,-0.75,3.953125,5.46875,-1.5,1.765625,2.1875,2.46875,-0.5859375,2.515625,-2.125,-8.25,1.3125,-1.1484375,1.09375,7.5625,1.9375,-1.7734375,2.46875,0.88671875,-1.5703125,-1.7265625,4.0625,3.015625,-1.546875,4.25,-3.90625,5.40625,-3.28125,1.7265625,-3.265625,-6.15625,0.279296875,1.9296875,-5.5625,-4.09375,2.859375,0.216796875,5.78125,3.421875,-5.375,1.21875,-0.41796875,1.109375,2,0.30078125,-0.03759765625,-4.75,3.921875,4.1875,-2.40625,7.03125,-1.5703125,-1.6484375,-1.1171875,2.40625,-1.7734375,0.373046875,1.84375,0.287109375,-0.78125,-3.484375,0.96484375,0.5703125,-6.625,-7.21875,1.7265625,-1.7734375,7.0625,0.73046875,-0.859375,-3.15625,2,1.5546875,6.375,3.3125,3.765625,4.5,3.765625,-2.390625,2.671875,-3.6875,-6.09375,7,-6.53125,-1.8515625,1.015625,0.859375,-0.2578125,-1.0234375,-0.3515625,-0.71484375,-3.484375,-6.09375,-2.359375,-1.875,2.015625,-1.6484375,2.203125,0.57421875,-4.09375,-0.5703125,-1.6484375,-1.6875,-1.6640625,4.15625,-5.625,1.484375,5.71875,2.046875,-1.5234375,4.15625,3.09375,-0.47265625,-4.78125,0.7109375,-6.875,1.6015625,1.46875,-0.6015625,0.50390625,-8,2.03125,-2.4375,3.5,-0.671875,-0.05078125,-1.265625,-3.296875,-1.3984375,-0.91796875,-5.40625,-0.171875,1.6953125,1.125,-1.8359375,0.671875,3.078125,-0.52734375,0.384765625,-1.125,2.046875,0.40625,2.34375,-4.78125,-2.90625,1.28125,0.9140625,-2.03125,6.53125,0.91796875,0.79296875,3.546875,1.7265625,-5.5,-5.78125,3.921875,-2.8125,-1.796875,-3.25,2.421875,-1.359375,6.53125,-2.21875,-5.53125,-3.703125,1.6484375,3.15625,-2.609375,-3.09375,4.78125,1.8359375,2.765625,-2.15625,-7.5,1.609375,0.98828125,-0.146484375,-1.140625,8.625,-1.9296875,-0.4765625,-4.4375,-3.234375,2.046875,0.875,2.046875,-0.76171875,-1.2734375,0.69921875,0.4765625,-2.34375,-0.55078125,0.6015625,-2.546875,1.75,0.07177734375,4.875,-2.53125,0.3984375,-1.2734375,-0.50390625,-0.10009765625,4.3125,8.75,-1.765625,-0.96875,0.35546875,2.984375,-3.59375,6.6875,1.3515625,7.75,-1.1640625,0.25,1.03125,0.375,-2.171875,4.59375,-5.25,-2.84375,-1.890625,1.21875,-2.5625,0.671875,-3.984375,-0.498046875,4.40625,-0.455078125,-0.007568359375,2.609375,0.79296875,-0.201171875,-3.09375,-1.3125,-4.71875,-2.515625,-0.14453125,2.03125,-3.03125,-0.4921875,-0.33984375,5.84375,-0.357421875,-1.4453125,-2.59375,1.53125,1.859375,1.171875,-0.8046875,0.255859375,0.58984375,3.3125,-1.015625,-4.34375,-0.94921875,8.4375,4.21875,-6.875,1.5703125,-0.43359375,1.4453125,-4.8125,-1.4609375,-2.15625,-1.4921875,-4.1875,1.1328125,0.419921875,-3,-0.06494140625,4.5,-1.2890625,-0.15625,3.46875,4.0625,0.478515625,2.96875,-2.125,4.375,2.21875,-2.09375,-5.96875,-1.703125,0.48046875,-2.75,-1.4140625,2.03125,6.15625,0.55859375,2.625,-1.0625,2.28125,-1.6953125,3.78125,5.125,-4.59375,-2.703125,-2.3125,-9.5625,-4.03125,-1.7421875,-2.921875,-5.34375,-4.25,-0.86328125,-1.2421875,-8,0.0966796875,-2.234375,-3.265625,1.4453125,2.953125,1.7578125,-5.75,3.125,4.125,2.578125,2.546875,0.84765625,5.46875,-0.050537109375,-2.96875,1.4453125,-3.4375,4.15625,-1.03125,3.546875,6.25,-0.453125,-4.96875,4.78125,2.96875,5.53125,-7.375,-2.625,-0.337890625,-1.671875,-0.458984375,-1.7578125,2.546875,-4.5,-5.5,1.078125,-3.203125,1.2265625,4.6875,-0.8046875,6.78125,1.6328125,0.419921875,2.140625,2.71875,0.62109375,0.169921875,1.7421875,-5.9375,3.234375,-2.171875,3.265625,-0.296875,-1.5234375,2.734375,-0.7578125,-0.310546875,2.8125,2.734375,10.3125,0.515625,4,-2.3125,0.63671875,-1.7265625,-0.2392578125,2.25,2.015625,0.79296875,-1.4765625,0.7890625,-0.44921875,0.478515625,-0.4609375,-13.25,-1.9609375,-7.25,-1.9296875,7.0625,-2.1875,-1.9921875,1.4296875,2.6875,3.484375,5.125,-0.58984375,3.375,-0.60546875,0.80859375,5.96875,-4.25,1.03125,3.359375,2.546875,5.21875,0.154296875,-0.44921875,-3.203125,8,2.25,-1.4140625,0.8359375,2.796875,-1.3046875,-2.34375,3.09375,-3.171875,2.96875,-4.9375,0.5859375,4.15625,0.65625,-3.890625,-3.4375,-2,-0.62890625,1.3828125,1.375,-2.59375,0.18359375,0.94921875,-4.1875,3.328125,-0.59375,0.140625,-5.53125,1.03125,4.65625,0.703125,-0.109375,-1.8515625,1.4453125,-0.8984375,4.3125,2.78125,-2.734375,0.2734375,2.21875,1.7421875,-0.125,1.03125,1.1328125,2.921875,-3.09375,-0.353515625,-0.44140625,-1.625,1.4765625,-3.1875,1.6640625,3.203125,1.3984375,-3.984375,2.21875,0.79296875,-0.11669921875,2.96875,-5.125,-1.9921875,-1.1015625,-0.71484375,-4.0625,-0.9140625,-4.375,-0.1455078125,5.46875,-5,3.4375,-2.515625,8.1875,0.1298828125,-1.421875,1.2890625,-2.828125,2.59375,-3.390625,-1.234375,3.484375,-0.92578125,2.125,-3.546875,1.8984375,-2.078125,-0.46484375,6.09375,-3.953125,-1.9765625,0.7421875,3.21875,-5.0625,-3.296875,0.1611328125,0.8515625,0.009765625,-1.8984375,1.4765625,-2.03125,4.4375,-4.75,3.390625,-4.65625,-3.90625,0.28125,0.07568359375,7.90625,4.25,-3.796875,-3.421875,-0.6015625,-7.0625,-3.421875,-3.859375,6.65625,-0.52734375,0.96875,2.078125,2.390625,-0.01031494140625,1.46875,-2.96875,3.203125,5.28125,0.294921875,3.046875,2.1875,-1.125,-4.40625,0.3125,-3.171875,7.0625,3.0625,0.404296875,3,-1.8984375,1.484375,-1.03125,-1.0625,-2.828125,2.171875,1.71875,-2.5,-3.28125,1.046875,-3.859375,0.72265625,-5.40625,-2.578125,-5.3125,2.765625,2.3125,-0.81640625,-0.7578125,4.4375,0.318359375,3.328125,-5.53125,-3.890625,3.8125,0.9765625,0.333984375,2.84375,-0.6796875,-5.03125,-0.9375,0.201171875,1.9140625,-4.1875,-3.609375,3.328125,2.46875,0.283203125,-3.9375,-4.40625,-3.453125,2.390625,4.1875,-0.96484375,0.353515625,0.06005859375,-1.53125,2.171875,-2.65625,4.5,-3.109375,-4.15625,-0.47265625,0.734375,3.578125,-3.203125,-1.0703125,1.4296875,-3.4375,0.7578125,1.2734375,-0.11279296875,-1.9453125,3.171875,-2,-3.65625,-5.4375,5.78125,-2.0625,0.45703125,-3.875,-2.65625,-3.1875,-1.421875,-0.6640625,1.7421875,0.0703125,5.78125,-0.63671875,2.8125,0.478515625,-0.8828125,0.0712890625,3.453125,-0.271484375,-2.90625,1.8359375,-4.59375,-4.65625,0.7578125,-8.0625,-2.0625,2.90625,-2.40625,2.671875,-2.671875,2.375,-1.1015625,-2.21875,-1.8203125,-0.8203125,0.83984375,5.375,2.171875,0.2216796875,0.38671875,1.8984375,0.859375,-1.109375,-1.8515625,-0.25,5.34375,0.62109375,2.765625,-3.359375,-2.34375,4.46875,-0.59375,-3.75,0.8984375,-0.357421875,0.6640625,4.5625,0.9609375,-3.796875,-2.9375,-6.15625,4.03125,0.73828125,1.828125,-4.625,1.5,-3.0625,0.1748046875,2.03125,-6.5625,-2.546875,3.328125,2.828125,5.46875,1.328125,-2.421875,-4.53125,2.203125,-0.396484375,-1.6171875,-2.234375,-1.7265625,-0.96875,-3.765625,4.125,-2.515625,4.25,-1.3359375,-2.8125,-0.8671875,0.61328125,-0.203125,0.47265625,-0.353515625,-0.88671875,4.0625,-0.3515625,7,2.171875,-4.0625,4.59375,2.515625,0.412109375,-1.5625,3.75,-1.109375,-2.3125,3.921875,2.890625,-4.0625,4.96875,2.125,3.375,-3.46875,-2.1875,-0.9921875,4.5625,0.287109375,1.28125,-4.34375,0.1630859375,4.0625,-0.1884765625,0.8671875,-1.765625,0.3046875,0.65234375,0.52734375,2,1.921875,3.4375,-0.52734375,1,-0.92578125,-1.2265625,2.328125,-0.1328125,-0.703125,-1.8828125,3.21875,-1.6953125,-1.875,-6,1.2421875,-3.46875,2.21875,3.1875,2.875,2.234375,-2.828125,-1.625,-2.640625,-5.25,-3.140625,1.75,1.09375,-1.75,1.875,-0.1181640625,2.546875,5.84375,0.130859375,4.6875,-3.109375,2.5,1.140625,0.875,0.046630859375,4.3125,-1.8203125,-2.21875,3.640625,-4.46875,3.71875,-4.53125,-3.078125,-0.63671875,-0.10986328125,2.640625,6.625,-4.5625,-3.953125,5.21875,1.328125,4.59375,3.78125,-2.078125,-1.484375,0.79296875,1.3515625,5.46875,0.93359375,2.953125,-2.734375,6.9375,5.65625,0.90625,2.359375,0.166015625,-2.6875,-6.4375,5.125,1.3984375,1.984375,-2.375,1.6875,3.109375,0.1533203125,3.640625,-5.5,0.8671875,1.2109375,0.90625,0.5234375,-3.15625,0.103515625,2.640625,0.33203125,-1.6875,5.84375,0.97265625,4.125,-0.72265625,3.34375,2.328125,3.703125,-2.03125,1.5234375,-3.46875,3.578125,-1.3984375,2.15625,-5.5,1.0546875,3.640625,4.3125,-1.625,-3.5625,2.21875,0.275390625,-0.5,-4.46875,4.21875,3.59375,2.5625,-6.9375,-3.328125,-0.05029296875,0.2060546875,1.234375,-3.484375,1.171875,1.6796875,-4.625,-3.265625,1.296875,1.625,-5.65625,-6.0625,-3.203125,1.65625,1.3203125,3.1875,3.21875,-0.8203125,3.40625,-0.55078125,3.046875,4.28125,-1.1328125,1.5546875,0.9375,-2.75,4.125,-0.263671875,-2.671875,1.5546875,-0.50390625,-2.140625,0.50390625,-2.296875,-1.0703125,-4.21875,-0.85546875,2.328125,-1.09375,5.125,-3.96875,0.30078125,3.609375,-1.4375,-2.28125,-2.65625,0.5703125,-2.921875,-2.578125,-1.9140625,3.609375,2.984375,2.046875,0.58203125,-0.6015625,-3.265625,-6.40625,-5.65625,3.578125,-2.515625,2.859375,0.439453125,-4.25,2.078125,2.8125,1.78125,-0.1640625,-0.55859375,2.765625,4.59375,0.455078125,-1.7265625,-0.466796875,3.609375,-4.5625,-3.78125,0.515625,1,-3.171875,2.28125,-3.125,-1.8359375,0.79296875,4.5,-0.5078125,-2.859375,-1.75,-2.40625,-2.875,-3.03125,-2.859375,2.5625,1.859375,3.296875,0.1689453125,-0.421875,-5,3.71875,16.875,0.9375,-4.71875,2.421875,-3.140625,2.65625,3.171875,4.8125,-1.7109375,-1.96875,-2.1875,1.765625,0.01031494140625,1.4140625,-2.140625,1.7421875,1.9921875,-0.48828125,-4.125,-1.9765625,-1.328125,0.84765625,-0.7578125,2.96875,0.408203125,2.265625,-0.734375,-0.259765625,0.2333984375,-3.234375,-4.46875,-4.4375,2.265625,-1.7578125,4.75,-4.25,5.375,0.1845703125,-2.9375,-2.09375,-3.296875,-3.171875,1.0234375,-0.75,-1.9453125,4.34375,-0.72265625,1.09375,0.37890625,-0.337890625,-3.546875,-3.046875,-2.6875,7.25,0.62890625,-5.71875,-1.546875,-4.84375,-4.5625,0.58984375,2.796875,-2.328125,1.6328125,1.453125,-1.828125,-2.171875,-1.953125,0.85546875,3,-5.125,-5.625,0.13671875,1.5546875,3.359375,2.796875,-4.0625,1.5703125,5.3125,2.6875,0.69140625,-0.75,1.4453125,-1.3828125,-2.5,-0.91015625,1.4609375,-4.03125,1.109375,1.4453125,-4.875,11.25,-8.625,4.8125,4.0625,-4.75,-0.1865234375,2.796875,1.796875,-1.6796875,-0.169921875,2.953125,2.453125,3.359375,-0.306640625,6.09375,1.5234375,0.388671875,0.73828125,2.9375,3.578125,2.4375,2.9375,-0.828125,-1.9609375,1.3046875,1.7734375,-2.484375,-3.46875,-1.4609375,-4.4375,6,1.6171875,-2.765625,-1.2578125,-10.5,-3.421875,-2.328125,-5.84375,4.5,-2.65625,2.46875,3.421875,-0.609375,-1.078125,-2.53125,-5,2.296875,4.0625,0.208984375,-0.3984375,-6.0625,2.84375,3.546875,-3.984375,-2.09375,1.4453125,-3.265625,3.296875,-0.1923828125,4.9375,-3.578125,3.9375,2.03125,-2.546875,-5.8125,3.171875,-3.765625,-2.234375,-5.3125,-2.453125,-2.078125,-3.328125,-0.6171875,-0.35546875,-2.078125,-1.03125,1.6171875,-0.60546875,-3.15625,2.921875,2.96875,-4.375,-2.625,0.58203125,0.73046875,-4.28125,1.1875,5.1875,-0.54296875,1.5,0.55078125,0.078125,-0.3203125,-4.34375,0.81640625,1.71875,-4.03125,-0.71875,-1.359375,-2.828125,-2.4375,-2.78125,-3.375,3.875,3.59375,-5.0625,1.9609375,-0.34765625,0.014892578125,-1.4453125,-1.546875,6.4375,2.234375,-1.6484375,5.59375,1.03125,-4.15625,-2,-2.046875,-1.1484375,-1.2734375,6.3125,1.2578125,2.375,-5.90625,7.53125,2.453125,1.7265625,-0.43359375,2.34375,1.6796875,-3.71875,-5.40625,2.46875,2.75,3.84375,-4.59375,0.6328125,0.53515625,0.53125,-4.28125,1.90625,-0.259765625,0.482421875,-3.140625,-7.59375,-0.109375,0.90625,-1.8828125,1.5234375,4.25,-2.96875,1.3828125,0.95703125,-0.58984375,3.640625,3.28125,-2.828125,1.90625,-0.1904296875,2.625,-2.34375,1.4921875,-3.71875,-4.96875,-3.109375,-1.765625,1.8828125,-2.625,0.67578125,-0.357421875,-4.1875,2.109375,-2.25,1.125,1.09375,0.2578125,-6.25,3.984375,5.1875,-4.15625,4.4375,-5.53125,-2.4375,-1.640625,2.21875,-1.9140625,-6.46875,2.0625,4.5,-3.390625,2.203125,3.546875,-1.625,-0.4453125,-2.25,5.3125,-1.015625,4.78125,-0.6953125,3.953125,3.9375,-1.28125,-0.061279296875,-5.125,0.470703125,-2.28125,-3.84375,5.53125,-1.921875,2.46875,5.21875,4.9375,-9,-1.96875,0.54296875,-0.1845703125,3.578125,3.109375,-1.3671875,1.0234375,0.028076171875,-0.30859375,4.4375,-0.9296875,-1.46875,-3.65625,4.96875,-0.1728515625,-4.0625,2.984375,2.609375,-4.15625,4.34375,-2.75,-2.6875,-0.6875,-0.1396484375,-5.625,1.8046875,2.6875,-0.92578125,3.4375,3.109375,1.203125,3.59375,-2.640625,-10.0625,0.0703125,2.75,5.3125,1.7265625,2.3125,0.0859375,-1.0625,3.640625,-4.5625,0.46875,-1.484375,-9.5,0.255859375,-4.15625,-1.609375,-3.453125,-1.4921875,-1.9453125,3.90625,1.3984375,-0.8515625,3.5,2.921875,0.453125,4.15625,-0.361328125,-3.578125,1.2734375,1.75,-5.28125,-1.90625,4.8125,3.578125,-2.203125,-2.0625,3.84375,-4.28125,-0.70703125,4.3125,4.28125,2.15625,-0.828125,-3.234375,2.84375,-2.546875,-2.828125,1.703125,-3.421875,2.453125,-1.4375,2.578125,1.296875,-2.640625,-2.03125,-4.15625,-2.71875,3.484375,0.28515625,0.9765625,-2.265625,-1.1171875,3.234375,3.5625,-2.359375,-2.109375,2.796875,-1.3515625,-4.28125,-1.0859375,1.0859375,-5.90625,-2.609375,2.734375,3.4375,-2.5625,-3.5625,-2.125,1.6171875,1.3046875,-0.8984375,-0.1318359375,-3.53125,2.65625,5.0625,-2.9375,-3.75,-1.6171875,-0.486328125,-5.03125,-3.609375,-0.1767578125,1.140625,-0.73046875,3.890625,-1.40625,0.47265625,4.4375,-3.65625,-3.21875,3.96875,3.359375,-3.203125,-1.46875,2.25,-3.375,1.03125,5.4375,-2.390625,-2.234375,0.41796875,-2.171875,-4.28125,2.34375,1.2265625,-3.734375,-7.875,5.96875,1.0703125,4.34375,4.125,-3.90625,4.0625,-4.6875,1.8828125,-1.265625,1.015625,1.3828125,-5.65625,-1.1875,-2.5,-3.5,0.5390625,-1.734375,-3.5625,0.66015625,8.0625,-1.328125,-2.59375,-2.953125,-3.515625,3.3125,-4.15625,-7.625,0.1181640625,-7.34375,1.734375,-2.1875,1.75,-5.59375,1.9140625,-1.078125,1.734375,-2.984375,0.27734375,-0.384765625,1.21875,0.54296875,4.6875,1.2109375,1.984375,-0.1484375,2.71875,0.0791015625,1.875,-1.453125,-0.4921875,1.21875,-1.234375,0.33203125,0.69921875,-2.734375,0.1708984375,-1.7578125,-0.263671875,-1.015625,1.7578125,2.9375,-0.640625,-0.291015625,-1.6875,1.703125,-4.5,1.3125,-1.796875,0.859375,-0.78515625,-1.0078125,1.9609375,-2.328125,1.6640625,1.015625,1.640625,0.01068115234375,-1.5,2.234375,2.6875,-0.031982421875,-2.328125,-1.8046875,-0.55859375,-1.7421875,1.7421875,0.55078125,-2.0625,2.9375,-1.640625,-0.41015625,0.890625,1.7265625,0.44140625,-1.6484375,2.40625,-1.8671875,1.2890625,1.0859375,-1.5234375,2.609375,0.63671875,1.03125,1.2734375,0.9765625,-2,0.64453125,0.2578125,-1.4375,-0.291015625,3.484375,-1.7265625,0.31640625,-1.078125,-0.5625,1.0859375,-0.8671875,1.2109375,0.15625,-0.396484375,-2.75,2.640625,-2.125,-1.2578125,-0.42578125,0.29296875,-0.5703125,0.8984375,0.08935546875,1.2109375,-0.29296875,2.28125,-0.73828125,2.171875,-0.020263671875,-0.2060546875,1.3359375,3.421875,-1.984375,0.7421875,-2.0625,-1.1328125,1.3203125,-0.3046875,1.15625,-0.93359375,-2,1.2421875,1.1328125,-2.984375,-0.734375,2.265625,-0.189453125,-1.1328125,-0.609375,1.2265625,-0.75390625,-0.38671875,0.419921875,-0.89453125,2,3.265625,-1.0625,2.5,-1.453125,0.396484375,0.73046875,1.046875,2.3125,0.07958984375,-2.34375,-0.9296875,2.71875,-1.4375,0.37109375,0.890625,-1.53125,-0.1396484375,1.3359375,0.5703125,1.640625,-0.06982421875,-1.859375,-0.330078125,-0.6796875,1.609375,1.65625,-1.6875,0.68359375,-1.8359375,-0.53125,-1.015625,2.765625,-1.7578125,-2.140625,-0.78515625,-1.1015625,-0.83203125,-0.498046875,0.11962890625,-0.1298828125,0.60546875,1.125,1.5,0.4296875,-0.609375,1.4375,-0.08056640625,0.68359375,-1.1875,-1.5234375,1.484375,1.2421875,2.34375,-1.359375,1.34375,0.9296875,0.8828125,-1.1796875,1.9453125,-0.5234375,0.314453125,0.010986328125,-0.1181640625,1.40625,2.21875,0.318359375,0.5859375,-0.1328125,1.40625,0.69921875,1.375,-1.3046875,-2.203125,-1.0078125,-1.4296875,-2.125,0.361328125,-0.0615234375,-1.3046875,-0.1904296875,0.034912109375,-0.86328125,1.375,1.1796875,1.5390625,-0.828125,-0.58203125,0.1787109375,-0.328125,0.25390625,0.8828125,-0.8046875,-0.78125,-1.1171875,-2.0625,1.578125,0.88671875,-1.09375,-0.2890625,2.0625,-1.5,1.0078125,-2.78125,0.55078125,-1.828125,-0.341796875,0.0859375,-3.265625,0.34765625,-0.12451171875,-2.15625,-3.078125,-1.75,-0.85546875,-2.375,-0.3203125,4,-0.81640625,-1.21875,2.03125,0.08203125,-1.0078125,-0.94921875,1.7578125,2.84375,-0.8203125,3.859375,0.349609375,-0.16015625,-1.3984375,-1.265625,0.52734375,-1.2890625,0.294921875,-0.84765625,-0.8046875,-1.6796875,-3.109375,0.05859375,-4.1875,-2.125,0.1337890625,0.90625,1.890625,-0.08447265625,-0.7421875,-0.56640625,-0.96875,2.796875,-0.267578125,0.18359375,1.4375,0.27734375,0.46875,-1.4140625,0.92578125,-0.84375,2.953125,-1.171875,-0.50390625,-2.65625,-1.5546875,-4.1875,1.453125,2.484375,0.421875,2.96875,1.3671875,-0.5546875,-2.5625,0.07421875,0.00909423828125,-4.75,-0.373046875,-0.7265625,0.07275390625,-1.4140625,-0.7109375,-0.1318359375,-0.609375,-1.328125,-0.51953125,-1.828125,-0.271484375,-2.28125,2.984375,1.7890625,1.875,2.3125,0.3125,-0.31640625,1.1875,2.359375,1.1484375,0.6953125,0.255859375,0.408203125,-1.09375,2.09375,0.337890625,0.4609375,-1.2265625,0.2275390625,1.1875,2.5625,1.734375,-0.76171875,0.85546875,0.328125,-1.9140625,-1.40625,0.31640625,0.296875,1.140625,0.333984375,1.03125,-1.2890625,0.416015625,-0.6875,0.9453125,1.7578125,-1.953125,1.109375,-0.134765625,0.1787109375,-1.5,1.203125,1.15625,1.8203125,-0.48046875,2.140625,1.1640625,0.48828125,1.8515625,2.609375,-0.361328125,1.421875,-0.86328125,1.953125,0.51953125,-2.484375,3.15625,-0.34375,-0.47265625,-0.56640625,1.2890625,1.359375,-0.60546875,-0.25,-0.38671875,2.015625,0.52734375,0.14453125,1.8828125,0.67578125,-0.546875,-0.77734375,-0.6015625,-1.09375,-2.328125,-1.0078125,-3.0625,-0.37109375,-0.9375,1.765625,-0.828125,-1.484375,-0.142578125,1.390625,-0.02099609375,1.3203125,1.6171875,-1.0859375,2.09375,0.154296875,0.1962890625,0.89453125,-0.97265625,-1.2421875,1.15625,0.82421875,-0.59765625,4.625,0.1962890625,2.28125,-0.65625,-1.0390625,-0.78515625,3.59375,-0.44921875,-0.4375,-1.6953125,1.140625,-0.296875,-1.25,-0.76953125,-1.3984375,-0.9765625,1.78125,-0.87109375,-3.234375,-2.171875,0.330078125,-1.875,0.48828125,-1.859375,-1.0390625,2.40625,1.734375,-0.63671875,0.216796875,1.125,-1.0234375,0.58984375,-0.4296875,0.3515625,1.6015625,-1.2109375,1.765625,0.5859375,2.796875,-3.921875,-0.298828125,2.171875,1.578125,-0.458984375,-1.015625,-0.51171875,2.109375,0.369140625,-0.018798828125,-0.50390625,-4.46875,0.0135498046875,-0.043212890625,-3.21875,-0.09423828125,0.4921875,1.2421875,0.6640625,-3.15625,0.73046875,-1.5078125,-1.6328125,3.46875,-0.55078125,-0.41796875,0.58203125,1.1640625,-0.83203125,-0.84765625,1.53125,0.17578125,-3.484375,-1.1015625,-0.1591796875,-0.875,0.59765625,0.01373291015625,0.099609375,0.546875,-0.36328125,-1.171875,-1.1328125,-0.33984375,-0.08056640625,1.015625,4,1.1484375,1.265625,1.2109375,-2.125,4.5625,-2.515625,-0.96484375,1.1015625,1.3515625,-1.1796875,3.921875,1.109375,0.2265625,-2,0.55859375,2.96875,0.765625,0.9453125,0.671875,1.28125,1.7421875,1.78125,-1,-1.8671875,1.5,-0.35546875,-2.5,0.012451171875,0.2578125],\"index\":1,\"object\":\"embedding\"}],\"model\":\"doubao-embedding-text-240715\",\"object\":\"list\",\"usage\":{\"prompt_tokens\":7,\"total_tokens\":7}}"), - new TypeReference>>() {}); - Assertions.assertEquals(2, lists.size()); - Assertions.assertEquals(2560, lists.get(0).size()); - } -} 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 index 2de785a1a8b..f32cc870559 100644 --- 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 @@ -24,17 +24,12 @@ 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.nlpmodel.llm.remote.custom.CustomModel; -import org.apache.seatunnel.transform.nlpmodel.llm.remote.openai.OpenAIModel; +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; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; public class LLMRequestJsonTest { @@ -63,50 +58,4 @@ void testOpenAIRequestJson() throws IOException { OBJECT_MAPPER.writeValueAsString(node)); model.close(); } - - @Test - void testCustomRequestJson() throws IOException { - SeaTunnelRowType rowType = - new SeaTunnelRowType( - new String[] {"id", "name"}, - new SeaTunnelDataType[] {BasicType.INT_TYPE, BasicType.STRING_TYPE}); - - Map header = new HashMap<>(); - header.put("Content-Type", "application/json"); - header.put("Authorization", "Bearer " + "apikey"); - - List> messagesList = new ArrayList<>(); - - Map systemMessage = new HashMap<>(); - systemMessage.put("role", "system"); - systemMessage.put("content", "${prompt}"); - messagesList.add(systemMessage); - - Map userMessage = new HashMap<>(); - userMessage.put("role", "user"); - userMessage.put("content", "${input}"); - messagesList.add(userMessage); - - Map resultMap = new HashMap<>(); - resultMap.put("model", "${model}"); - resultMap.put("messages", messagesList); - - CustomModel model = - new CustomModel( - rowType, - SqlType.STRING, - "Determine whether someone is Chinese or American by their name", - "custom-model", - "https://api.custom.com/v1/chat/completions", - header, - resultMap, - "{\"model\":\"${model}\",\"messages\":[{\"role\":\"system\",\"content\":\"${prompt}\"},{\"role\":\"user\",\"content\":\"${data}\"}]}"); - ObjectNode node = - model.createJsonNodeFromData( - "Determine whether someone is Chinese or American by their name", - "{\"id\":1, \"name\":\"John\"}"); - Assertions.assertEquals( - "{\"messages\":[{\"role\":\"system\",\"content\":\"Determine whether someone is Chinese or American by their name\"},{\"role\":\"user\",\"content\":\"{\\\"id\\\":1, \\\"name\\\":\\\"John\\\"}\"}],\"model\":\"custom-model\"}", - OBJECT_MAPPER.writeValueAsString(node)); - } } diff --git a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java index 4e5d864369f..11b240dd993 100644 --- a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java +++ b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java @@ -24,10 +24,6 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; -import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; -import org.apache.seatunnel.api.source.event.ReaderCloseEvent; -import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; import lombok.extern.slf4j.Slf4j; @@ -140,7 +136,6 @@ public void open() throws Exception { ThreadPoolExecutorFactory.createScheduledThreadPoolExecutor( parallelism, "parallel-split-enumerator-executor"); splitEnumerator.open(); - coordinatedEnumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); restoredSplitStateMap.forEach( (subtaskId, splits) -> { splitEnumerator.addSplitsBack(splits, subtaskId); @@ -152,10 +147,6 @@ public void open() throws Exception { entry -> { try { entry.getValue().open(); - readerContextMap - .get(entry.getKey()) - .getEventListener() - .onEvent(new ReaderOpenEvent()); splitEnumerator.registerReader(entry.getKey()); } catch (Exception e) { throw new RuntimeException(e); @@ -212,7 +203,6 @@ public void close() throws IOException { for (Map.Entry> entry : readerMap.entrySet()) { readerRunningMap.get(entry.getKey()).set(false); entry.getValue().close(); - readerContextMap.get(entry.getKey()).getEventListener().onEvent(new ReaderCloseEvent()); } if (executorService != null) { @@ -221,7 +211,6 @@ public void close() throws IOException { try (SourceSplitEnumerator closed = splitEnumerator) { // just close the resources - coordinatedEnumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); } } diff --git a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java index ed794a5b6cb..4cc1bfd1418 100644 --- a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java +++ b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java @@ -23,10 +23,6 @@ import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; -import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; -import org.apache.seatunnel.api.source.event.ReaderCloseEvent; -import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; import org.slf4j.Logger; @@ -119,9 +115,7 @@ public void open() throws Exception { splitEnumerator.addSplitsBack(restoredSplitState, subtaskId); } reader.open(); - readerContext.getEventListener().onEvent(new ReaderOpenEvent()); parallelEnumeratorContext.register(); - parallelEnumeratorContext.getEventListener().onEvent(new EnumeratorOpenEvent()); splitEnumerator.registerReader(subtaskId); } @@ -176,8 +170,6 @@ public void close() throws IOException { if (reader != null) { LOG.debug("Close the data reader for the Apache SeaTunnel source."); reader.close(); - readerContext.getEventListener().onEvent(new ReaderCloseEvent()); - parallelEnumeratorContext.getEventListener().onEvent(new EnumeratorCloseEvent()); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java index 2ebbcba4f91..4a720e347b2 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSink.java @@ -66,7 +66,10 @@ public SinkWriter, FlinkWriterState> if (states == null || states.isEmpty()) { return new FlinkSinkWriter<>( - sink.createWriter(stContext), 1, catalogTable.getSeaTunnelRowType(), stContext); + sink.createWriter(stContext), + 1, + catalogTable.getSeaTunnelRowType(), + stContext.getMetricsContext()); } else { List restoredState = states.stream().map(FlinkWriterState::getState).collect(Collectors.toList()); @@ -74,7 +77,7 @@ public SinkWriter, FlinkWriterState> sink.restoreWriter(stContext, restoredState), states.get(0).getCheckpointId() + 1, catalogTable.getSeaTunnelRowType(), - stContext); + stContext.getMetricsContext()); } } diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java index 8de831aee17..725bf606f93 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/sink/FlinkSinkWriter.java @@ -23,7 +23,6 @@ import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SupportResourceShare; -import org.apache.seatunnel.api.sink.event.WriterCloseEvent; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -54,8 +53,6 @@ public class FlinkSinkWriter private final org.apache.seatunnel.api.sink.SinkWriter sinkWriter; - private final org.apache.seatunnel.api.sink.SinkWriter.Context context; - private final Counter sinkWriteCount; private final Counter sinkWriteBytes; @@ -70,11 +67,9 @@ public class FlinkSinkWriter org.apache.seatunnel.api.sink.SinkWriter sinkWriter, long checkpointId, SeaTunnelDataType dataType, - org.apache.seatunnel.api.sink.SinkWriter.Context context) { - this.context = context; + MetricsContext metricsContext) { this.sinkWriter = sinkWriter; this.checkpointId = checkpointId; - MetricsContext metricsContext = context.getMetricsContext(); this.sinkWriteCount = metricsContext.counter(MetricNames.SINK_WRITE_COUNT); this.sinkWriteBytes = metricsContext.counter(MetricNames.SINK_WRITE_BYTES); this.sinkWriterQPS = metricsContext.meter(MetricNames.SINK_WRITE_QPS); @@ -123,7 +118,6 @@ public List> snapshotState() throws IOException { @Override public void close() throws Exception { sinkWriter.close(); - context.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java index 7d8052bfd18..e457d69f27c 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceEnumerator.java @@ -19,8 +19,6 @@ import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.source.event.EnumeratorCloseEvent; -import org.apache.seatunnel.api.source.event.EnumeratorOpenEvent; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; @@ -51,7 +49,6 @@ public class FlinkSourceEnumerator private final SplitEnumeratorContext> enumeratorContext; - private final SourceSplitEnumerator.Context context; private final int parallelism; private final Object lock = new Object(); @@ -65,14 +62,12 @@ public FlinkSourceEnumerator( SplitEnumeratorContext> enumContext) { this.sourceSplitEnumerator = enumerator; this.enumeratorContext = enumContext; - this.context = new FlinkSourceSplitEnumeratorContext<>(enumeratorContext); this.parallelism = enumeratorContext.currentParallelism(); } @Override public void start() { sourceSplitEnumerator.open(); - context.getEventListener().onEvent(new EnumeratorOpenEvent()); } @Override @@ -111,7 +106,6 @@ public EnumStateT snapshotState(long checkpointId) throws Exception { @Override public void close() throws IOException { sourceSplitEnumerator.close(); - context.getEventListener().onEvent(new EnumeratorCloseEvent()); } @Override diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java index fb1dc85174e..c2f9cde5005 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/source/FlinkSourceReader.java @@ -20,8 +20,6 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.apache.seatunnel.api.source.SourceSplit; -import org.apache.seatunnel.api.source.event.ReaderCloseEvent; -import org.apache.seatunnel.api.source.event.ReaderOpenEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.flink.api.connector.source.ReaderOutput; @@ -68,7 +66,6 @@ public FlinkSourceReader( public void start() { try { sourceReader.open(); - context.getEventListener().onEvent(new ReaderOpenEvent()); } catch (Exception e) { throw new RuntimeException(e); } @@ -124,7 +121,6 @@ public void handleSourceEvents(SourceEvent sourceEvent) { @Override public void close() throws Exception { sourceReader.close(); - context.getEventListener().onEvent(new ReaderCloseEvent()); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java index a9eac500629..434b1ef9799 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriter.java @@ -21,7 +21,6 @@ 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.event.WriterCloseEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; @@ -48,19 +47,16 @@ public class SparkDataWriter implements DataWriter sinkWriter, @Nullable SinkCommitter sinkCommitter, MultiTableManager multiTableManager, - long epochId, - org.apache.seatunnel.api.sink.SinkWriter.Context context) { + long epochId) { this.sinkWriter = sinkWriter; this.sinkCommitter = sinkCommitter; this.epochId = epochId == 0 ? 1 : epochId; this.multiTableManager = multiTableManager; - this.context = context; initResourceManger(); } @@ -101,7 +97,6 @@ public WriterCommitMessage commit() throws IOException { new SparkWriterCommitMessage<>(latestCommitInfoT); cleanCommitInfo(); sinkWriter.close(); - context.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java index b684654103a..3a646f3aca2 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/sink/writer/SparkDataWriterFactory.java @@ -63,6 +63,6 @@ public DataWriter createDataWriter(int partitionId, long taskId, lo throw new RuntimeException("Failed to create SinkCommitter.", e); } return new SparkDataWriter<>( - writer, committer, new MultiTableManager(catalogTables), epochId, context); + writer, committer, new MultiTableManager(catalogTables), epochId); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java index c2c24aa9147..59f931e38f1 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java @@ -21,7 +21,6 @@ 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.event.WriterCloseEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; @@ -48,19 +47,16 @@ public class SeaTunnelSparkDataWriter implements DataWriter protected volatile MultiTableResourceManager resourceManager; private final MultiTableManager multiTableManager; - private final SinkWriter.Context context; public SeaTunnelSparkDataWriter( SinkWriter sinkWriter, @Nullable SinkCommitter sinkCommitter, MultiTableManager multiTableManager, - long epochId, - SinkWriter.Context context) { + long epochId) { this.sinkWriter = sinkWriter; this.sinkCommitter = sinkCommitter; this.multiTableManager = multiTableManager; this.epochId = epochId == 0 ? 1 : epochId; - this.context = context; initResourceManger(); } @@ -93,7 +89,6 @@ public WriterCommitMessage commit() throws IOException { new SeaTunnelSparkWriterCommitMessage<>(latestCommitInfoT); cleanCommitInfo(); sinkWriter.close(); - context.getEventListener().onEvent(new WriterCloseEvent()); try { if (resourceManager != null) { resourceManager.close(); diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java index 255a9cd339f..b83787cac1e 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java @@ -64,7 +64,7 @@ public DataWriter createWriter(int partitionId, long taskId) { throw new RuntimeException("Failed to create SinkCommitter.", e); } return new SeaTunnelSparkDataWriter<>( - writer, committer, new MultiTableManager(catalogTables), 0, context); + writer, committer, new MultiTableManager(catalogTables), 0); } @Override diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index c2f431836b6..7f95aaf01c4 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -46,10 +46,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.8-SNAPSHOT-optional.jar -protobuf-java-util-3.25.3.jar -protobuf-java-3.25.3.jar -protoc-jar-3.11.4.jar -error_prone_annotations-2.18.0.jar -gson-2.8.9.jar -j2objc-annotations-2.8.jar +seatunnel-janino-2.3.8-SNAPSHOT-optional.jar \ No newline at end of file From a5c6d8970f0c75dd31ca0205ba1d6e38f1e7e06e Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 22:47:29 +0800 Subject: [PATCH 14/19] Revert "init" This reverts commit c7b60632a99da9c142d2c8d7220b5496c4b4b782. --- .../seatunnel/connectors/seatunnel/hbase/sink/HbaseSink.java | 5 +++++ .../connectors/seatunnel/hbase/sink/HbaseSinkFactory.java | 5 +++++ .../connectors/seatunnel/hbase/sink/HbaseSinkWriter.java | 1 + 3 files changed, 11 insertions(+) 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 14f8ec8c7ef..a3e8b69471b 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 @@ -23,6 +23,8 @@ import org.apache.seatunnel.api.sink.SaveModeHandler; import org.apache.seatunnel.api.sink.SchemaSaveMode; import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.shade.com.typesafe.config.Config; + import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSink; import org.apache.seatunnel.api.sink.SupportSaveMode; @@ -30,9 +32,12 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.sink.SupportMultiTableSink; +import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseConfig; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; import org.apache.seatunnel.connectors.seatunnel.hbase.state.HbaseAggregatedCommitInfo; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java index 9fab21954a7..3c6896455a0 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkFactory.java @@ -21,10 +21,15 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.sink.SinkCommonOptions; import org.apache.seatunnel.api.table.connector.TableSink; +import org.apache.seatunnel.api.sink.SinkCommonOptions; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; import org.apache.seatunnel.connectors.seatunnel.hbase.constant.HbaseIdentifier; +import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext; +import org.apache.seatunnel.connectors.seatunnel.hbase.config.HbaseParameters; import com.google.auto.service.AutoService; diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java index a41a2386297..6278d42ff1f 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/sink/HbaseSinkWriter.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; +import org.apache.seatunnel.api.sink.SupportMultiTableSinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; From 9a044a7d18ffa022d7a537515ada76e1257d9079 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 22:47:32 +0800 Subject: [PATCH 15/19] Revert "init" This reverts commit 65e75c40ba56975cca9cae30d538e3240ce5c0aa. --- .../connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java index b9a3fc25fd4..6894fba17fc 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalogFactory.java @@ -34,7 +34,7 @@ public class HbaseCatalogFactory implements CatalogFactory { public Catalog createCatalog(String catalogName, ReadonlyConfig options) { // Create an instance of HbaseCatalog, passing in the catalog name, namespace, and Hbase // parameters - HbaseParameters hbaseParameters = HbaseParameters.buildWithConfig(options); + HbaseParameters hbaseParameters = HbaseParameters.buildWithSinkConfig(options.toConfig()); return new HbaseCatalog(catalogName, hbaseParameters.getNamespace(), hbaseParameters); } From 50e8936f37fe3bef02bf3e457301f0cf044b41bd Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 22:59:16 +0800 Subject: [PATCH 16/19] improve some problem --- docs/sidebars.js | 24 +++++++++++++++--- .../src/test/resources/json/e2e.json.lzo | Bin 3466 -> 0 bytes .../src/test/resources/text/e2e.txt.lzo | Bin 2720 -> 0 bytes 3 files changed, 21 insertions(+), 3 deletions(-) delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/json/e2e.json.lzo delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/text/e2e.txt.lzo diff --git a/docs/sidebars.js b/docs/sidebars.js index 6061df55084..9f717d28aa2 100644 --- a/docs/sidebars.js +++ b/docs/sidebars.js @@ -84,7 +84,7 @@ const sidebars = { }, { "type": "category", - "label": "Concept", + "label": "Concepts", "items": [ "concept/config", "concept/connector-v2-features", @@ -106,7 +106,7 @@ const sidebars = { "link": { "type": "generated-index", "title": "Source(V2) of SeaTunnel", - "description": "List all source(v2) supported Apache SeaTunnel for now.", + "description": "List all source(v2) supported by Apache SeaTunnel for now.", "slug": "/connector-v2/source", "keywords": ["source"], "image": "/img/favicon.ico" @@ -124,7 +124,7 @@ const sidebars = { "link": { "type": "generated-index", "title": "Sink(V2) of SeaTunnel", - "description": "List all sink(v2) supported Apache SeaTunnel for now.", + "description": "List all sink(v2) supported by Apache SeaTunnel for now.", "slug": "/connector-v2/sink", "keywords": ["sink"], "image": "/img/favicon.ico" @@ -136,6 +136,24 @@ const sidebars = { } ] }, + { + "type": "category", + "label": "Formats", + "link": { + "type": "generated-index", + "title": "Formats", + "description": "List some special formats (not all) supported by Apache SeaTunnel for now.", + "slug": "/connector-v2/formats", + "keywords": ["formats"], + "image": "/img/favicon.ico" + }, + "items": [ + { + "type": "autogenerated", + "dirName": "connector-v2/formats" + } + ] + }, "connector-v2/source-common-options", "connector-v2/sink-common-options", "connector-v2/Error-Quick-Reference-Manual", diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/json/e2e.json.lzo b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/json/e2e.json.lzo deleted file mode 100644 index 3c56a9efc030e2795c2a1411fe0c36d88b4754ba..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3466 zcmY*cOK%(38Kq=dc8qA8#z~MAAmC6=q{u`1zUSUi(uN`_k)lLW)KITDP$`mnP!#nd zEm46B7x2PG6hI3tgqCf+Z~>ulkrv3pi!QS0B8#G%ZrY+i1GI}Y%^&EZK+jcx0x=AD z&T!`5JLfy!IiG(#wUiGHT>I{|7hbqoc<$Yv%b|-uKYFtImmh{NE~5`G{XGPN-mI(@W`I*W*=!NGzE)gah}?Iva(QpB+==+_Y3+HQqJ+3qan86FTDpLS01@pf zLKGF!V=fiiGadtY-!8D<+S_QZVbS^cUTO@RRF+q^w;Q!`Gvcq78~ZhTr?t`SY&2UD zKWfOuOIzhe4xA;L*|BO@h0d0$Ex_VL zV`~CGFSMsSDd5e$*~}jBc%g0(iYg|SXM^phJ;-FLo8aeYt=(J#-o2MDc7V6n=Vq{D zWPWw5unn9#DpkN;q*GX*8wcKP?~hf0$LCinnQylq)!fsJGG$z&iN}QG#6{0Zv;m6f za@5o)f}WrP2gYYm{q5G9nwX9sJecu+>8RtK;U`WnlbSGGT*gNdFiW4-|Jil~US@(Y z4n086Mn3ExdgAmEsugAO*dIQ3uJai2ghEfE*R^eZ6?^LR&9R+&ZA*w>JM__OLrBQL z9E@jxch@#21K`G9VZ*k@WU*Rb1wPs;Zdl^myZNjoI+sc(CxKf>lPe_`D@L#zgo0Cm zG#G_2oKTl~1Ul0ftPzNS!pDefY%43F-#=}B-R_->SF>vm?hbw8wD#KH|JV^yJiL3l zcZH(4;W37*G_4)CU7>HT1)~L<4YM!?H6(eJlt2iew7Gy*BXT=0w4E zO}@C>UX5bK*KuHO&|e;ClM?n1|8sG81Gx#0yM`$Om)G2dWN>C&yM=IXkgMdzyYKnG zaty~hJxVZxPwUr)z@`nOBj=9eX{})ah6tlUiY9M8cCL6aO_dff;?obl{K^584C0Xt zWusE_*qPH;Nv|I*&Bf^%7LF2m_Uh0X#D?b8ib1pz;bc#RI>rnFi77OW$;8&#iw!qVZ;XeYB;v~9Q+jHRYP!*;2evFe#G8e14Y$gEP}_C8s30fGpfNlflmv3Jq6dR$QYZv>8QjI3{kz|zo`%)J zf4ffaLHS;C_4GaSGe?~Kb!0s3iIe8DR=+anAn9>Bk`6lxjV-_x{(*fe#y>s${HuN* z<7ju3DSGA%n3#bml`@1$Lp|@${%|sUso0#`E|S3~x9*IGiFxpvrR+sbIT8yUiZW_~ z?81UEDo{kmsRZfJ;j@qKoH|!2jS>|Q=EF1I1!ei!bjA)90hy-hqpN@WS3lAp{v?@t;1745t2~Bh;BzqA!SrB{M3Qv--~94+-~40n1n<$MTC7**b4W=!wtgi7ivc?Wewsn6|M(Vur>dian z&Q*fvWk9u06r3(JT297~viRuz=}dkDB;;m0$pzpbJG*T&VRGtlJpr8CsmIO5KvlZuG_&rov#Nr z9a6RN{0wkmc4xwd=3?t`@CR<_Qan))?2Nt`Z_Ll(_go`ipNTzoLN}3*R@I;kvWLoW zd>7FUx1(rQX{<$~ac!=`SGZBiJ#%^s&CSEw_kQZ2oP*EYp>((hF<^#{{VR|oI)uP# zZa;Pg&@P;#vEf@9eJ`JU@LBs3tPRV5F$!)R6Js1%h>-#lE52UbdVCAs)?aJxv`Y({ zc1#~R_Ip4B{j~Q@Q1CMTH(-!eI4VY4j$Ls9$wH+VUr=;ngB~G!><=JGSwK8r$ZsaP K?|k!LuKxi@%3&M; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/text/e2e.txt.lzo b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/text/e2e.txt.lzo deleted file mode 100644 index a118d2c1b7c4c67b4352f68b60dfb7b9e617fab4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2720 zcmXYzxo>M(9mnrI=Q4mbBMsf?Js~1d$2$9Bq0x(dy(Et9I9?LF8pmtmC9(ay7Dz}_ zyhwDk6jQu{;uVw>&{4bsAzGx7_y-W8M1q3v%}f@_xpwY7=evA<=lQZ*3qDJ|N`3zM z$<>eFgg*#gzJB$7^!H!>^yTI0*YqDjkopsT7s0>lWjohssua9()A8cbXzMhYJbXYhPN}*QTIBm5jTw5c&v-0JoHOe?^376I@ zBaKo-Fnodwp+#6}#*1D!>dn@}P|A=?r%cE_^Gb3rL}-ms+*u(6W0req7z;&OS?`om zdFeu*)g_F)IC-RcnaN9pG7DB;Yi5!|~m*_#J zmD#vP{`WfgLNFq%;7T|qjCCftn{zg8lu(L8UB(%%2rYm2N$_9JP6az*GNXKkiB$dk z>3F2wYGHawc4yb!m^KExL6#h+!*Pxd&ac}YawW4w$V&#})1Ow5UkY9 zYfB38q`CfM9Q?u?B9s-LNZbMkpA- zrct@1A{?afUVs|jaEoZ6Cx`~N!{M`3Fxf~2iGYcRUb(UifBRMNnrUn=1Xs9)Q4BP) z@B;#%Z3Y@jh^akSGAYrwae@u*PQVQ2Gr-oWS1e|uMQL5#(Mq>GoJZ3_e$}FP!$I{S zYRqQUJ}uYUtztB2KHc_dsn}>gMH75JPn>}sk|}^8X$d;+s7i2&kT?WGdH@k7QVyfg z35DKYfpu#Dy&_apmr45>46~Zf>1L6FCo`WUCLrs{;Pg@n5pH6S@q$6p2wnj@uPVYWG%+`*F9Ne~PI5n_5kL}Pr2`6MOC1BD=` zO;Q2g7%t6+=io;|Vyp;ZU-*h0zTl!$yWfq!`7(H&z))dG0g-5igkV?|@TGOy3JEWq z#cNaJg}Z@TZh)= z$LHYZn&5+=hgQ-Fhv^3=Ie;$`tT>2+aR@3S571A)z=EgW&w^xt8GIv5FKM_f49laB zamt^fi4qS4pzPPb34VxPfHyJ?{ge_C^aepZ8ust!;K`hFeZp-9anLOFc=xt*r01nt zvrN`!+4fs{zsi^Mq|#}19_ih>GmFVZzi{_N>#feVNcOpYc}5rOUA`DPJO&0aMnDTC z7}vxhc9`Fu5pIo!0q`3q4V{>T#f1C*q`&y)uR+S`5O7DOaRk87#efVERtySj{ku1x zO2?E#7@TqdkotZ6DO^mdhZ6!zH2QT+DvfC~Mi5?co&-UF8{9&4lnQ#{-VPF4OjC>` zDV-OnulV*|uuOsj33x_XNDz((xtnfODdrmo+FL!;ucF7N>?EeU@}hQ#;%%=L({a6) zkE83{eAJ?6z42%f#qrg$O&~dLj(I>aKX9rN3Znu-14$gC0@{Q89M^ya#2k8&;qKyi z`Gl+5k=5o2#Sk>SriIe!TBJKc?77q5AkQ%4R(n$TQYSWbC99Y z+}1W9V#vU7Qc7;=CDNaW3OZnb7?wsV@dODEG?0R+{?9n&l6fIyM%#>ZKl$z;`hREn BZzKQ! From 3a67d705777a9549bae1dbccce8d1ac9352eb91d Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Fri, 6 Sep 2024 23:00:54 +0800 Subject: [PATCH 17/19] improve some problem --- seatunnel-connectors-v2/connector-hbase/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-hbase/pom.xml b/seatunnel-connectors-v2/connector-hbase/pom.xml index b2ca1849911..bda49ade0ec 100644 --- a/seatunnel-connectors-v2/connector-hbase/pom.xml +++ b/seatunnel-connectors-v2/connector-hbase/pom.xml @@ -39,7 +39,6 @@ org.apache.seatunnel connector-common ${project.version} - compile From c8a1aaebf6413126c40cc3e25435fc2aa5038166 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Sat, 7 Sep 2024 00:18:45 +0800 Subject: [PATCH 18/19] improve some problem --- .../connectors/seatunnel/hbase/client/HbaseClient.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java index c3944772c92..aec64bf7cf4 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/client/HbaseClient.java @@ -309,9 +309,7 @@ public void truncateTable(String databaseName, String tableName) { */ public boolean isExistsData(String databaseName, String tableName) { try { - Table table = - connection.getTable( - TableName.valueOf(databaseName, tableName)); + Table table = connection.getTable(TableName.valueOf(databaseName, tableName)); Scan scan = new Scan(); scan.setCaching(1); scan.setLimit(1); From 9bacca7fe765d1191a8043ba0a0f67544cc03f03 Mon Sep 17 00:00:00 2001 From: zhangshenghang Date: Sat, 7 Sep 2024 15:52:22 +0800 Subject: [PATCH 19/19] improve some problem --- .../seatunnel/hbase/catalog/HbaseCatalog.java | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java index 6659f313552..f6a48150732 100644 --- a/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java +++ b/seatunnel-connectors-v2/connector-hbase/src/main/java/org/apache/seatunnel/connectors/seatunnel/hbase/catalog/HbaseCatalog.java @@ -22,9 +22,7 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.InfoPreviewResult; import org.apache.seatunnel.api.table.catalog.PreviewResult; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.catalog.exception.CatalogException; import org.apache.seatunnel.api.table.catalog.exception.DatabaseAlreadyExistException; import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; @@ -35,7 +33,6 @@ import lombok.extern.slf4j.Slf4j; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -113,16 +110,7 @@ public boolean tableExists(TablePath tablePath) throws CatalogException { @Override public CatalogTable getTable(TablePath tablePath) throws CatalogException, TableNotExistException { - checkNotNull(tablePath, "tablePath cannot be null"); - // Hbase cannot retrieve the columns of the table - return CatalogTable.of( - TableIdentifier.of( - catalogName, tablePath.getDatabaseName(), tablePath.getTableName()), - // Hbase cannot obtain column names, so TableSchema was directly built here - TableSchema.builder().build(), - buildTableOptions(tablePath), - Collections.emptyList(), - ""); + throw new UnsupportedOperationException("Not implement"); } @Override