hailin0 commented on code in PR #7158: URL: https://github.com/apache/seatunnel/pull/7158#discussion_r1672366446
########## seatunnel-connectors-v2/connector-milvus/src/main/java/org/apache/seatunnel/connectors/seatunnel/milvus/sink/MilvusSinkWriter.java: ########## @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.milvus.sink; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.SinkCommitter; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.milvus.config.MilvusSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.milvus.sink.batch.MilvusBatchWriter; +import org.apache.seatunnel.connectors.seatunnel.milvus.sink.batch.MilvusBufferBatchWriter; +import org.apache.seatunnel.connectors.seatunnel.milvus.state.MilvusCommitInfo; +import org.apache.seatunnel.connectors.seatunnel.milvus.state.MilvusSinkState; + +import io.milvus.v2.client.ConnectConfig; +import io.milvus.v2.client.MilvusClientV2; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.milvus.config.MilvusSinkConfig.BATCH_SIZE; + +@Slf4j +/** MilvusSinkWriter is a sink writer that will write {@link SeaTunnelRow} to Milvus. */ +public class MilvusSinkWriter + implements SinkWriter<SeaTunnelRow, MilvusCommitInfo, MilvusSinkState> { + private final Context context; + + private final ReadonlyConfig config; + private MilvusBatchWriter batchWriter; + + public MilvusSinkWriter( + Context context, + CatalogTable catalogTable, + ReadonlyConfig config, + List<MilvusSinkState> milvusSinkStates) { + this.context = context; + this.config = config; + ConnectConfig connectConfig = + ConnectConfig.builder() + .uri(config.get(MilvusSinkConfig.URL)) + .token(config.get(MilvusSinkConfig.TOKEN)) + .build(); + this.batchWriter = + new MilvusBufferBatchWriter( + catalogTable, + config.get(BATCH_SIZE), + getAutoId(catalogTable.getTableSchema().getPrimaryKey()), + config.get(MilvusSinkConfig.ENABLE_UPSERT), + new MilvusClientV2(connectConfig)); + } + + /** + * write data to third party data receiver. + * + * @param element the data need be written. + * @throws IOException throw IOException when write data failed. + */ + @Override + public void write(SeaTunnelRow element) { + batchWriter.addToBatch(element); + if (batchWriter.needFlush()) { + batchWriter.flush(); + } + } + + private Boolean getAutoId(PrimaryKey primaryKey) { + if (null != primaryKey && null != primaryKey.getEnableAutoId()) { + return primaryKey.getEnableAutoId(); + } else { + return config.get(MilvusSinkConfig.ENABLE_AUTO_ID); + } + } + + /** + * prepare the commit, will be called before {@link #snapshotState(long checkpointId)}. If you + * need to use 2pc, you can return the commit info in this method, and receive the commit info + * in {@link SinkCommitter#commit(List)}. If this method failed (by throw exception), **Only** + * Spark engine will call {@link #abortPrepare()} + * + * @return the commit info need to commit + */ + @Override + public Optional<MilvusCommitInfo> prepareCommit() throws IOException { + return Optional.empty(); Review Comment: call `batchWriter.flush();`? reference https://github.com/apache/seatunnel/blob/dbdbdf015bceb90767787be465399c00b7baa167/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java#L133 ########## seatunnel-connectors-v2/connector-milvus/src/main/java/org/apache/seatunnel/connectors/seatunnel/milvus/catalog/MilvusCatalog.java: ########## @@ -0,0 +1,402 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.milvus.catalog; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.InfoPreviewResult; +import org.apache.seatunnel.api.table.catalog.PreviewResult; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.VectorIndex; +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.type.ArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.connectors.seatunnel.milvus.config.MilvusSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.milvus.exception.MilvusConnectionErrorCode; +import org.apache.seatunnel.connectors.seatunnel.milvus.exception.MilvusConnectorException; + +import org.apache.commons.collections4.CollectionUtils; + +import io.milvus.client.MilvusServiceClient; +import io.milvus.common.clientenum.ConsistencyLevelEnum; +import io.milvus.grpc.DataType; +import io.milvus.grpc.ListDatabasesResponse; +import io.milvus.grpc.ShowCollectionsResponse; +import io.milvus.grpc.ShowType; +import io.milvus.param.ConnectParam; +import io.milvus.param.IndexType; +import io.milvus.param.MetricType; +import io.milvus.param.R; +import io.milvus.param.RpcStatus; +import io.milvus.param.collection.CreateCollectionParam; +import io.milvus.param.collection.CreateDatabaseParam; +import io.milvus.param.collection.DropCollectionParam; +import io.milvus.param.collection.DropDatabaseParam; +import io.milvus.param.collection.FieldType; +import io.milvus.param.collection.HasCollectionParam; +import io.milvus.param.collection.ShowCollectionsParam; +import io.milvus.param.index.CreateIndexParam; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkNotNull; + +@Slf4j +public class MilvusCatalog implements Catalog { + + private final String catalogName; + private final ReadonlyConfig config; + + private MilvusServiceClient client; + + public MilvusCatalog(String catalogName, ReadonlyConfig config) { + this.catalogName = catalogName; + this.config = config; + } + + @Override + public void open() throws CatalogException { + ConnectParam connectParam = + ConnectParam.newBuilder() + .withUri(config.get(MilvusSinkConfig.URL)) + .withToken(config.get(MilvusSinkConfig.TOKEN)) + .build(); + try { + this.client = new MilvusServiceClient(connectParam); + } catch (Exception e) { + throw new CatalogException(String.format("Failed to open catalog %s", catalogName), e); + } + } + + @Override + public void close() throws CatalogException { + this.client.close(); + } + + @Override + public String name() { + return catalogName; + } + + @Override + public PreviewResult previewAction( + ActionType actionType, TablePath tablePath, Optional<CatalogTable> catalogTable) { + if (actionType == ActionType.CREATE_TABLE) { + return new InfoPreviewResult("create collection " + tablePath.getTableName()); + } else if (actionType == ActionType.DROP_TABLE) { + return new InfoPreviewResult("drop collection " + tablePath.getTableName()); + } else if (actionType == ActionType.CREATE_DATABASE) { + return new InfoPreviewResult("create database " + tablePath.getDatabaseName()); + } else if (actionType == ActionType.DROP_DATABASE) { + return new InfoPreviewResult("drop database " + tablePath.getDatabaseName()); + } else { + throw new UnsupportedOperationException("Unsupported action type: " + actionType); + } + } + + @Override + public String getDefaultDatabase() throws CatalogException { + return "default"; + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + List<String> databases = this.listDatabases(); + return databases.contains(databaseName); + } + + @Override + public List<String> listDatabases() throws CatalogException { + R<ListDatabasesResponse> response = this.client.listDatabases(); + return response.getData().getDbNamesList(); + } + + @Override + public List<String> listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + R<ShowCollectionsResponse> response = + this.client.showCollections( + ShowCollectionsParam.newBuilder() + .withDatabaseName(databaseName) + .withShowType(ShowType.All) + .build()); + + return response.getData().getCollectionNamesList(); + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + R<Boolean> response = + this.client.hasCollection( + HasCollectionParam.newBuilder() + .withDatabaseName(tablePath.getDatabaseName()) + .withCollectionName(tablePath.getTableName()) + .build()); + if (response.getData() != null) { + return response.getData(); + } + throw new MilvusConnectorException( + MilvusConnectionErrorCode.SERVER_RESPONSE_FAILED, + response.getMessage(), + response.getException()); + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + throw new RuntimeException("not implemented"); + } + + @Override + public void createTable(TablePath tablePath, CatalogTable catalogTable, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "Table path cannot be null"); + if (!databaseExists(tablePath.getDatabaseName())) { + throw new DatabaseNotExistException(catalogName, tablePath.getDatabaseName()); + } + if (tableExists(tablePath)) { + if (ignoreIfExists) { + return; + } + throw new TableAlreadyExistException(catalogName, tablePath); + } + + checkNotNull(catalogTable, "catalogTable must not be null"); + TableSchema tableSchema = catalogTable.getTableSchema(); + checkNotNull(tableSchema, "tableSchema must not be null"); + createTableInternal(tablePath, catalogTable); + + if (CollectionUtils.isNotEmpty(tableSchema.getVectorIndexes())) { + createIndexInternal(tablePath, tableSchema.getVectorIndexes()); + } + } + + private void createIndexInternal(TablePath tablePath, List<VectorIndex> vectorIndexes) { + for (VectorIndex index : vectorIndexes) { + CreateIndexParam createIndexParam = + CreateIndexParam.newBuilder() + .withDatabaseName(tablePath.getDatabaseName()) + .withCollectionName(tablePath.getTableName()) + .withFieldName(index.getFieldName()) + .withIndexName(index.getIndexName()) + .withIndexType(IndexType.valueOf(index.getIndexType())) + .withMetricType(MetricType.valueOf(index.getMetricType())) + .build(); + + client.createIndex(createIndexParam); + } + } + + public void createTableInternal(TablePath tablePath, CatalogTable catalogTable) { + try { + TableSchema tableSchema = catalogTable.getTableSchema(); + List<FieldType> fieldTypes = new ArrayList<>(); + for (Column column : tableSchema.getColumns()) { + fieldTypes.add(convertToFieldType(column, tableSchema.getPrimaryKey())); + } + + Boolean enableDynamicField = + (null != tableSchema.getEnableDynamicField()) + ? tableSchema.getEnableDynamicField() + : config.get(MilvusSinkConfig.ENABLE_DYNAMIC_FIELD); + + CreateCollectionParam.Builder builder = + CreateCollectionParam.newBuilder() + .withDatabaseName(tablePath.getDatabaseName()) + .withCollectionName(tablePath.getTableName()) + .withFieldTypes(fieldTypes) + .withEnableDynamicField(enableDynamicField) + .withConsistencyLevel(ConsistencyLevelEnum.BOUNDED); + if (null != catalogTable.getComment()) { + builder.withDescription(catalogTable.getComment()); + } + + CreateCollectionParam createCollectionParam = builder.build(); + R<RpcStatus> response = this.client.createCollection(createCollectionParam); + if (!Objects.equals(response.getStatus(), R.success().getStatus())) { + throw new CatalogException( + String.format("Create collection failed, err=%s", response.getMessage()), + response.getException()); + } + } catch (Exception e) { + throw new CatalogException( + String.format("Failed creating collection %s", tablePath.getFullName()), e); + } + } + + private FieldType convertToFieldType(Column column, PrimaryKey primaryKey) { Review Comment: create `MilvustypeConverter.java` move `convertToFieldType` and `convertToDataType` into `MilvustypeConverter` reference https://github.com/apache/seatunnel/blob/dev/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresTypeConverter.java ########## seatunnel-connectors-v2/connector-milvus/src/main/java/org/apache/seatunnel/connectors/seatunnel/milvus/sink/batch/MilvusBufferBatchWriter.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.connectors.seatunnel.milvus.sink.batch; + +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.common.utils.SeaTunnelException; +import org.apache.seatunnel.connectors.seatunnel.milvus.convert.MilvusConvertUtils; +import org.apache.seatunnel.connectors.seatunnel.milvus.exception.MilvusConnectionErrorCode; +import org.apache.seatunnel.connectors.seatunnel.milvus.exception.MilvusConnectorException; + +import org.apache.commons.collections4.CollectionUtils; + +import com.alibaba.fastjson.JSONObject; +import io.milvus.v2.client.MilvusClientV2; +import io.milvus.v2.service.vector.request.InsertReq; +import io.milvus.v2.service.vector.request.UpsertReq; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.seatunnel.api.table.catalog.PrimaryKey.isPrimaryKeyField; + +public class MilvusBufferBatchWriter implements MilvusBatchWriter { + + private final int batchSize; + private final CatalogTable catalogTable; + private final Boolean autoId; + private final Boolean enableUpsert; + private final String collectionName; + private MilvusClientV2 milvusClient; + + private volatile List<JSONObject> milvusDataCache; + private volatile int writeCount = 0; + + public MilvusBufferBatchWriter( + CatalogTable catalogTable, + Integer batchSize, + Boolean autoId, + Boolean enableUpsert, + MilvusClientV2 milvusClient) { + this.catalogTable = catalogTable; + this.autoId = autoId; + this.enableUpsert = enableUpsert; + this.milvusClient = milvusClient; + this.collectionName = catalogTable.getTablePath().getTableName(); + this.batchSize = batchSize; + this.milvusDataCache = new ArrayList<>(batchSize); + } + + @Override + public void addToBatch(SeaTunnelRow element) { + JSONObject data = buildMilvusData(element); + milvusDataCache.add(data); + writeCount++; + } + + @Override + public boolean needFlush() { + return this.writeCount >= this.batchSize; + } + + @Override + public boolean flush() { Review Comment: ```suggestion public synchronized boolean flush() { ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@seatunnel.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org