From c29b60e18e3d3d42ead4f9fd1c11397fba7731ad Mon Sep 17 00:00:00 2001 From: "cai.zhang" Date: Mon, 27 Nov 2023 13:50:27 +0800 Subject: [PATCH] enhance: Support Array DataType for bulk_insert (#28341) issue: #28272 Support array DataType for bulk_insert with json, binlog files. Signed-off-by: Cai Zhang --- internal/util/importutil/binlog_adapter.go | 231 +++++++++------- .../util/importutil/binlog_adapter_test.go | 83 +++++- internal/util/importutil/binlog_file.go | 173 +++++++----- internal/util/importutil/binlog_file_test.go | 108 ++++++++ internal/util/importutil/binlog_parser.go | 26 +- .../util/importutil/binlog_parser_test.go | 2 + internal/util/importutil/collection_info.go | 13 +- internal/util/importutil/csv_handler.go | 86 +++--- internal/util/importutil/csv_handler_test.go | 166 +++++++++++ internal/util/importutil/csv_parser.go | 30 +- internal/util/importutil/csv_parser_test.go | 20 +- internal/util/importutil/csv_reader.go | 18 +- internal/util/importutil/import_options.go | 5 +- internal/util/importutil/import_util.go | 260 +++++++++++++++--- internal/util/importutil/import_util_test.go | 251 ++++++++++++++++- internal/util/importutil/import_wrapper.go | 33 +-- .../util/importutil/import_wrapper_test.go | 33 +-- internal/util/importutil/json_handler.go | 50 ++-- internal/util/importutil/json_parser.go | 42 +-- internal/util/importutil/json_parser_test.go | 12 + internal/util/importutil/numpy_adapter.go | 74 ++--- internal/util/importutil/numpy_parser.go | 132 ++++----- internal/util/importutil/numpy_parser_test.go | 22 +- pkg/util/merr/errors.go | 3 + pkg/util/merr/utils.go | 8 + 25 files changed, 1399 insertions(+), 482 deletions(-) diff --git a/internal/util/importutil/binlog_adapter.go b/internal/util/importutil/binlog_adapter.go index 2e8171fbf82bd..64bd2b39b0b29 100644 --- a/internal/util/importutil/binlog_adapter.go +++ b/internal/util/importutil/binlog_adapter.go @@ -23,24 +23,24 @@ import ( "strconv" "strings" - "github.com/cockroachdb/errors" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) -// A struct to hold insert log paths and delta log paths of a segment +// SegmentFilesHolder A struct to hold insert log paths and delta log paths of a segment type SegmentFilesHolder struct { segmentID int64 // id of the segment fieldFiles map[storage.FieldID][]string // mapping of field id and data file path deltaFiles []string // a list of delta log file path, typically has only one item } -// Adapter class to process insertlog/deltalog of a backuped segment +// BinlogAdapter Adapter class to process insertlog/deltalog of a backuped segment // This class do the following works: // 1. read insert log of each field, then constructs SegmentData in memory. // 2. read delta log to remove deleted entities(TimeStampField is used to apply or skip the operation). @@ -78,17 +78,17 @@ func NewBinlogAdapter(ctx context.Context, ) (*BinlogAdapter, error) { if collectionInfo == nil { log.Warn("Binlog adapter: collection schema is nil") - return nil, errors.New("collection schema is nil") + return nil, merr.WrapErrImportFailed("collection schema is nil") } if chunkManager == nil { log.Warn("Binlog adapter: chunk manager pointer is nil") - return nil, errors.New("chunk manager pointer is nil") + return nil, merr.WrapErrImportFailed("chunk manager pointer is nil") } if flushFunc == nil { log.Warn("Binlog adapter: flush function is nil") - return nil, errors.New("flush function is nil") + return nil, merr.WrapErrImportFailed("flush function is nil") } adapter := &BinlogAdapter{ @@ -113,7 +113,7 @@ func NewBinlogAdapter(ctx context.Context, func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error { if segmentHolder == nil { log.Warn("Binlog adapter: segment files holder is nil") - return errors.New("segment files holder is nil") + return merr.WrapErrImportFailed("segment files holder is nil") } log.Info("Binlog adapter: read segment", zap.Int64("segmentID", segmentHolder.segmentID)) @@ -149,7 +149,7 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error { shardData := initShardData(p.collectionInfo.Schema, p.collectionInfo.PartitionIDs) if shardData == nil { log.Warn("Binlog adapter: fail to initialize in-memory segment data", zap.Int("shardID", i)) - return fmt.Errorf("fail to initialize in-memory segment data for shard id %d", i) + return merr.WrapErrImportFailed(fmt.Sprintf("fail to initialize in-memory segment data for shard id %d", i)) } shardsData = append(shardsData, shardData) } @@ -157,7 +157,7 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error { // read binlog files batch by batch primaryKey := p.collectionInfo.PrimaryKey for i := 0; i < batchCount; i++ { - // batchFiles excludes the primary key field and the timestamp field + // batchFiles excludes the primary key field and the timestamp field. // timestamp field is used to compare the tsEndPoint to skip some rows, no need to pass old timestamp to new segment. // once a new segment generated, the timestamp field will be re-generated, too. batchFiles := make(map[storage.FieldID]string) @@ -201,7 +201,7 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error { } } else { log.Warn("Binlog adapter: unsupported primary key type", zap.Int("type", int(primaryKey.GetDataType()))) - return fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType()) + return merr.WrapErrImportFailed(fmt.Sprintf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType())) } // if shardList is empty, that means all the primary keys have been deleted(or skipped), no need to read other files @@ -214,7 +214,7 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error { // outside context might be canceled(service stop, or future enhancement for canceling import task) if isCanceled(p.ctx) { log.Warn("Binlog adapter: import task was canceled") - return errors.New("import task was canceled") + return merr.WrapErrImportFailed("import task was canceled") } err = p.readInsertlog(fieldID, file, shardsData, shardList) @@ -235,25 +235,25 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error { } // verify method verify the schema and binlog files -// 1. each field must has binlog file +// 1. each field must have binlog file // 2. binlog file count of each field must be equal // 3. the collectionSchema doesn't contain TimeStampField and RowIDField since the import_wrapper excludes them, -// but the segmentHolder.fieldFiles need to contains the two fields. +// but the segmentHolder.fieldFiles need to contain the two fields. func (p *BinlogAdapter) verify(segmentHolder *SegmentFilesHolder) error { if segmentHolder == nil { log.Warn("Binlog adapter: segment files holder is nil") - return errors.New("segment files holder is nil") + return merr.WrapErrImportFailed("segment files holder is nil") } firstFieldFileCount := 0 - // each field must has binlog file + // each field must have binlog file for i := 0; i < len(p.collectionInfo.Schema.Fields); i++ { schema := p.collectionInfo.Schema.Fields[i] files, ok := segmentHolder.fieldFiles[schema.FieldID] if !ok { log.Warn("Binlog adapter: a field has no binlog file", zap.Int64("fieldID", schema.FieldID)) - return fmt.Errorf("the field %d has no binlog file", schema.FieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("the field %d has no binlog file", schema.FieldID)) } if i == 0 { @@ -261,26 +261,26 @@ func (p *BinlogAdapter) verify(segmentHolder *SegmentFilesHolder) error { } } - // the segmentHolder.fieldFiles need to contains RowIDField + // the segmentHolder.fieldFiles need to contain RowIDField _, ok := segmentHolder.fieldFiles[common.RowIDField] if !ok { log.Warn("Binlog adapter: the binlog files of RowIDField is missed") - return errors.New("the binlog files of RowIDField is missed") + return merr.WrapErrImportFailed("the binlog files of RowIDField is missed") } - // the segmentHolder.fieldFiles need to contains TimeStampField + // the segmentHolder.fieldFiles need to contain TimeStampField _, ok = segmentHolder.fieldFiles[common.TimeStampField] if !ok { log.Warn("Binlog adapter: the binlog files of TimeStampField is missed") - return errors.New("the binlog files of TimeStampField is missed") + return merr.WrapErrImportFailed("the binlog files of TimeStampField is missed") } // binlog file count of each field must be equal for _, files := range segmentHolder.fieldFiles { if firstFieldFileCount != len(files) { log.Warn("Binlog adapter: file count of each field must be equal", zap.Int("firstFieldFileCount", firstFieldFileCount)) - return fmt.Errorf("binlog file count of each field must be equal, first field files count: %d, other field files count: %d", - firstFieldFileCount, len(files)) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog file count of each field must be equal, first field files count: %d, other field files count: %d", + firstFieldFileCount, len(files))) } } @@ -318,7 +318,7 @@ func (p *BinlogAdapter) readDeltalogs(segmentHolder *SegmentFilesHolder) (map[in return nil, deletedIDDict, nil } log.Warn("Binlog adapter: unsupported primary key type", zap.Int("type", int(primaryKey.GetDataType()))) - return nil, nil, fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType()) + return nil, nil, merr.WrapErrImportFailed(fmt.Sprintf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType())) } // decodeDeleteLogs decodes string array(read from delta log) to storage.DeleteLog array @@ -363,8 +363,8 @@ func (p *BinlogAdapter) decodeDeleteLogs(segmentHolder *SegmentFilesHolder) ([]* log.Warn("Binlog adapter: delta log data type is not equal to collection's primary key data type", zap.Int64("deltaDataType", deleteLogs[i].PkType), zap.Int64("pkDataType", int64(primaryKey.GetDataType()))) - return nil, fmt.Errorf("delta log data type %d is not equal to collection's primary key data type %d", - deleteLogs[i].PkType, primaryKey.GetDataType()) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("delta log data type %d is not equal to collection's primary key data type %d", + deleteLogs[i].PkType, primaryKey.GetDataType())) } } @@ -382,13 +382,13 @@ func (p *BinlogAdapter) decodeDeleteLog(deltaStr string) (*storage.DeleteLog, er splits := strings.Split(deltaStr, ",") if len(splits) != 2 { log.Warn("Binlog adapter: the format of deletion string is incorrect", zap.String("deltaStr", deltaStr)) - return nil, fmt.Errorf("the format of deletion string is incorrect, '%s' can not be split", deltaStr) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("the format of deletion string is incorrect, '%s' can not be split", deltaStr)) } pk, err := strconv.ParseInt(splits[0], 10, 64) if err != nil { log.Warn("Binlog adapter: failed to parse primary key of deletion string from old version", zap.String("deltaStr", deltaStr), zap.Error(err)) - return nil, fmt.Errorf("failed to parse primary key of deletion string '%s' from old version, error: %w", deltaStr, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key of deletion string '%s' from old version, error: %v", deltaStr, err)) } deleteLog.Pk = &storage.Int64PrimaryKey{ Value: pk, @@ -398,7 +398,7 @@ func (p *BinlogAdapter) decodeDeleteLog(deltaStr string) (*storage.DeleteLog, er if err != nil { log.Warn("Binlog adapter: failed to parse timestamp of deletion string from old version", zap.String("deltaStr", deltaStr), zap.Error(err)) - return nil, fmt.Errorf("failed to parse timestamp of deletion string '%s' from old version, error: %w", deltaStr, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse timestamp of deletion string '%s' from old version, error: %v", deltaStr, err)) } } @@ -411,13 +411,13 @@ func (p *BinlogAdapter) readDeltalog(logPath string) ([]string, error) { binlogFile, err := NewBinlogFile(p.chunkManager) if err != nil { log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) - return nil, fmt.Errorf("failed to initialize binlog file '%s', error: %w", logPath, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize binlog file '%s', error: %v", logPath, err)) } err = binlogFile.Open(logPath) if err != nil { log.Warn("Binlog adapter: failed to open delta log", zap.String("logPath", logPath), zap.Error(err)) - return nil, fmt.Errorf("failed to open delta log '%s', error: %w", logPath, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to open delta log '%s', error: %v", logPath, err)) } defer binlogFile.Close() @@ -425,7 +425,7 @@ func (p *BinlogAdapter) readDeltalog(logPath string) ([]string, error) { data, err := binlogFile.ReadVarchar() if err != nil { log.Warn("Binlog adapter: failed to read delta log", zap.String("logPath", logPath), zap.Error(err)) - return nil, fmt.Errorf("failed to read delta log '%s', error: %w", logPath, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read delta log '%s', error: %v", logPath, err)) } log.Info("Binlog adapter: successfully read deltalog", zap.Int("deleteCount", len(data))) @@ -438,13 +438,13 @@ func (p *BinlogAdapter) readTimestamp(logPath string) ([]int64, error) { binlogFile, err := NewBinlogFile(p.chunkManager) if err != nil { log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) - return nil, fmt.Errorf("failed to initialize binlog file '%s', error: %w", logPath, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize binlog file '%s', error: %v", logPath, err)) } err = binlogFile.Open(logPath) if err != nil { log.Warn("Binlog adapter: failed to open timestamp log file", zap.String("logPath", logPath)) - return nil, fmt.Errorf("failed to open timestamp log file '%s', error: %w", logPath, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to open timestamp log file '%s', error: %v", logPath, err)) } defer binlogFile.Close() @@ -452,7 +452,7 @@ func (p *BinlogAdapter) readTimestamp(logPath string) ([]int64, error) { int64List, err := binlogFile.ReadInt64() if err != nil { log.Warn("Binlog adapter: failed to read timestamp data from log file", zap.String("logPath", logPath)) - return nil, fmt.Errorf("failed to read timestamp data from log file '%s', error: %w", logPath, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read timestamp data from log file '%s', error: %v", logPath, err)) } log.Info("Binlog adapter: read timestamp from log file", zap.Int("tsCount", len(int64List))) @@ -466,13 +466,13 @@ func (p *BinlogAdapter) readPrimaryKeys(logPath string) ([]int64, []string, erro binlogFile, err := NewBinlogFile(p.chunkManager) if err != nil { log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) - return nil, nil, fmt.Errorf("failed to initialize binlog file '%s', error: %w", logPath, err) + return nil, nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize binlog file '%s', error: %v", logPath, err)) } err = binlogFile.Open(logPath) if err != nil { log.Warn("Binlog adapter: failed to open primary key binlog", zap.String("logPath", logPath)) - return nil, nil, fmt.Errorf("failed to open primary key binlog '%s', error: %w", logPath, err) + return nil, nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to open primary key binlog '%s', error: %v", logPath, err)) } defer binlogFile.Close() @@ -482,7 +482,7 @@ func (p *BinlogAdapter) readPrimaryKeys(logPath string) ([]int64, []string, erro idList, err := binlogFile.ReadInt64() if err != nil { log.Warn("Binlog adapter: failed to read int64 primary key from binlog", zap.String("logPath", logPath), zap.Error(err)) - return nil, nil, fmt.Errorf("failed to read int64 primary key from binlog '%s', error: %w", logPath, err) + return nil, nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int64 primary key from binlog '%s', error: %v", logPath, err)) } log.Info("Binlog adapter: succeed to read int64 primary key binlog", zap.Int("len", len(idList))) return idList, nil, nil @@ -490,13 +490,13 @@ func (p *BinlogAdapter) readPrimaryKeys(logPath string) ([]int64, []string, erro idList, err := binlogFile.ReadVarchar() if err != nil { log.Warn("Binlog adapter: failed to read varchar primary key from binlog", zap.String("logPath", logPath), zap.Error(err)) - return nil, nil, fmt.Errorf("failed to read varchar primary key from binlog '%s', error: %w", logPath, err) + return nil, nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read varchar primary key from binlog '%s', error: %v", logPath, err)) } log.Info("Binlog adapter: succeed to read varchar primary key binlog", zap.Int("len", len(idList))) return nil, idList, nil } log.Warn("Binlog adapter: unsupported primary key type", zap.Int("type", int(primaryKey.GetDataType()))) - return nil, nil, fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType()) + return nil, nil, merr.WrapErrImportFailed(fmt.Sprintf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType())) } // getShardingListByPrimaryInt64 method generates a shard id list by primary key(int64) list and deleted list. @@ -511,7 +511,7 @@ func (p *BinlogAdapter) getShardingListByPrimaryInt64(primaryKeys []int64, if len(timestampList) != len(primaryKeys) { log.Warn("Binlog adapter: primary key length is not equal to timestamp list length", zap.Int("primaryKeysLen", len(primaryKeys)), zap.Int("timestampLen", len(timestampList))) - return nil, fmt.Errorf("primary key length %d is not equal to timestamp list length %d", len(primaryKeys), len(timestampList)) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("primary key length %d is not equal to timestamp list length %d", len(primaryKeys), len(timestampList))) } log.Info("Binlog adapter: building shard list", zap.Int("pkLen", len(primaryKeys)), zap.Int("tsLen", len(timestampList))) @@ -565,7 +565,7 @@ func (p *BinlogAdapter) getShardingListByPrimaryVarchar(primaryKeys []string, if len(timestampList) != len(primaryKeys) { log.Warn("Binlog adapter: primary key length is not equal to timestamp list length", zap.Int("primaryKeysLen", len(primaryKeys)), zap.Int("timestampLen", len(timestampList))) - return nil, fmt.Errorf("primary key length %d is not equal to timestamp list length %d", len(primaryKeys), len(timestampList)) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("primary key length %d is not equal to timestamp list length %d", len(primaryKeys), len(timestampList))) } log.Info("Binlog adapter: building shard list", zap.Int("pkLen", len(primaryKeys)), zap.Int("tsLen", len(timestampList))) @@ -616,7 +616,7 @@ func (p *BinlogAdapter) verifyField(fieldID storage.FieldID, memoryData []ShardD _, ok := fields[fieldID] if !ok { log.Warn("Binlog adapter: the field ID doesn't exist in collection schema", zap.Int64("fieldID", fieldID)) - return fmt.Errorf("the field ID %d doesn't exist in collection schema", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("the field ID %d doesn't exist in collection schema", fieldID)) } } return nil @@ -637,20 +637,20 @@ func (p *BinlogAdapter) readInsertlog(fieldID storage.FieldID, logPath string, err := p.verifyField(fieldID, memoryData) if err != nil { log.Warn("Binlog adapter: could not read binlog file", zap.String("logPath", logPath), zap.Error(err)) - return fmt.Errorf("could not read binlog file %s, error: %w", logPath, err) + return merr.WrapErrImportFailed(fmt.Sprintf("could not read binlog file %s, error: %v", logPath, err)) } // open the insert log file binlogFile, err := NewBinlogFile(p.chunkManager) if err != nil { log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) - return fmt.Errorf("failed to initialize binlog file %s, error: %w", logPath, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize binlog file %s, error: %v", logPath, err)) } err = binlogFile.Open(logPath) if err != nil { log.Warn("Binlog adapter: failed to open insert log", zap.String("logPath", logPath), zap.Error(err)) - return fmt.Errorf("failed to open insert log %s, error: %w", logPath, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to open insert log %s, error: %v", logPath, err)) } defer binlogFile.Close() @@ -766,8 +766,19 @@ func (p *BinlogAdapter) readInsertlog(fieldID storage.FieldID, logPath string, if err != nil { return err } + case schemapb.DataType_Array: + data, err := binlogFile.ReadArray() + if err != nil { + return err + } + + err = p.dispatchArrayToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + default: - return fmt.Errorf("unsupported data type %d", binlogFile.DataType()) + return merr.WrapErrImportFailed(fmt.Sprintf("unsupported data type %d", binlogFile.DataType())) } log.Info("Binlog adapter: read data into shard list", zap.Int("dataType", int(binlogFile.DataType())), zap.Int("shardLen", len(shardList))) @@ -780,10 +791,10 @@ func (p *BinlogAdapter) dispatchBoolToShards(data []bool, memoryData []ShardData // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: bool field row count is not equal to shard list row count %d", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("bool field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("bool field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -791,7 +802,7 @@ func (p *BinlogAdapter) dispatchBoolToShards(data []bool, memoryData []ShardData } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: bool field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("bool field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("bool field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -801,7 +812,7 @@ func (p *BinlogAdapter) dispatchBoolToShards(data []bool, memoryData []ShardData if !ok { log.Warn("Binlog adapter: binlog is bool type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is bool type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is bool type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -815,10 +826,10 @@ func (p *BinlogAdapter) dispatchInt8ToShards(data []int8, memoryData []ShardData // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: int8 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("int8 field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("int8 field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entity acoording to shard list + // dispatch entity according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -826,7 +837,7 @@ func (p *BinlogAdapter) dispatchInt8ToShards(data []int8, memoryData []ShardData } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: int8 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("int8 field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("int8 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -836,7 +847,7 @@ func (p *BinlogAdapter) dispatchInt8ToShards(data []int8, memoryData []ShardData if !ok { log.Warn("Binlog adapter: binlog is int8 type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is int8 type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is int8 type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -850,10 +861,10 @@ func (p *BinlogAdapter) dispatchInt16ToShards(data []int16, memoryData []ShardDa // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: int16 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("int16 field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("int16 field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -861,7 +872,7 @@ func (p *BinlogAdapter) dispatchInt16ToShards(data []int16, memoryData []ShardDa } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: int16 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("int16 field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("int16 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -871,7 +882,7 @@ func (p *BinlogAdapter) dispatchInt16ToShards(data []int16, memoryData []ShardDa if !ok { log.Warn("Binlog adapter: binlog is int16 type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is int16 type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is int16 type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -885,10 +896,10 @@ func (p *BinlogAdapter) dispatchInt32ToShards(data []int32, memoryData []ShardDa // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: int32 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("int32 field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("int32 field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -896,7 +907,7 @@ func (p *BinlogAdapter) dispatchInt32ToShards(data []int32, memoryData []ShardDa } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: int32 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("int32 field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("int32 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -906,7 +917,7 @@ func (p *BinlogAdapter) dispatchInt32ToShards(data []int32, memoryData []ShardDa if !ok { log.Warn("Binlog adapter: binlog is int32 type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is int32 type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is int32 type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -920,10 +931,10 @@ func (p *BinlogAdapter) dispatchInt64ToShards(data []int64, memoryData []ShardDa // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: int64 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("int64 field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("int64 field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -931,7 +942,7 @@ func (p *BinlogAdapter) dispatchInt64ToShards(data []int64, memoryData []ShardDa } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: int64 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("int64 field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("int64 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -941,7 +952,7 @@ func (p *BinlogAdapter) dispatchInt64ToShards(data []int64, memoryData []ShardDa if !ok { log.Warn("Binlog adapter: binlog is int64 type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is int64 type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is int64 type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -955,10 +966,10 @@ func (p *BinlogAdapter) dispatchFloatToShards(data []float32, memoryData []Shard // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: float field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("float field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("float field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -966,7 +977,7 @@ func (p *BinlogAdapter) dispatchFloatToShards(data []float32, memoryData []Shard } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: float field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("float field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("float field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -976,7 +987,7 @@ func (p *BinlogAdapter) dispatchFloatToShards(data []float32, memoryData []Shard if !ok { log.Warn("Binlog adapter: binlog is float type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is float type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is float type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -990,10 +1001,10 @@ func (p *BinlogAdapter) dispatchDoubleToShards(data []float64, memoryData []Shar // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: double field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("double field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("double field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -1001,7 +1012,7 @@ func (p *BinlogAdapter) dispatchDoubleToShards(data []float64, memoryData []Shar } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: double field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("double field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("double field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -1011,7 +1022,7 @@ func (p *BinlogAdapter) dispatchDoubleToShards(data []float64, memoryData []Shar if !ok { log.Warn("Binlog adapter: binlog is double type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is double type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is double type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -1025,10 +1036,10 @@ func (p *BinlogAdapter) dispatchVarcharToShards(data []string, memoryData []Shar // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: varchar field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("varchar field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("varchar field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -1036,7 +1047,7 @@ func (p *BinlogAdapter) dispatchVarcharToShards(data []string, memoryData []Shar } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: varchar field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("varchar field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("varchar field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -1046,7 +1057,7 @@ func (p *BinlogAdapter) dispatchVarcharToShards(data []string, memoryData []Shar if !ok { log.Warn("Binlog adapter: binlog is varchar type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is varchar type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is varchar type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -1060,10 +1071,10 @@ func (p *BinlogAdapter) dispatchBytesToShards(data [][]byte, memoryData []ShardD // verify row count if len(data) != len(shardList) { log.Warn("Binlog adapter: JSON field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("varchar JSON row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("varchar JSON row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i, val := range data { shardID := shardList[i] if shardID < 0 { @@ -1071,7 +1082,7 @@ func (p *BinlogAdapter) dispatchBytesToShards(data [][]byte, memoryData []ShardD } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: JSON field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("JSON field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("JSON field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -1081,7 +1092,7 @@ func (p *BinlogAdapter) dispatchBytesToShards(data [][]byte, memoryData []ShardD if !ok { log.Warn("Binlog adapter: binlog is JSON type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is JSON type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is JSON type, unequal to field %d", fieldID)) } fieldData.Data = append(fieldData.Data, val) } @@ -1098,10 +1109,10 @@ func (p *BinlogAdapter) dispatchBinaryVecToShards(data []byte, dim int, memoryDa if count != len(shardList) { log.Warn("Binlog adapter: binary vector field row count is not equal to shard list row count", zap.Int("dataLen", count), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("binary vector field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("binary vector field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i := 0; i < count; i++ { shardID := shardList[i] if shardID < 0 { @@ -1109,7 +1120,7 @@ func (p *BinlogAdapter) dispatchBinaryVecToShards(data []byte, dim int, memoryDa } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: binary vector field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("binary vector field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("binary vector field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -1119,13 +1130,13 @@ func (p *BinlogAdapter) dispatchBinaryVecToShards(data []byte, dim int, memoryDa if !ok { log.Warn("Binlog adapter: binlog is binary vector type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is binary vector type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is binary vector type, unequal to field %d", fieldID)) } if fieldData.Dim != dim { log.Warn("Binlog adapter: binary vector dimension mismatch", zap.Int("sourceDim", dim), zap.Int("schemaDim", fieldData.Dim)) - return fmt.Errorf("binary vector dimension %d is not equal to schema dimension %d", dim, fieldData.Dim) + return merr.WrapErrImportFailed(fmt.Sprintf("binary vector dimension %d is not equal to schema dimension %d", dim, fieldData.Dim)) } for j := 0; j < bytesPerVector; j++ { val := data[bytesPerVector*i+j] @@ -1145,10 +1156,10 @@ func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memory if count != len(shardList) { log.Warn("Binlog adapter: float vector field row count is not equal to shard list row count", zap.Int("dataLen", count), zap.Int("shardLen", len(shardList))) - return fmt.Errorf("float vector field row count %d is not equal to shard list row count %d", len(data), len(shardList)) + return merr.WrapErrImportFailed(fmt.Sprintf("float vector field row count %d is not equal to shard list row count %d", len(data), len(shardList))) } - // dispatch entities acoording to shard list + // dispatch entities according to shard list for i := 0; i < count; i++ { shardID := shardList[i] if shardID < 0 { @@ -1156,7 +1167,7 @@ func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memory } if shardID >= int32(len(memoryData)) { log.Warn("Binlog adapter: float vector field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) - return fmt.Errorf("float vector field's shard ID %d is larger than shards number %d", shardID, len(memoryData)) + return merr.WrapErrImportFailed(fmt.Sprintf("float vector field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) } partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here @@ -1166,13 +1177,13 @@ func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memory if !ok { log.Warn("Binlog adapter: binlog is float vector type, unequal to field", zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) - return fmt.Errorf("binlog is float vector type, unequal to field %d", fieldID) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is float vector type, unequal to field %d", fieldID)) } if fieldData.Dim != dim { log.Warn("Binlog adapter: float vector dimension mismatch", zap.Int("sourceDim", dim), zap.Int("schemaDim", fieldData.Dim)) - return fmt.Errorf("binary vector dimension %d is not equal to schema dimension %d", dim, fieldData.Dim) + return merr.WrapErrImportFailed(fmt.Sprintf("binary vector dimension %d is not equal to schema dimension %d", dim, fieldData.Dim)) } for j := 0; j < dim; j++ { val := data[dim*i+j] @@ -1182,3 +1193,39 @@ func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memory return nil } + +func (p *BinlogAdapter) dispatchArrayToShards(data []*schemapb.ScalarField, memoryData []ShardData, + shardList []int32, fieldID storage.FieldID, +) error { + // verify row count + if len(data) != len(shardList) { + log.Warn("Binlog adapter: Array field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return merr.WrapErrImportFailed(fmt.Sprintf("array row count %d is not equal to shard list row count %d", len(data), len(shardList))) + } + + // dispatch entities according to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + if shardID >= int32(len(memoryData)) { + log.Warn("Binlog adapter: Array field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData))) + return merr.WrapErrImportFailed(fmt.Sprintf("array field's shard ID %d is larger than shards number %d", shardID, len(memoryData))) + } + + partitions := memoryData[shardID] // initBlockData() can ensure the existence, no need to check bound here + fields := partitions[p.collectionInfo.PartitionIDs[0]] // NewBinlogAdapter() can ensure only one partition + field := fields[fieldID] // initBlockData() can ensure the existence, no need to check existence here + fieldData, ok := field.(*storage.ArrayFieldData) // avoid data type mismatch between binlog file and schema + if !ok { + log.Warn("Binlog adapter: binlog is array type, unequal to field", + zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID)) + return merr.WrapErrImportFailed(fmt.Sprintf("binlog is array type, unequal to field %d", fieldID)) + } + fieldData.Data = append(fieldData.Data, val) + // TODO @cai: set element type + } + + return nil +} diff --git a/internal/util/importutil/binlog_adapter_test.go b/internal/util/importutil/binlog_adapter_test.go index 866169a7971c3..251a3861c7f89 100644 --- a/internal/util/importutil/binlog_adapter_test.go +++ b/internal/util/importutil/binlog_adapter_test.go @@ -134,7 +134,7 @@ func Test_BinlogAdapterVerify(t *testing.T) { // row id field missed holder.fieldFiles = make(map[int64][]string) - for i := int64(102); i <= 112; i++ { + for i := int64(102); i <= 113; i++ { holder.fieldFiles[i] = make([]string, 0) } err = adapter.verify(holder) @@ -156,7 +156,7 @@ func Test_BinlogAdapterVerify(t *testing.T) { assert.Error(t, err) // succeed - for i := int64(102); i <= 112; i++ { + for i := int64(102); i <= 113; i++ { holder.fieldFiles[i] = []string{ "a", } @@ -667,6 +667,7 @@ func Test_BinlogAdapterReadInt64PK(t *testing.T) { int64(110): {"110_insertlog"}, int64(111): {"111_insertlog"}, int64(112): {"112_insertlog"}, + int64(113): {"113_insertlog"}, } holder.deltaFiles = []string{"deltalog"} err = adapter.Read(holder) @@ -689,6 +690,7 @@ func Test_BinlogAdapterReadInt64PK(t *testing.T) { "110_insertlog": createBinlogBuf(t, schemapb.DataType_BinaryVector, fieldsData[110].([][]byte)), "111_insertlog": createBinlogBuf(t, schemapb.DataType_FloatVector, fieldsData[111].([][]float32)), "112_insertlog": createBinlogBuf(t, schemapb.DataType_JSON, fieldsData[112].([][]byte)), + "113_insertlog": createBinlogBuf(t, schemapb.DataType_Array, fieldsData[113].([]*schemapb.ScalarField)), "deltalog": createDeltalogBuf(t, deletedItems, false), } @@ -1013,6 +1015,79 @@ func Test_BinlogAdapterDispatch(t *testing.T) { assert.Equal(t, 0, shardsData[2][partitionID][fieldID].RowNum()) }) + t.Run("dispatch Array data", func(t *testing.T) { + fieldID := int64(113) + // row count mismatch + data := []*schemapb.ScalarField{ + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{1, 2, 3, 4, 5}, + }, + }, + }, + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{7, 8, 9}, + }, + }, + }, + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{10, 11}, + }, + }, + }, + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{}, + }, + }, + }, + } + err = adapter.dispatchArrayToShards(data, shardsData, shardList, fieldID) + assert.Error(t, err) + for _, shardData := range shardsData { + assert.Equal(t, 0, shardData[partitionID][fieldID].RowNum()) + } + + // illegal shard ID + err = adapter.dispatchArrayToShards(data, shardsData, []int32{9, 1, 0, 2}, fieldID) + assert.Error(t, err) + + // succeed + err = adapter.dispatchArrayToShards([]*schemapb.ScalarField{ + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{}, + }, + }, + }, + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{}, + }, + }, + }, + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{}, + }, + }, + }, + }, shardsData, shardList, fieldID) + assert.NoError(t, err) + assert.Equal(t, 1, shardsData[0][partitionID][fieldID].RowNum()) + assert.Equal(t, 1, shardsData[1][partitionID][fieldID].RowNum()) + assert.Equal(t, 0, shardsData[2][partitionID][fieldID].RowNum()) + }) + t.Run("dispatch binary vector data", func(t *testing.T) { fieldID := int64(110) // row count mismatch @@ -1186,6 +1261,10 @@ func Test_BinlogAdapterReadInsertlog(t *testing.T) { failedFunc(111, "floatvector", schemapb.DataType_FloatVector, 110, schemapb.DataType_BinaryVector) }) + t.Run("failed to dispatch Array data", func(t *testing.T) { + failedFunc(113, "array", schemapb.DataType_Array, 111, schemapb.DataType_FloatVector) + }) + // succeed chunkManager.readBuf["int32"] = createBinlogBuf(t, schemapb.DataType_Int32, fieldsData[105].([]int32)) err = adapter.readInsertlog(105, "int32", shardsData, []int32{0, 1, 1}) diff --git a/internal/util/importutil/binlog_file.go b/internal/util/importutil/binlog_file.go index 2353de7299d4f..98d2ca142fa96 100644 --- a/internal/util/importutil/binlog_file.go +++ b/internal/util/importutil/binlog_file.go @@ -20,12 +20,12 @@ import ( "context" "fmt" - "github.com/cockroachdb/errors" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" ) // BinlogFile class is a wrapper of storage.BinlogReader, to read binlog file, block by block. @@ -40,7 +40,7 @@ type BinlogFile struct { func NewBinlogFile(chunkManager storage.ChunkManager) (*BinlogFile, error) { if chunkManager == nil { log.Warn("Binlog file: chunk manager pointer is nil") - return nil, errors.New("chunk manager pointer is nil") + return nil, merr.WrapErrImportFailed("chunk manager pointer is nil") } binlogFile := &BinlogFile{ @@ -54,20 +54,20 @@ func (p *BinlogFile) Open(filePath string) error { p.Close() if len(filePath) == 0 { log.Warn("Binlog file: binlog path is empty") - return errors.New("binlog path is empty") + return merr.WrapErrImportFailed("binlog path is empty") } // TODO add context bytes, err := p.chunkManager.Read(context.TODO(), filePath) if err != nil { log.Warn("Binlog file: failed to open binlog", zap.String("filePath", filePath), zap.Error(err)) - return fmt.Errorf("failed to open binlog %s", filePath) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to open binlog %s", filePath)) } p.reader, err = storage.NewBinlogReader(bytes) if err != nil { log.Warn("Binlog file: failed to initialize binlog reader", zap.String("filePath", filePath), zap.Error(err)) - return fmt.Errorf("failed to initialize binlog reader for binlog %s, error: %w", filePath, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize binlog reader for binlog %s, error: %v", filePath, err)) } log.Info("Binlog file: open binlog successfully", zap.String("filePath", filePath)) @@ -95,7 +95,7 @@ func (p *BinlogFile) DataType() schemapb.DataType { func (p *BinlogFile) ReadBool() ([]bool, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]bool, 0) @@ -103,7 +103,7 @@ func (p *BinlogFile) ReadBool() ([]bool, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -113,18 +113,18 @@ func (p *BinlogFile) ReadBool() ([]bool, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Bool { log.Warn("Binlog file: binlog data type is not bool") - return nil, errors.New("binlog data type is not bool") + return nil, merr.WrapErrImportFailed("binlog data type is not bool") } data, err := event.PayloadReaderInterface.GetBoolFromPayload() if err != nil { log.Warn("Binlog file: failed to read bool data", zap.Error(err)) - return nil, fmt.Errorf("failed to read bool data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read bool data, error: %v", err)) } result = append(result, data...) @@ -138,7 +138,7 @@ func (p *BinlogFile) ReadBool() ([]bool, error) { func (p *BinlogFile) ReadInt8() ([]int8, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]int8, 0) @@ -146,7 +146,7 @@ func (p *BinlogFile) ReadInt8() ([]int8, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -156,18 +156,18 @@ func (p *BinlogFile) ReadInt8() ([]int8, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Int8 { log.Warn("Binlog file: binlog data type is not int8") - return nil, errors.New("binlog data type is not int8") + return nil, merr.WrapErrImportFailed("binlog data type is not int8") } data, err := event.PayloadReaderInterface.GetInt8FromPayload() if err != nil { log.Warn("Binlog file: failed to read int8 data", zap.Error(err)) - return nil, fmt.Errorf("failed to read int8 data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int8 data, error: %v", err)) } result = append(result, data...) @@ -181,7 +181,7 @@ func (p *BinlogFile) ReadInt8() ([]int8, error) { func (p *BinlogFile) ReadInt16() ([]int16, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]int16, 0) @@ -189,7 +189,7 @@ func (p *BinlogFile) ReadInt16() ([]int16, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -199,18 +199,18 @@ func (p *BinlogFile) ReadInt16() ([]int16, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Int16 { log.Warn("Binlog file: binlog data type is not int16") - return nil, errors.New("binlog data type is not int16") + return nil, merr.WrapErrImportFailed("binlog data type is not int16") } data, err := event.PayloadReaderInterface.GetInt16FromPayload() if err != nil { log.Warn("Binlog file: failed to read int16 data", zap.Error(err)) - return nil, fmt.Errorf("failed to read int16 data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int16 data, error: %v", err)) } result = append(result, data...) @@ -224,7 +224,7 @@ func (p *BinlogFile) ReadInt16() ([]int16, error) { func (p *BinlogFile) ReadInt32() ([]int32, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]int32, 0) @@ -232,7 +232,7 @@ func (p *BinlogFile) ReadInt32() ([]int32, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -242,18 +242,18 @@ func (p *BinlogFile) ReadInt32() ([]int32, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Int32 { log.Warn("Binlog file: binlog data type is not int32") - return nil, errors.New("binlog data type is not int32") + return nil, merr.WrapErrImportFailed("binlog data type is not int32") } data, err := event.PayloadReaderInterface.GetInt32FromPayload() if err != nil { log.Warn("Binlog file: failed to read int32 data", zap.Error(err)) - return nil, fmt.Errorf("failed to read int32 data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int32 data, error: %v", err)) } result = append(result, data...) @@ -267,7 +267,7 @@ func (p *BinlogFile) ReadInt32() ([]int32, error) { func (p *BinlogFile) ReadInt64() ([]int64, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]int64, 0) @@ -275,7 +275,7 @@ func (p *BinlogFile) ReadInt64() ([]int64, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -285,18 +285,18 @@ func (p *BinlogFile) ReadInt64() ([]int64, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Int64 { log.Warn("Binlog file: binlog data type is not int64") - return nil, errors.New("binlog data type is not int64") + return nil, merr.WrapErrImportFailed("binlog data type is not int64") } data, err := event.PayloadReaderInterface.GetInt64FromPayload() if err != nil { log.Warn("Binlog file: failed to read int64 data", zap.Error(err)) - return nil, fmt.Errorf("failed to read int64 data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int64 data, error: %v", err)) } result = append(result, data...) @@ -310,7 +310,7 @@ func (p *BinlogFile) ReadInt64() ([]int64, error) { func (p *BinlogFile) ReadFloat() ([]float32, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]float32, 0) @@ -318,7 +318,7 @@ func (p *BinlogFile) ReadFloat() ([]float32, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -328,18 +328,18 @@ func (p *BinlogFile) ReadFloat() ([]float32, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Float { log.Warn("Binlog file: binlog data type is not float") - return nil, errors.New("binlog data type is not float") + return nil, merr.WrapErrImportFailed("binlog data type is not float") } data, err := event.PayloadReaderInterface.GetFloatFromPayload() if err != nil { log.Warn("Binlog file: failed to read float data", zap.Error(err)) - return nil, fmt.Errorf("failed to read float data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float data, error: %v", err)) } result = append(result, data...) @@ -353,7 +353,7 @@ func (p *BinlogFile) ReadFloat() ([]float32, error) { func (p *BinlogFile) ReadDouble() ([]float64, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]float64, 0) @@ -361,7 +361,7 @@ func (p *BinlogFile) ReadDouble() ([]float64, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -371,18 +371,18 @@ func (p *BinlogFile) ReadDouble() ([]float64, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Double { log.Warn("Binlog file: binlog data type is not double") - return nil, errors.New("binlog data type is not double") + return nil, merr.WrapErrImportFailed("binlog data type is not double") } data, err := event.PayloadReaderInterface.GetDoubleFromPayload() if err != nil { log.Warn("Binlog file: failed to read double data", zap.Error(err)) - return nil, fmt.Errorf("failed to read double data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read double data, error: %v", err)) } result = append(result, data...) @@ -396,7 +396,7 @@ func (p *BinlogFile) ReadDouble() ([]float64, error) { func (p *BinlogFile) ReadVarchar() ([]string, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([]string, 0) @@ -404,7 +404,7 @@ func (p *BinlogFile) ReadVarchar() ([]string, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -415,18 +415,18 @@ func (p *BinlogFile) ReadVarchar() ([]string, error) { // special case: delete event data type is varchar if event.TypeCode != storage.InsertEventType && event.TypeCode != storage.DeleteEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if (p.DataType() != schemapb.DataType_VarChar) && (p.DataType() != schemapb.DataType_String) { log.Warn("Binlog file: binlog data type is not varchar") - return nil, errors.New("binlog data type is not varchar") + return nil, merr.WrapErrImportFailed("binlog data type is not varchar") } data, err := event.PayloadReaderInterface.GetStringFromPayload() if err != nil { log.Warn("Binlog file: failed to read varchar data", zap.Error(err)) - return nil, fmt.Errorf("failed to read varchar data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read varchar data, error: %v", err)) } result = append(result, data...) @@ -440,7 +440,7 @@ func (p *BinlogFile) ReadVarchar() ([]string, error) { func (p *BinlogFile) ReadJSON() ([][]byte, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, errors.New("binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") } result := make([][]byte, 0) @@ -448,7 +448,7 @@ func (p *BinlogFile) ReadJSON() ([][]byte, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -458,18 +458,61 @@ func (p *BinlogFile) ReadJSON() ([][]byte, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, errors.New("binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_JSON { log.Warn("Binlog file: binlog data type is not JSON") - return nil, errors.New("binlog data type is not JSON") + return nil, merr.WrapErrImportFailed("binlog data type is not JSON") } data, err := event.PayloadReaderInterface.GetJSONFromPayload() if err != nil { log.Warn("Binlog file: failed to read JSON data", zap.Error(err)) - return nil, fmt.Errorf("failed to read JSON data, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read JSON data, error: %v", err)) + } + + result = append(result, data...) + } + + return result, nil +} + +// ReadArray method reads all the blocks of a binlog by a data type. +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +func (p *BinlogFile) ReadArray() ([]*schemapb.ScalarField, error) { + if p.reader == nil { + log.Warn("Binlog file: binlog reader not yet initialized") + return nil, merr.WrapErrImportFailed("binlog reader not yet initialized") + } + + result := make([]*schemapb.ScalarField, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Warn("Binlog file: binlog file is not insert log") + return nil, merr.WrapErrImportFailed("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Array { + log.Warn("Binlog file: binlog data type is not Array") + return nil, merr.WrapErrImportFailed("binlog data type is not Array") + } + + data, err := event.PayloadReaderInterface.GetArrayFromPayload() + if err != nil { + log.Warn("Binlog file: failed to read Array data", zap.Error(err)) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read Array data, error: %v", err)) } result = append(result, data...) @@ -484,7 +527,7 @@ func (p *BinlogFile) ReadJSON() ([][]byte, error) { func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, 0, errors.New("binlog reader not yet initialized") + return nil, 0, merr.WrapErrImportFailed("binlog reader not yet initialized") } dim := 0 @@ -493,7 +536,7 @@ func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, 0, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -503,18 +546,18 @@ func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, 0, errors.New("binlog file is not insert log") + return nil, 0, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_BinaryVector { log.Warn("Binlog file: binlog data type is not binary vector") - return nil, 0, errors.New("binlog data type is not binary vector") + return nil, 0, merr.WrapErrImportFailed("binlog data type is not binary vector") } data, dimenson, err := event.PayloadReaderInterface.GetBinaryVectorFromPayload() if err != nil { log.Warn("Binlog file: failed to read binary vector data", zap.Error(err)) - return nil, 0, fmt.Errorf("failed to read binary vector data, error: %w", err) + return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to read binary vector data, error: %v", err)) } dim = dimenson @@ -527,7 +570,7 @@ func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) { func (p *BinlogFile) ReadFloat16Vector() ([]byte, int, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, 0, errors.New("binlog reader not yet initialized") + return nil, 0, merr.WrapErrImportFailed("binlog reader not yet initialized") } dim := 0 @@ -536,7 +579,7 @@ func (p *BinlogFile) ReadFloat16Vector() ([]byte, int, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, 0, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -546,18 +589,18 @@ func (p *BinlogFile) ReadFloat16Vector() ([]byte, int, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, 0, errors.New("binlog file is not insert log") + return nil, 0, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_Float16Vector { log.Warn("Binlog file: binlog data type is not float16 vector") - return nil, 0, errors.New("binlog data type is not float16 vector") + return nil, 0, merr.WrapErrImportFailed("binlog data type is not float16 vector") } data, dimenson, err := event.PayloadReaderInterface.GetFloat16VectorFromPayload() if err != nil { log.Warn("Binlog file: failed to read float16 vector data", zap.Error(err)) - return nil, 0, fmt.Errorf("failed to read float16 vector data, error: %w", err) + return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float16 vector data, error: %v", err)) } dim = dimenson @@ -573,7 +616,7 @@ func (p *BinlogFile) ReadFloat16Vector() ([]byte, int, error) { func (p *BinlogFile) ReadFloatVector() ([]float32, int, error) { if p.reader == nil { log.Warn("Binlog file: binlog reader not yet initialized") - return nil, 0, errors.New("binlog reader not yet initialized") + return nil, 0, merr.WrapErrImportFailed("binlog reader not yet initialized") } dim := 0 @@ -582,7 +625,7 @@ func (p *BinlogFile) ReadFloatVector() ([]float32, int, error) { event, err := p.reader.NextEventReader() if err != nil { log.Warn("Binlog file: failed to iterate events reader", zap.Error(err)) - return nil, 0, fmt.Errorf("failed to iterate events reader, error: %w", err) + return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err)) } // end of the file @@ -592,18 +635,18 @@ func (p *BinlogFile) ReadFloatVector() ([]float32, int, error) { if event.TypeCode != storage.InsertEventType { log.Warn("Binlog file: binlog file is not insert log") - return nil, 0, errors.New("binlog file is not insert log") + return nil, 0, merr.WrapErrImportFailed("binlog file is not insert log") } if p.DataType() != schemapb.DataType_FloatVector { log.Warn("Binlog file: binlog data type is not float vector") - return nil, 0, errors.New("binlog data type is not float vector") + return nil, 0, merr.WrapErrImportFailed("binlog data type is not float vector") } data, dimension, err := event.PayloadReaderInterface.GetFloatVectorFromPayload() if err != nil { log.Warn("Binlog file: failed to read float vector data", zap.Error(err)) - return nil, 0, fmt.Errorf("failed to read float vector data, error: %w", err) + return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float vector data, error: %v", err)) } dim = dimension diff --git a/internal/util/importutil/binlog_file_test.go b/internal/util/importutil/binlog_file_test.go index 4a80983d8fe2b..ec12a754d1b4e 100644 --- a/internal/util/importutil/binlog_file_test.go +++ b/internal/util/importutil/binlog_file_test.go @@ -129,6 +129,17 @@ func createBinlogBuf(t *testing.T, dataType schemapb.DataType, data interface{}) // without the two lines, the case will crash at here. // the "original_size" is come from storage.originalSizeKey w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_Array: + rows := data.([]*schemapb.ScalarField) + sizeTotal := 0 + for i := 0; i < len(rows); i++ { + err = evt.AddOneArrayToPayload(rows[i]) + assert.NoError(t, err) + sizeTotal += binary.Size(rows[i]) + } + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) case schemapb.DataType_BinaryVector: vectors := data.([][]byte) for i := 0; i < len(vectors); i++ { @@ -276,6 +287,10 @@ func Test_BinlogFileOpen(t *testing.T) { assert.Nil(t, dataFloat16Vector) assert.Equal(t, 0, dim) assert.Error(t, err) + + dataArray, err := binlogFile.ReadArray() + assert.Nil(t, dataArray) + assert.Error(t, err) } func Test_BinlogFileBool(t *testing.T) { @@ -780,6 +795,99 @@ func Test_BinlogFileJSON(t *testing.T) { binlogFile.Close() } +func Test_BinlogFileArray(t *testing.T) { + source := []*schemapb.ScalarField{ + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{1, 2, 3}, + }, + }, + }, + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{4, 5}, + }, + }, + }, + { + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{6, 7, 8, 9}, + }, + }, + }, + } + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Array, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.NoError(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.NoError(t, err) + assert.Equal(t, schemapb.DataType_Array, binlogFile.DataType()) + + data, err := binlogFile.ReadArray() + assert.NoError(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.ElementsMatch(t, source[i].GetIntData().GetData(), data[i].GetIntData().GetData()) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.NoError(t, err) + err = binlogFile.Open("dummy") + assert.NoError(t, err) + + d, dim, err := binlogFile.ReadBinaryVector() + assert.Zero(t, len(d)) + assert.Zero(t, dim) + assert.Error(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.NoError(t, err) + + data, err = binlogFile.ReadArray() + assert.Zero(t, len(data)) + assert.Error(t, err) + + // failed to iterate events reader + binlogFile.reader.Close() + data, err = binlogFile.ReadArray() + assert.Zero(t, len(data)) + assert.Error(t, err) + + binlogFile.Close() + + chunkManager.readBuf["dummy"] = createBinlogBuf(t, schemapb.DataType_Bool, []bool{true, false}) + binlogFile, err = NewBinlogFile(chunkManager) + assert.NoError(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.NoError(t, err) + data, err = binlogFile.ReadArray() + assert.Error(t, err) + assert.Nil(t, data) + binlogFile.Close() +} + func Test_BinlogFileBinaryVector(t *testing.T) { vectors := make([][]byte, 0) vectors = append(vectors, []byte{1, 3, 5, 7}) diff --git a/internal/util/importutil/binlog_parser.go b/internal/util/importutil/binlog_parser.go index 9ea2ce039a55f..edcffcc85c8d8 100644 --- a/internal/util/importutil/binlog_parser.go +++ b/internal/util/importutil/binlog_parser.go @@ -24,11 +24,11 @@ import ( "strconv" "strings" - "github.com/cockroachdb/errors" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" ) type BinlogParser struct { @@ -64,23 +64,23 @@ func NewBinlogParser(ctx context.Context, ) (*BinlogParser, error) { if collectionInfo == nil { log.Warn("Binlog parser: collection schema is nil") - return nil, errors.New("collection schema is nil") + return nil, merr.WrapErrImportFailed("collection schema is nil") } if chunkManager == nil { log.Warn("Binlog parser: chunk manager pointer is nil") - return nil, errors.New("chunk manager pointer is nil") + return nil, merr.WrapErrImportFailed("chunk manager pointer is nil") } if flushFunc == nil { log.Warn("Binlog parser: flush function is nil") - return nil, errors.New("flush function is nil") + return nil, merr.WrapErrImportFailed("flush function is nil") } if tsStartPoint > tsEndPoint { log.Warn("Binlog parser: the tsStartPoint should be less than tsEndPoint", zap.Uint64("tsStartPoint", tsStartPoint), zap.Uint64("tsEndPoint", tsEndPoint)) - return nil, fmt.Errorf("Binlog parser: the tsStartPoint %d should be less than tsEndPoint %d", tsStartPoint, tsEndPoint) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("Binlog parser: the tsStartPoint %d should be less than tsEndPoint %d", tsStartPoint, tsEndPoint)) } v := &BinlogParser{ @@ -121,7 +121,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo insertlogs, _, err := p.chunkManager.ListWithPrefix(context.TODO(), insertlogRoot, true) if err != nil { log.Warn("Binlog parser: list insert logs error", zap.Error(err)) - return nil, fmt.Errorf("failed to list insert logs with root path %s, error: %w", insertlogRoot, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to list insert logs with root path %s, error: %v", insertlogRoot, err)) } // collect insert log paths @@ -139,7 +139,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo fieldID, err := strconv.ParseInt(fieldStrID, 10, 64) if err != nil { log.Warn("Binlog parser: failed to parse field id", zap.String("fieldPath", fieldPath), zap.Error(err)) - return nil, fmt.Errorf("failed to parse field id from insert log path %s, error: %w", insertlog, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse field id from insert log path %s, error: %v", insertlog, err)) } segmentPath := path.Dir(fieldPath) @@ -147,7 +147,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo segmentID, err := strconv.ParseInt(segmentStrID, 10, 64) if err != nil { log.Warn("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err)) - return nil, fmt.Errorf("failed to parse segment id from insert log path %s, error: %w", insertlog, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse segment id from insert log path %s, error: %v", insertlog, err)) } holder, ok := holders[segmentID] @@ -186,7 +186,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo deltalogs, _, err := p.chunkManager.ListWithPrefix(context.TODO(), deltalogRoot, true) if err != nil { log.Warn("Binlog parser: failed to list delta logs", zap.Error(err)) - return nil, fmt.Errorf("failed to list delta logs, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to list delta logs, error: %v", err)) } log.Info("Binlog parser: list delta logs", zap.Int("logsCount", len(deltalogs))) @@ -197,7 +197,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo segmentID, err := strconv.ParseInt(segmentStrID, 10, 64) if err != nil { log.Warn("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err)) - return nil, fmt.Errorf("failed to parse segment id from delta log path %s, error: %w", deltalog, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse segment id from delta log path %s, error: %v", deltalog, err)) } // if the segment id doesn't exist, no need to process this deltalog @@ -221,14 +221,14 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo func (p *BinlogParser) parseSegmentFiles(segmentHolder *SegmentFilesHolder) error { if segmentHolder == nil { log.Warn("Binlog parser: segment files holder is nil") - return errors.New("segment files holder is nil") + return merr.WrapErrImportFailed("segment files holder is nil") } adapter, err := NewBinlogAdapter(p.ctx, p.collectionInfo, p.blockSize, MaxTotalSizeInMemory, p.chunkManager, p.callFlushFunc, p.tsStartPoint, p.tsEndPoint) if err != nil { log.Warn("Binlog parser: failed to create binlog adapter", zap.Error(err)) - return fmt.Errorf("failed to create binlog adapter, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to create binlog adapter, error: %v", err)) } return adapter.Read(segmentHolder) @@ -240,7 +240,7 @@ func (p *BinlogParser) parseSegmentFiles(segmentHolder *SegmentFilesHolder) erro func (p *BinlogParser) Parse(filePaths []string) error { if len(filePaths) != 1 && len(filePaths) != 2 { log.Warn("Binlog parser: illegal paths for binlog import, partition binlog path and delta path are required") - return errors.New("illegal paths for binlog import, partition binlog path and delta path are required") + return merr.WrapErrImportFailed("illegal paths for binlog import, partition binlog path and delta path are required") } insertlogPath := filePaths[0] diff --git a/internal/util/importutil/binlog_parser_test.go b/internal/util/importutil/binlog_parser_test.go index d2be3fd63f3d8..afd7ce2b19f06 100644 --- a/internal/util/importutil/binlog_parser_test.go +++ b/internal/util/importutil/binlog_parser_test.go @@ -337,6 +337,7 @@ func Test_BinlogParserParse(t *testing.T) { "123/110/a", "123/111/a", "123/112/a", + "123/113/a", } chunkManager.readBuf = map[string][]byte{ "123/0/a": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)), @@ -352,6 +353,7 @@ func Test_BinlogParserParse(t *testing.T) { "123/110/a": createBinlogBuf(t, schemapb.DataType_BinaryVector, fieldsData[110].([][]byte)), "123/111/a": createBinlogBuf(t, schemapb.DataType_FloatVector, fieldsData[111].([][]float32)), "123/112/a": createBinlogBuf(t, schemapb.DataType_JSON, fieldsData[112].([][]byte)), + "123/113/a": createBinlogBuf(t, schemapb.DataType_Array, fieldsData[113].([]*schemapb.ScalarField)), } callTime := 0 diff --git a/internal/util/importutil/collection_info.go b/internal/util/importutil/collection_info.go index f7fc31270e874..b7b64829fad67 100644 --- a/internal/util/importutil/collection_info.go +++ b/internal/util/importutil/collection_info.go @@ -19,10 +19,9 @@ package importutil import ( "fmt" - "github.com/cockroachdb/errors" - "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/util/merr" ) type CollectionInfo struct { @@ -43,11 +42,11 @@ func NewCollectionInfo(collectionSchema *schemapb.CollectionSchema, partitionIDs []int64, ) (*CollectionInfo, error) { if shardNum <= 0 { - return nil, fmt.Errorf("illegal shard number %d", shardNum) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("illegal shard number %d", shardNum)) } if len(partitionIDs) == 0 { - return nil, errors.New("partition list is empty") + return nil, merr.WrapErrImportFailed("partition list is empty") } info := &CollectionInfo{ @@ -65,7 +64,7 @@ func NewCollectionInfo(collectionSchema *schemapb.CollectionSchema, func (c *CollectionInfo) resetSchema(collectionSchema *schemapb.CollectionSchema) error { if collectionSchema == nil { - return errors.New("collection schema is null") + return merr.WrapErrImportFailed("collection schema is null") } fields := make([]*schemapb.FieldSchema, 0) @@ -92,11 +91,11 @@ func (c *CollectionInfo) resetSchema(collectionSchema *schemapb.CollectionSchema } if primaryKey == nil { - return errors.New("collection schema has no primary key") + return merr.WrapErrImportFailed("collection schema has no primary key") } if partitionKey == nil && len(c.PartitionIDs) != 1 { - return errors.New("only allow one partition when there is no partition key") + return merr.WrapErrImportFailed("only allow one partition when there is no partition key") } c.Schema = &schemapb.CollectionSchema{ diff --git a/internal/util/importutil/csv_handler.go b/internal/util/importutil/csv_handler.go index f06596f2cbc54..b518f4ab2675d 100644 --- a/internal/util/importutil/csv_handler.go +++ b/internal/util/importutil/csv_handler.go @@ -23,7 +23,6 @@ import ( "strconv" "strings" - "github.com/cockroachdb/errors" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" @@ -31,6 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -60,14 +60,14 @@ func NewCSVRowConsumer(ctx context.Context, ) (*CSVRowConsumer, error) { if collectionInfo == nil { log.Warn("CSV row consumer: collection schema is nil") - return nil, errors.New("collection schema is nil") + return nil, merr.WrapErrImportFailed("collection schema is nil") } v := &CSVRowConsumer{ ctx: ctx, collectionInfo: collectionInfo, rowIDAllocator: idAlloc, - validators: make(map[storage.FieldID]*CSVValidator, 0), + validators: make(map[storage.FieldID]*CSVValidator), rowCounter: 0, shardsData: make([]ShardData, 0, collectionInfo.ShardNum), blockSize: blockSize, @@ -77,14 +77,14 @@ func NewCSVRowConsumer(ctx context.Context, if err := v.initValidators(collectionInfo.Schema); err != nil { log.Warn("CSV row consumer: fail to initialize csv row-based consumer", zap.Error(err)) - return nil, fmt.Errorf("fail to initialize csv row-based consumer, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("fail to initialize csv row-based consumer, error: %v", err)) } for i := 0; i < int(collectionInfo.ShardNum); i++ { shardData := initShardData(collectionInfo.Schema, collectionInfo.PartitionIDs) if shardData == nil { log.Warn("CSV row consumer: fail to initialize in-memory segment data", zap.Int("shardID", i)) - return nil, fmt.Errorf("fail to initialize in-memory segment data for shard id %d", i) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("fail to initialize in-memory segment data for shard id %d", i)) } v.shardsData = append(v.shardsData, shardData) } @@ -92,7 +92,7 @@ func NewCSVRowConsumer(ctx context.Context, // primary key is autoid, id generator is required if v.collectionInfo.PrimaryKey.GetAutoID() && idAlloc == nil { log.Warn("CSV row consumer: ID allocator is nil") - return nil, errors.New("ID allocator is nil") + return nil, merr.WrapErrImportFailed("ID allocator is nil") } return v, nil @@ -106,7 +106,7 @@ type CSVValidator struct { func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSchema) error { if collectionSchema == nil { - return errors.New("collection schema is nil") + return merr.WrapErrImportFailed("collection schema is nil") } validators := v.validators @@ -124,7 +124,7 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch validators[schema.GetFieldID()].convertFunc = func(str string, field storage.FieldData) error { var value bool if err := json.Unmarshal([]byte(str), &value); err != nil { - return fmt.Errorf("illegal value '%v' for bool type field '%s'", str, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for bool type field '%s'", str, schema.GetName())) } field.(*storage.BoolFieldData).Data = append(field.(*storage.BoolFieldData).Data, value) return nil @@ -151,7 +151,7 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch validators[schema.GetFieldID()].convertFunc = func(str string, field storage.FieldData) error { value, err := strconv.ParseInt(str, 0, 8) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int8 field '%s', error: %w", str, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int8 field '%s', error: %v", str, schema.GetName(), err)) } field.(*storage.Int8FieldData).Data = append(field.(*storage.Int8FieldData).Data, int8(value)) return nil @@ -160,7 +160,7 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch validators[schema.GetFieldID()].convertFunc = func(str string, field storage.FieldData) error { value, err := strconv.ParseInt(str, 0, 16) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int16 field '%s', error: %w", str, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int16 field '%s', error: %v", str, schema.GetName(), err)) } field.(*storage.Int16FieldData).Data = append(field.(*storage.Int16FieldData).Data, int16(value)) return nil @@ -169,7 +169,7 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch validators[schema.GetFieldID()].convertFunc = func(str string, field storage.FieldData) error { value, err := strconv.ParseInt(str, 0, 32) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int32 field '%s', error: %w", str, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int32 field '%s', error: %v", str, schema.GetName(), err)) } field.(*storage.Int32FieldData).Data = append(field.(*storage.Int32FieldData).Data, int32(value)) return nil @@ -178,7 +178,7 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch validators[schema.GetFieldID()].convertFunc = func(str string, field storage.FieldData) error { value, err := strconv.ParseInt(str, 0, 64) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int64 field '%s', error: %w", str, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int64 field '%s', error: %v", str, schema.GetName(), err)) } field.(*storage.Int64FieldData).Data = append(field.(*storage.Int64FieldData).Data, value) return nil @@ -194,23 +194,23 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch desc := json.NewDecoder(strings.NewReader(str)) desc.UseNumber() if err := desc.Decode(&arr); err != nil { - return fmt.Errorf("'%v' is not an array for binary vector field '%s'", str, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("'%v' is not an array for binary vector field '%s'", str, schema.GetName())) } // we use uint8 to represent binary vector in csv file, each uint8 value represents 8 dimensions. if len(arr)*8 != dim { - return fmt.Errorf("bit size %d doesn't equal to vector dimension %d of field '%s'", len(arr)*8, dim, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("bit size %d doesn't equal to vector dimension %d of field '%s'", len(arr)*8, dim, schema.GetName())) } for i := 0; i < len(arr); i++ { if num, ok := arr[i].(json.Number); ok { value, err := strconv.ParseUint(string(num), 0, 8) if err != nil { - return fmt.Errorf("failed to parse value '%v' for binary vector field '%s', error: %w", num, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for binary vector field '%s', error: %v", num, schema.GetName(), err)) } field.(*storage.BinaryVectorFieldData).Data = append(field.(*storage.BinaryVectorFieldData).Data, byte(value)) } else { - return fmt.Errorf("illegal value '%v' for binary vector field '%s'", str, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for binary vector field '%s'", str, schema.GetName())) } } @@ -227,11 +227,11 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch desc := json.NewDecoder(strings.NewReader(str)) desc.UseNumber() if err := desc.Decode(&arr); err != nil { - return fmt.Errorf("'%v' is not an array for float vector field '%s'", str, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("'%v' is not an array for float vector field '%s'", str, schema.GetName())) } if len(arr) != dim { - return fmt.Errorf("array size %d doesn't equal to vector dimension %d of field '%s'", len(arr), dim, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("array size %d doesn't equal to vector dimension %d of field '%s'", len(arr), dim, schema.GetName())) } for i := 0; i < len(arr); i++ { @@ -242,7 +242,7 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch } field.(*storage.FloatVectorFieldData).Data = append(field.(*storage.FloatVectorFieldData).Data, float32(value)) } else { - return fmt.Errorf("illegal value '%v' for float vector field '%s'", str, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for float vector field '%s'", str, schema.GetName())) } } @@ -259,13 +259,25 @@ func (v *CSVRowConsumer) initValidators(collectionSchema *schemapb.CollectionSch validators[schema.GetFieldID()].convertFunc = func(str string, field storage.FieldData) error { var dummy interface{} if err := json.Unmarshal([]byte(str), &dummy); err != nil { - return fmt.Errorf("failed to parse value '%v' for JSON field '%s', error: %w", str, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for JSON field '%s', error: %v", str, schema.GetName(), err)) } field.(*storage.JSONFieldData).Data = append(field.(*storage.JSONFieldData).Data, []byte(str)) return nil } + case schemapb.DataType_Array: + validators[schema.GetFieldID()].convertFunc = func(str string, field storage.FieldData) error { + var arr []interface{} + desc := json.NewDecoder(strings.NewReader(str)) + desc.UseNumber() + if err := desc.Decode(&arr); err != nil { + return merr.WrapErrImportFailed(fmt.Sprintf("'%v' is not an array for array field '%s'", str, schema.GetName())) + } + + return getArrayElementData(schema, arr, field) + } + default: - return fmt.Errorf("unsupport data type: %s", getTypeName(collectionSchema.Fields[i].DataType)) + return merr.WrapErrImportFailed(fmt.Sprintf("unsupport data type: %s", getTypeName(collectionSchema.Fields[i].DataType))) } } return nil @@ -282,7 +294,7 @@ func (v *CSVRowConsumer) RowCount() int64 { func (v *CSVRowConsumer) Handle(rows []map[storage.FieldID]string) error { if v == nil || v.validators == nil || len(v.validators) == 0 { log.Warn("CSV row consumer is not initialized") - return errors.New("CSV row consumer is not initialized") + return merr.WrapErrImportFailed("CSV row consumer is not initialized") } // if rows is nil, that means read to end of file, force flush all data if rows == nil { @@ -297,7 +309,7 @@ func (v *CSVRowConsumer) Handle(rows []map[storage.FieldID]string) error { err := tryFlushBlocks(v.ctx, v.shardsData, v.collectionInfo.Schema, v.callFlushFunc, v.blockSize, MaxTotalSizeInMemory, false) if err != nil { log.Warn("CSV row consumer: try flush data but failed", zap.Error(err)) - return fmt.Errorf("try flush data but failed, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("try flush data but failed, error: %v", err)) } // prepare autoid, no matter int64 or varchar pk, we always generate autoid since the hidden field RowIDField requires them @@ -308,24 +320,24 @@ func (v *CSVRowConsumer) Handle(rows []map[storage.FieldID]string) error { if v.collectionInfo.PrimaryKey.AutoID { if v.rowIDAllocator == nil { log.Warn("CSV row consumer: primary keys is auto-generated but IDAllocator is nil") - return fmt.Errorf("primary keys is auto-generated but IDAllocator is nil") + return merr.WrapErrImportFailed("primary keys is auto-generated but IDAllocator is nil") } var err error rowIDBegin, rowIDEnd, err = v.rowIDAllocator.Alloc(uint32(len(rows))) if err != nil { log.Warn("CSV row consumer: failed to generate primary keys", zap.Int("count", len(rows)), zap.Error(err)) - return fmt.Errorf("failed to generate %d primary keys, error: %w", len(rows), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to generate %d primary keys, error: %v", len(rows), err)) } if rowIDEnd-rowIDBegin != int64(len(rows)) { log.Warn("CSV row consumer: try to generate primary keys but allocated ids are not enough", zap.Int("count", len(rows)), zap.Int64("generated", rowIDEnd-rowIDBegin)) - return fmt.Errorf("try to generate %d primary keys but only %d keys were allocated", len(rows), rowIDEnd-rowIDBegin) + return merr.WrapErrImportFailed(fmt.Sprintf("try to generate %d primary keys but only %d keys were allocated", len(rows), rowIDEnd-rowIDBegin)) } log.Info("CSV row consumer: auto-generate primary keys", zap.Int64("begin", rowIDBegin), zap.Int64("end", rowIDEnd)) if primaryValidator.isString { // if pk is varchar, no need to record auto-generated row ids log.Warn("CSV row consumer: string type primary key connot be auto-generated") - return errors.New("string type primary key connot be auto-generated") + return merr.WrapErrImportFailed("string type primary key connot be auto-generated") } v.autoIDRange = append(v.autoIDRange, rowIDBegin, rowIDEnd) } @@ -361,8 +373,8 @@ func (v *CSVRowConsumer) Handle(rows []map[storage.FieldID]string) error { if err != nil { log.Warn("CSV row consumer: failed to parse primary key at the row", zap.String("value", pkStr), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to parse primary key '%s' at the row %d, error: %w", - pkStr, rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key '%s' at the row %d, error: %v", + pkStr, rowNumber, err)) } } @@ -370,7 +382,7 @@ func (v *CSVRowConsumer) Handle(rows []map[storage.FieldID]string) error { if err != nil { log.Warn("CSV row consumer: failed to hash primary key at the row", zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to hash primary key %d at the row %d, error: %w", pk, rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to hash primary key %d at the row %d, error: %v", pk, rowNumber, err)) } // hash to shard based on pk, hash to partition if partition key exist @@ -395,8 +407,8 @@ func (v *CSVRowConsumer) Handle(rows []map[storage.FieldID]string) error { if err := validator.convertFunc(value, v.shardsData[shardID][partitionID][fieldID]); err != nil { log.Warn("CSV row consumer: failed to convert value for field at the row", zap.String("fieldName", validator.fieldName), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to convert value for field '%s' at the row %d, error: %w", - validator.fieldName, rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert value for field '%s' at the row %d, error: %v", + validator.fieldName, rowNumber, err)) } } } @@ -405,12 +417,12 @@ func (v *CSVRowConsumer) Handle(rows []map[storage.FieldID]string) error { return nil } -// hashToPartition hash partition key to get an partition ID, return the first partition ID if no partition key exist +// hashToPartition hash partition key to get a partition ID, return the first partition ID if no partition key exist // CollectionInfo ensures only one partition ID in the PartitionIDs if no partition key exist func (v *CSVRowConsumer) hashToPartition(row map[storage.FieldID]string, rowNumber int64) (int64, error) { if v.collectionInfo.PartitionKey == nil { if len(v.collectionInfo.PartitionIDs) != 1 { - return 0, fmt.Errorf("collection '%s' partition list is empty", v.collectionInfo.Schema.Name) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("collection '%s' partition list is empty", v.collectionInfo.Schema.Name)) } // no partition key, directly return the target partition id return v.collectionInfo.PartitionIDs[0], nil @@ -429,15 +441,15 @@ func (v *CSVRowConsumer) hashToPartition(row map[storage.FieldID]string, rowNumb if err != nil { log.Warn("CSV row consumer: failed to parse partition key at the row", zap.String("value", value), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return 0, fmt.Errorf("failed to parse partition key '%s' at the row %d, error: %w", - value, rowNumber, err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse partition key '%s' at the row %d, error: %v", + value, rowNumber, err)) } hashValue, err = typeutil.Hash32Int64(pk) if err != nil { log.Warn("CSV row consumer: failed to hash partition key at the row", zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return 0, fmt.Errorf("failed to hash partition key %d at the row %d, error: %w", pk, rowNumber, err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to hash partition key %d at the row %d, error: %v", pk, rowNumber, err)) } } diff --git a/internal/util/importutil/csv_handler_test.go b/internal/util/importutil/csv_handler_test.go index a2a388b6a2741..9590a30a35cf4 100644 --- a/internal/util/importutil/csv_handler_test.go +++ b/internal/util/importutil/csv_handler_test.go @@ -189,6 +189,10 @@ func Test_CSVRowConsumerInitValidators(t *testing.T) { checkConvertFunc("FieldFloatVector", validVal, invalidVal) invalidVal = `[1]` checkConvertFunc("FieldFloatVector", validVal, invalidVal) + + validVal = "[1,2,3,4]" + invalidVal = "[bool, false]" + checkConvertFunc("FieldArray", validVal, invalidVal) }) t.Run("init error cases", func(t *testing.T) { @@ -278,6 +282,168 @@ func Test_CSVRowConsumerInitValidators(t *testing.T) { assert.Error(t, err) assert.Equal(t, 2, fieldData.RowNum()) }) + + t.Run("array field", func(t *testing.T) { + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Bool, + }, + }, + } + consumer.validators = make(map[int64]*CSVValidator) + err = consumer.initValidators(schema) + assert.NoError(t, err) + + v, ok := consumer.validators[113] + assert.True(t, ok) + + fields := initBlockData(schema) + assert.NotNil(t, fields) + fieldData := fields[113] + + err = v.convertFunc("[true, false]", fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Int64, + }, + }, + } + consumer.validators = make(map[int64]*CSVValidator) + err = consumer.initValidators(schema) + assert.NoError(t, err) + + v, ok = consumer.validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc("[1,2,3,4]", fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Float, + }, + }, + } + consumer.validators = make(map[int64]*CSVValidator) + err = consumer.initValidators(schema) + assert.NoError(t, err) + + v, ok = consumer.validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc("[1.1,2.2,3.3,4.4]", fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Double, + }, + }, + } + consumer.validators = make(map[int64]*CSVValidator) + err = consumer.initValidators(schema) + assert.NoError(t, err) + + v, ok = consumer.validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc("[1.2,2.3,3.4,4.5]", fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_VarChar, + }, + }, + } + consumer.validators = make(map[int64]*CSVValidator) + err = consumer.initValidators(schema) + assert.NoError(t, err) + + v, ok = consumer.validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc(`["abc", "vv"]`, fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + }) } func Test_CSVRowConsumerHandleIntPK(t *testing.T) { diff --git a/internal/util/importutil/csv_parser.go b/internal/util/importutil/csv_parser.go index 3c4cb49b43be8..fb537c189a4ea 100644 --- a/internal/util/importutil/csv_parser.go +++ b/internal/util/importutil/csv_parser.go @@ -24,11 +24,11 @@ import ( "strconv" "strings" - "github.com/cockroachdb/errors" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -43,7 +43,7 @@ type CSVParser struct { func NewCSVParser(ctx context.Context, collectionInfo *CollectionInfo, updateProgressFunc func(percent int64)) (*CSVParser, error) { if collectionInfo == nil { log.Warn("CSV parser: collection schema is nil") - return nil, errors.New("collection schema is nil") + return nil, merr.WrapErrImportFailed("collection schema is nil") } parser := &CSVParser{ @@ -108,7 +108,7 @@ func (p *CSVParser) combineDynamicRow(dynamicValues map[string]string, row map[s desc.UseNumber() if err := desc.Decode(&mp); err != nil { log.Warn("CSV parser: illegal value for dynamic field, not a JSON object") - return errors.New("illegal value for dynamic field, not a JSON object") + return merr.WrapErrImportFailed("illegal value for dynamic field, not a JSON object") } } // case 4 @@ -145,7 +145,7 @@ func (p *CSVParser) combineDynamicRow(dynamicValues map[string]string, row map[s bs, err := json.Marshal(mp) if err != nil { log.Warn("CSV parser: illegal value for dynamic field, not a JSON object") - return errors.New("illegal value for dynamic field, not a JSON object") + return merr.WrapErrImportFailed("illegal value for dynamic field, not a JSON object") } row[dynamicFieldID] = string(bs) } else if !ok && len(dynamicValues) == 0 { @@ -168,7 +168,7 @@ func (p *CSVParser) verifyRow(raw []string) (map[storage.FieldID]string, error) if fieldID == p.collectionInfo.PrimaryKey.GetFieldID() && p.collectionInfo.PrimaryKey.GetAutoID() { // primary key is auto-id, no need to provide log.Warn("CSV parser: the primary key is auto-generated, no need to provide", zap.String("fieldName", fieldName)) - return nil, fmt.Errorf("the primary key '%s' is auto-generated, no need to provide", fieldName) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("the primary key '%s' is auto-generated, no need to provide", fieldName)) } if ok { @@ -179,7 +179,7 @@ func (p *CSVParser) verifyRow(raw []string) (map[storage.FieldID]string, error) } else { // no dynamic field. if user provided redundant field, return error log.Warn("CSV parser: the field is not defined in collection schema", zap.String("fieldName", fieldName)) - return nil, fmt.Errorf("the field '%s' is not defined in collection schema", fieldName) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("the field '%s' is not defined in collection schema", fieldName)) } } // some fields not provided? @@ -198,7 +198,7 @@ func (p *CSVParser) verifyRow(raw []string) (map[storage.FieldID]string, error) if !ok { // not auto-id primary key, no dynamic field, must provide value log.Warn("CSV parser: a field value is missed", zap.String("fieldName", k)) - return nil, fmt.Errorf("value of field '%s' is missed", k) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("value of field '%s' is missed", k)) } } } @@ -215,7 +215,7 @@ func (p *CSVParser) verifyRow(raw []string) (map[storage.FieldID]string, error) func (p *CSVParser) ParseRows(reader *IOReader, handle CSVRowHandler) error { if reader == nil || handle == nil { log.Warn("CSV Parser: CSV parse handle is nil") - return errors.New("CSV parse handle is nil") + return merr.WrapErrImportFailed("CSV parse handle is nil") } // discard bom in the file RuneScanner := reader.r.(io.RuneScanner) @@ -228,7 +228,9 @@ func (p *CSVParser) ParseRows(reader *IOReader, handle CSVRowHandler) error { return err } if bom != '\ufeff' { - RuneScanner.UnreadRune() + if err = RuneScanner.UnreadRune(); err != nil { + return err + } } r := NewReader(reader.r) @@ -252,7 +254,7 @@ func (p *CSVParser) ParseRows(reader *IOReader, handle CSVRowHandler) error { break } else if err != nil { log.Warn("CSV Parser: failed to parse the field value", zap.Error(err)) - return fmt.Errorf("failed to read the field value, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to read the field value, error: %v", err)) } p.fieldsName = fieldsName // read buffer @@ -265,7 +267,7 @@ func (p *CSVParser) ParseRows(reader *IOReader, handle CSVRowHandler) error { break } else if err != nil { log.Warn("CSV parser: failed to parse row value", zap.Error(err)) - return fmt.Errorf("failed to parse row value, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse row value, error: %v", err)) } row, err := p.verifyRow(values) @@ -280,7 +282,7 @@ func (p *CSVParser) ParseRows(reader *IOReader, handle CSVRowHandler) error { isEmpty = false if err = handle.Handle(buf); err != nil { log.Warn("CSV parser: failed to convert row value to entity", zap.Error(err)) - return fmt.Errorf("failed to convert row value to entity, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert row value to entity, error: %v", err)) } // clean the buffer buf = make([]map[storage.FieldID]string, 0, p.bufRowCount) @@ -290,14 +292,14 @@ func (p *CSVParser) ParseRows(reader *IOReader, handle CSVRowHandler) error { isEmpty = false if err = handle.Handle(buf); err != nil { log.Warn("CSV parser: failed to convert row value to entity", zap.Error(err)) - return fmt.Errorf("failed to convert row value to entity, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert row value to entity, error: %v", err)) } } // outside context might be canceled(service stop, or future enhancement for canceling import task) if isCanceled(p.ctx) { log.Warn("CSV parser: import task was canceled") - return errors.New("import task was canceled") + return merr.WrapErrImportFailed("import task was canceled") } // nolint // this break means we require the first row must be fieldsName diff --git a/internal/util/importutil/csv_parser_test.go b/internal/util/importutil/csv_parser_test.go index 70738e7fc490b..c93a34d6a20ce 100644 --- a/internal/util/importutil/csv_parser_test.go +++ b/internal/util/importutil/csv_parser_test.go @@ -86,8 +86,8 @@ func Test_CSVParserParseRows_IntPK(t *testing.T) { } reader := strings.NewReader( - `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector - true,10,101,1001,10001,3.14,1.56,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]"`) + `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector,FieldArray + true,10,101,1001,10001,3.14,1.56,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]","[1,2,3,4]"`) t.Run("parse success", func(t *testing.T) { err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer) @@ -117,28 +117,28 @@ func Test_CSVParserParseRows_IntPK(t *testing.T) { // csv parse error, fields len error reader := strings.NewReader( - `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector - 0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]"`) + `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector,FieldArray + 0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]","[1,2,3,4]"`) err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer) assert.Error(t, err) // redundant field reader = strings.NewReader( - `dummy,FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector - 1,true,0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]"`) + `dummy,FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector,FieldArray + 1,true,0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]","[1,2,3,4]"`) err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer) assert.Error(t, err) // field missed reader = strings.NewReader( - `FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector - 0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]"`) + `FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector,FieldArray + 0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]","[1,2,3,4]"`) err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer) assert.Error(t, err) // handle() error - content := `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector - true,0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]"` + content := `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector,FieldArray + true,0,100,1000,99999999999999999,3,1,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]","[1,2,3,4]"` consumer.handleErr = errors.New("error") reader = strings.NewReader(content) err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer) diff --git a/internal/util/importutil/csv_reader.go b/internal/util/importutil/csv_reader.go index b0c51d9398f9f..decd30c5c4cdb 100644 --- a/internal/util/importutil/csv_reader.go +++ b/internal/util/importutil/csv_reader.go @@ -63,6 +63,8 @@ import ( "unicode/utf8" "github.com/cockroachdb/errors" + + "github.com/milvus-io/milvus/pkg/util/merr" ) // A ParseError is returned for parsing errors. @@ -75,7 +77,7 @@ type ParseError struct { } func (e *ParseError) Error() string { - if e.Err == ErrFieldCount { + if errors.Is(e.Err, ErrFieldCount) { return fmt.Sprintf("record on line %d: %v", e.Line, e.Err) } if e.StartLine != e.Line { @@ -88,15 +90,15 @@ func (e *ParseError) Unwrap() error { return e.Err } // These are the errors that can be returned in ParseError.Err. var ( - ErrBareQuote = errors.New("bare \" in non-quoted-field") - ErrQuote = errors.New("extraneous or missing \" in quoted-field") - ErrFieldCount = errors.New("wrong number of fields") + ErrBareQuote = merr.WrapErrImportFailed("bare \" in non-quoted-field") + ErrQuote = merr.WrapErrImportFailed("extraneous or missing \" in quoted-field") + ErrFieldCount = merr.WrapErrImportFailed("wrong number of fields") // Deprecated: ErrTrailingComma is no longer used. - ErrTrailingComma = errors.New("extra delimiter at end of line") + ErrTrailingComma = merr.WrapErrImportFailed("extra delimiter at end of line") ) -var errInvalidDelim = errors.New("csv: invalid field or comment delimiter") +var errInvalidDelim = merr.WrapErrImportFailed("csv: invalid field or comment delimiter") func validDelim(r rune) bool { return r != 0 && r != '"' && r != '\r' && r != '\n' && utf8.ValidRune(r) && r != utf8.RuneError @@ -257,9 +259,9 @@ func (r *Reader) ReadAll() (records [][]string, err error) { // The result is only valid until the next call to readLine. func (r *Reader) readLine() ([]byte, error) { line, err := r.r.ReadSlice('\n') - if err == bufio.ErrBufferFull { + if errors.Is(err, bufio.ErrBufferFull) { r.rawBuffer = append(r.rawBuffer[:0], line...) - for err == bufio.ErrBufferFull { + for errors.Is(err, bufio.ErrBufferFull) { line, err = r.r.ReadSlice('\n') r.rawBuffer = append(r.rawBuffer, line...) } diff --git a/internal/util/importutil/import_options.go b/internal/util/importutil/import_options.go index 748e450c1dc30..b877a8848b1cc 100644 --- a/internal/util/importutil/import_options.go +++ b/internal/util/importutil/import_options.go @@ -21,10 +21,9 @@ import ( "strconv" "strings" - "github.com/cockroachdb/errors" - "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) @@ -81,7 +80,7 @@ func ValidateOptions(options []*commonpb.KeyValuePair) error { } } if startTs > endTs { - return errors.New("start_ts shouldn't be larger than end_ts") + return merr.WrapErrImportFailed("start_ts shouldn't be larger than end_ts") } return nil } diff --git a/internal/util/importutil/import_util.go b/internal/util/importutil/import_util.go index 5f8231b616929..e543830b9bd25 100644 --- a/internal/util/importutil/import_util.go +++ b/internal/util/importutil/import_util.go @@ -25,7 +25,6 @@ import ( "strconv" "strings" - "github.com/cockroachdb/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -34,6 +33,7 @@ import ( "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -112,6 +112,11 @@ func initBlockData(collectionSchema *schemapb.CollectionSchema) BlockData { blockData[schema.GetFieldID()] = &storage.JSONFieldData{ Data: make([][]byte, 0), } + case schemapb.DataType_Array: + blockData[schema.GetFieldID()] = &storage.ArrayFieldData{ + Data: make([]*schemapb.ScalarField, 0), + ElementType: schema.GetElementType(), + } default: log.Warn("Import util: unsupported data type", zap.String("DataType", getTypeName(schema.DataType))) return nil @@ -137,12 +142,12 @@ func initShardData(collectionSchema *schemapb.CollectionSchema, partitionIDs []i func parseFloat(s string, bitsize int, fieldName string) (float64, error) { value, err := strconv.ParseFloat(s, bitsize) if err != nil { - return 0, fmt.Errorf("failed to parse value '%s' for field '%s', error: %w", s, fieldName, err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%s' for field '%s', error: %v", s, fieldName, err)) } err = typeutil.VerifyFloat(value) if err != nil { - return 0, fmt.Errorf("illegal value '%s' for field '%s', error: %w", s, fieldName, err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%s' for field '%s', error: %v", s, fieldName, err)) } return value, nil @@ -162,7 +167,7 @@ type Validator struct { // initValidators constructs valiator methods and data conversion methods func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[storage.FieldID]*Validator) error { if collectionSchema == nil { - return errors.New("collection schema is nil") + return merr.WrapErrImportFailed("collection schema is nil") } for i := 0; i < len(collectionSchema.Fields); i++ { @@ -181,7 +186,7 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ if value, ok := obj.(bool); ok { field.(*storage.BoolFieldData).Data = append(field.(*storage.BoolFieldData).Data, value) } else { - return fmt.Errorf("illegal value '%v' for bool type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for bool type field '%s'", obj, schema.GetName())) } return nil @@ -195,7 +200,7 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ } field.(*storage.FloatFieldData).Data = append(field.(*storage.FloatFieldData).Data, float32(value)) } else { - return fmt.Errorf("illegal value '%v' for float type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for float type field '%s'", obj, schema.GetName())) } return nil @@ -209,7 +214,7 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ } field.(*storage.DoubleFieldData).Data = append(field.(*storage.DoubleFieldData).Data, value) } else { - return fmt.Errorf("illegal value '%v' for double type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for double type field '%s'", obj, schema.GetName())) } return nil } @@ -218,11 +223,11 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ if num, ok := obj.(json.Number); ok { value, err := strconv.ParseInt(string(num), 0, 8) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int8 field '%s', error: %w", num, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int8 field '%s', error: %v", num, schema.GetName(), err)) } field.(*storage.Int8FieldData).Data = append(field.(*storage.Int8FieldData).Data, int8(value)) } else { - return fmt.Errorf("illegal value '%v' for int8 type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int8 type field '%s'", obj, schema.GetName())) } return nil } @@ -231,11 +236,11 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ if num, ok := obj.(json.Number); ok { value, err := strconv.ParseInt(string(num), 0, 16) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int16 field '%s', error: %w", num, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int16 field '%s', error: %v", num, schema.GetName(), err)) } field.(*storage.Int16FieldData).Data = append(field.(*storage.Int16FieldData).Data, int16(value)) } else { - return fmt.Errorf("illegal value '%v' for int16 type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int16 type field '%s'", obj, schema.GetName())) } return nil } @@ -244,11 +249,11 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ if num, ok := obj.(json.Number); ok { value, err := strconv.ParseInt(string(num), 0, 32) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int32 field '%s', error: %w", num, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int32 field '%s', error: %v", num, schema.GetName(), err)) } field.(*storage.Int32FieldData).Data = append(field.(*storage.Int32FieldData).Data, int32(value)) } else { - return fmt.Errorf("illegal value '%v' for int32 type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int32 type field '%s'", obj, schema.GetName())) } return nil } @@ -257,11 +262,11 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ if num, ok := obj.(json.Number); ok { value, err := strconv.ParseInt(string(num), 0, 64) if err != nil { - return fmt.Errorf("failed to parse value '%v' for int64 field '%s', error: %w", num, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for int64 field '%s', error: %v", num, schema.GetName(), err)) } field.(*storage.Int64FieldData).Data = append(field.(*storage.Int64FieldData).Data, value) } else { - return fmt.Errorf("illegal value '%v' for int64 type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int64 type field '%s'", obj, schema.GetName())) } return nil } @@ -275,22 +280,22 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ validators[schema.GetFieldID()].convertFunc = func(obj interface{}, field storage.FieldData) error { arr, ok := obj.([]interface{}) if !ok { - return fmt.Errorf("'%v' is not an array for binary vector field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("'%v' is not an array for binary vector field '%s'", obj, schema.GetName())) } // we use uint8 to represent binary vector in json file, each uint8 value represents 8 dimensions. if len(arr)*8 != dim { - return fmt.Errorf("bit size %d doesn't equal to vector dimension %d of field '%s'", len(arr)*8, dim, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("bit size %d doesn't equal to vector dimension %d of field '%s'", len(arr)*8, dim, schema.GetName())) } for i := 0; i < len(arr); i++ { if num, ok := arr[i].(json.Number); ok { value, err := strconv.ParseUint(string(num), 0, 8) if err != nil { - return fmt.Errorf("failed to parse value '%v' for binary vector field '%s', error: %w", num, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for binary vector field '%s', error: %v", num, schema.GetName(), err)) } field.(*storage.BinaryVectorFieldData).Data = append(field.(*storage.BinaryVectorFieldData).Data, byte(value)) } else { - return fmt.Errorf("illegal value '%v' for binary vector field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for binary vector field '%s'", obj, schema.GetName())) } } @@ -306,10 +311,10 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ validators[schema.GetFieldID()].convertFunc = func(obj interface{}, field storage.FieldData) error { arr, ok := obj.([]interface{}) if !ok { - return fmt.Errorf("'%v' is not an array for float vector field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("'%v' is not an array for float vector field '%s'", obj, schema.GetName())) } if len(arr) != dim { - return fmt.Errorf("array size %d doesn't equal to vector dimension %d of field '%s'", len(arr), dim, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("array size %d doesn't equal to vector dimension %d of field '%s'", len(arr), dim, schema.GetName())) } for i := 0; i < len(arr); i++ { @@ -320,7 +325,7 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ } field.(*storage.FloatVectorFieldData).Data = append(field.(*storage.FloatVectorFieldData).Data, float32(value)) } else { - return fmt.Errorf("illegal value '%v' for float vector field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for float vector field '%s'", obj, schema.GetName())) } } @@ -333,7 +338,7 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ if value, ok := obj.(string); ok { field.(*storage.StringFieldData).Data = append(field.(*storage.StringFieldData).Data, value) } else { - return fmt.Errorf("illegal value '%v' for varchar type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for varchar type field '%s'", obj, schema.GetName())) } return nil } @@ -345,28 +350,197 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[ var dummy interface{} err := json.Unmarshal([]byte(value), &dummy) if err != nil { - return fmt.Errorf("failed to parse value '%v' for JSON field '%s', error: %w", value, schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for JSON field '%s', error: %v", value, schema.GetName(), err)) } field.(*storage.JSONFieldData).Data = append(field.(*storage.JSONFieldData).Data, []byte(value)) } else if mp, ok := obj.(map[string]interface{}); ok { bs, err := json.Marshal(mp) if err != nil { - return fmt.Errorf("failed to parse value for JSON field '%s', error: %w", schema.GetName(), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value for JSON field '%s', error: %v", schema.GetName(), err)) } field.(*storage.JSONFieldData).Data = append(field.(*storage.JSONFieldData).Data, bs) } else { - return fmt.Errorf("illegal value '%v' for JSON type field '%s'", obj, schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for JSON type field '%s'", obj, schema.GetName())) } return nil } + case schemapb.DataType_Array: + validators[schema.GetFieldID()].convertFunc = func(obj interface{}, field storage.FieldData) error { + arr, ok := obj.([]interface{}) + if !ok { + return merr.WrapErrImportFailed(fmt.Sprintf("'%v' is not an array for array field '%s'", obj, schema.GetName())) + } + return getArrayElementData(schema, arr, field) + } default: - return fmt.Errorf("unsupport data type: %s", getTypeName(collectionSchema.Fields[i].DataType)) + return merr.WrapErrImportFailed(fmt.Sprintf("unsupport data type: %s", getTypeName(collectionSchema.Fields[i].DataType))) } } return nil } +func getArrayElementData(schema *schemapb.FieldSchema, arr []interface{}, field storage.FieldData) error { + switch schema.GetElementType() { + case schemapb.DataType_Bool: + boolData := make([]bool, 0) + for i := 0; i < len(arr); i++ { + if value, ok := arr[i].(bool); ok { + boolData = append(boolData, value) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for bool array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_BoolData{ + BoolData: &schemapb.BoolArray{ + Data: boolData, + }, + }, + }) + case schemapb.DataType_Int8: + int8Data := make([]int32, 0) + for i := 0; i < len(arr); i++ { + if num, ok := arr[i].(json.Number); ok { + value, err := strconv.ParseInt(string(num), 0, 8) + if err != nil { + return err + } + int8Data = append(int8Data, int32(value)) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: int8Data, + }, + }, + }) + + case schemapb.DataType_Int16: + int16Data := make([]int32, 0) + for i := 0; i < len(arr); i++ { + if num, ok := arr[i].(json.Number); ok { + value, err := strconv.ParseInt(string(num), 0, 16) + if err != nil { + return err + } + int16Data = append(int16Data, int32(value)) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: int16Data, + }, + }, + }) + case schemapb.DataType_Int32: + intData := make([]int32, 0) + for i := 0; i < len(arr); i++ { + if num, ok := arr[i].(json.Number); ok { + value, err := strconv.ParseInt(string(num), 0, 32) + if err != nil { + return err + } + intData = append(intData, int32(value)) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: intData, + }, + }, + }) + case schemapb.DataType_Int64: + longData := make([]int64, 0) + for i := 0; i < len(arr); i++ { + if num, ok := arr[i].(json.Number); ok { + value, err := strconv.ParseInt(string(num), 0, 64) + if err != nil { + return err + } + longData = append(longData, value) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for long array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_LongData{ + LongData: &schemapb.LongArray{ + Data: longData, + }, + }, + }) + case schemapb.DataType_Float: + floatData := make([]float32, 0) + for i := 0; i < len(arr); i++ { + if num, ok := arr[i].(json.Number); ok { + value, err := parseFloat(string(num), 32, schema.GetName()) + if err != nil { + return err + } + floatData = append(floatData, float32(value)) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for float array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_FloatData{ + FloatData: &schemapb.FloatArray{ + Data: floatData, + }, + }, + }) + case schemapb.DataType_Double: + doubleData := make([]float64, 0) + for i := 0; i < len(arr); i++ { + if num, ok := arr[i].(json.Number); ok { + value, err := parseFloat(string(num), 32, schema.GetName()) + if err != nil { + return err + } + doubleData = append(doubleData, value) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for double array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_DoubleData{ + DoubleData: &schemapb.DoubleArray{ + Data: doubleData, + }, + }, + }) + case schemapb.DataType_String, schemapb.DataType_VarChar: + stringFieldData := &schemapb.ScalarField{ + Data: &schemapb.ScalarField_StringData{ + StringData: &schemapb.StringArray{ + Data: make([]string, 0), + }, + }, + } + for i := 0; i < len(arr); i++ { + if str, ok := arr[i].(string); ok { + stringFieldData.GetStringData().Data = append(stringFieldData.GetStringData().Data, str) + } else { + return merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for string array field '%s'", arr, schema.GetName())) + } + } + field.(*storage.ArrayFieldData).Data = append(field.(*storage.ArrayFieldData).Data, stringFieldData) + default: + return merr.WrapErrImportFailed(fmt.Sprintf("unsupport element type: %v", getTypeName(schema.GetElementType()))) + } + return nil +} + func printFieldsDataInfo(fieldsData BlockData, msg string, files []string) { stats := make([]zapcore.Field, 0) for k, v := range fieldsData { @@ -395,13 +569,13 @@ func getFieldDimension(schema *schemapb.FieldSchema) (int, error) { if key == common.DimKey { dim, err := strconv.Atoi(value) if err != nil { - return 0, fmt.Errorf("illegal vector dimension '%s' for field '%s', error: %w", value, schema.GetName(), err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("illegal vector dimension '%s' for field '%s', error: %v", value, schema.GetName(), err)) } return dim, nil } } - return 0, fmt.Errorf("vector dimension is not defined for field '%s'", schema.GetName()) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("vector dimension is not defined for field '%s'", schema.GetName())) } // triggerGC triggers golang gc to return all free memory back to the underlying system at once, @@ -426,7 +600,7 @@ func fillDynamicData(blockData BlockData, collectionSchema *schemapb.CollectionS } if dynamicFieldID < 0 { - return fmt.Errorf("the collection schema is dynamic but dynamic field is not found") + return merr.WrapErrImportFailed("the collection schema is dynamic but dynamic field is not found") } rowCount := 0 @@ -483,7 +657,7 @@ func tryFlushBlocks(ctx context.Context, // outside context might be canceled(service stop, or future enhancement for canceling import task) if isCanceled(ctx) { log.Warn("Import util: import task was canceled") - return errors.New("import task was canceled") + return merr.WrapErrImportFailed("import task was canceled") } shardData := shardsData[i] @@ -491,7 +665,7 @@ func tryFlushBlocks(ctx context.Context, err := fillDynamicData(blockData, collectionSchema) if err != nil { log.Warn("Import util: failed to fill dynamic field", zap.Error(err)) - return fmt.Errorf("failed to fill dynamic field, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to fill dynamic field, error: %v", err)) } // Note: even rowCount is 0, the size is still non-zero @@ -509,7 +683,7 @@ func tryFlushBlocks(ctx context.Context, if err != nil { log.Warn("Import util: failed to force flush block data", zap.Int("shardID", i), zap.Int64("partitionID", partitionID), zap.Error(err)) - return fmt.Errorf("failed to force flush block data for shard id %d to partition %d, error: %w", i, partitionID, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to force flush block data for shard id %d to partition %d, error: %v", i, partitionID, err)) } log.Info("Import util: force flush", zap.Int("rowCount", rowCount), zap.Int("size", size), zap.Int("shardID", i), zap.Int64("partitionID", partitionID)) @@ -517,7 +691,7 @@ func tryFlushBlocks(ctx context.Context, shardData[partitionID] = initBlockData(collectionSchema) if shardData[partitionID] == nil { log.Warn("Import util: failed to initialize FieldData list", zap.Int("shardID", i), zap.Int64("partitionID", partitionID)) - return fmt.Errorf("failed to initialize FieldData list for shard id %d to partition %d", i, partitionID) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize FieldData list for shard id %d to partition %d", i, partitionID)) } continue } @@ -530,7 +704,7 @@ func tryFlushBlocks(ctx context.Context, if err != nil { log.Warn("Import util: failed to flush block data", zap.Int("shardID", i), zap.Int64("partitionID", partitionID), zap.Error(err)) - return fmt.Errorf("failed to flush block data for shard id %d to partition %d, error: %w", i, partitionID, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to flush block data for shard id %d to partition %d, error: %v", i, partitionID, err)) } log.Info("Import util: block size exceed limit and flush", zap.Int("rowCount", rowCount), zap.Int("size", size), zap.Int("shardID", i), zap.Int64("partitionID", partitionID), zap.Int64("blockSize", blockSize)) @@ -538,7 +712,7 @@ func tryFlushBlocks(ctx context.Context, shardData[partitionID] = initBlockData(collectionSchema) if shardData[partitionID] == nil { log.Warn("Import util: failed to initialize FieldData list", zap.Int("shardID", i), zap.Int64("partitionID", partitionID)) - return fmt.Errorf("failed to initialize FieldData list for shard id %d to partition %d", i, partitionID) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize FieldData list for shard id %d to partition %d", i, partitionID)) } continue } @@ -559,14 +733,14 @@ func tryFlushBlocks(ctx context.Context, // outside context might be canceled(service stop, or future enhancement for canceling import task) if isCanceled(ctx) { log.Warn("Import util: import task was canceled") - return errors.New("import task was canceled") + return merr.WrapErrImportFailed("import task was canceled") } blockData := shardsData[biggestItem][biggestPartition] err := fillDynamicData(blockData, collectionSchema) if err != nil { log.Warn("Import util: failed to fill dynamic field", zap.Error(err)) - return fmt.Errorf("failed to fill dynamic field, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to fill dynamic field, error: %v", err)) } // Note: even rowCount is 0, the size is still non-zero @@ -583,8 +757,8 @@ func tryFlushBlocks(ctx context.Context, if err != nil { log.Warn("Import util: failed to flush biggest block data", zap.Int("shardID", biggestItem), zap.Int64("partitionID", biggestPartition)) - return fmt.Errorf("failed to flush biggest block data for shard id %d to partition %d, error: %w", - biggestItem, biggestPartition, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to flush biggest block data for shard id %d to partition %d, error: %v", + biggestItem, biggestPartition, err)) } log.Info("Import util: total size exceed limit and flush", zap.Int("rowCount", rowCount), zap.Int("size", size), zap.Int("totalSize", totalSize), zap.Int("shardID", biggestItem)) @@ -593,7 +767,7 @@ func tryFlushBlocks(ctx context.Context, if shardsData[biggestItem][biggestPartition] == nil { log.Warn("Import util: failed to initialize FieldData list", zap.Int("shardID", biggestItem), zap.Int64("partitionID", biggestPartition)) - return fmt.Errorf("failed to initialize FieldData list for shard id %d to partition %d", biggestItem, biggestPartition) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize FieldData list for shard id %d to partition %d", biggestItem, biggestPartition)) } } } @@ -642,7 +816,7 @@ func pkToShard(pk interface{}, shardNum uint32) (uint32, error) { intPK, ok := pk.(int64) if !ok { log.Warn("Numpy parser: primary key field must be int64 or varchar") - return 0, fmt.Errorf("primary key field must be int64 or varchar") + return 0, merr.WrapErrImportFailed("primary key field must be int64 or varchar") } hash, _ := typeutil.Hash32Int64(intPK) shard = hash % shardNum @@ -653,7 +827,7 @@ func pkToShard(pk interface{}, shardNum uint32) (uint32, error) { func UpdateKVInfo(infos *[]*commonpb.KeyValuePair, k string, v string) error { if infos == nil { - return errors.New("Import util: kv array pointer is nil") + return merr.WrapErrImportFailed("Import util: kv array pointer is nil") } found := false diff --git a/internal/util/importutil/import_util_test.go b/internal/util/importutil/import_util_test.go index 859b746f46688..8ec2819d60d5f 100644 --- a/internal/util/importutil/import_util_test.go +++ b/internal/util/importutil/import_util_test.go @@ -127,6 +127,14 @@ func sampleSchema() *schemapb.CollectionSchema { Description: "json", DataType: schemapb.DataType_JSON, }, + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + Description: "array", + DataType: schemapb.DataType_Array, + ElementType: schemapb.DataType_Int32, + }, }, } return schema @@ -145,6 +153,7 @@ type sampleRow struct { FieldJSON string FieldBinaryVector []int FieldFloatVector []float32 + FieldArray []int32 } type sampleContent struct { Rows []sampleRow @@ -324,7 +333,18 @@ func createFieldsData(collectionSchema *schemapb.CollectionSchema, rowCount int) jsonData = append(jsonData, []byte(fmt.Sprintf("{\"y\": %d}", i))) } fieldsData[schema.GetFieldID()] = jsonData - + case schemapb.DataType_Array: + arrayData := make([]*schemapb.ScalarField, 0) + for i := 0; i < rowCount; i++ { + arrayData = append(arrayData, &schemapb.ScalarField{ + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{ + Data: []int32{int32(i), int32(i + 1), int32(i + 2)}, + }, + }, + }) + } + fieldsData[schema.GetFieldID()] = arrayData default: return nil } @@ -372,6 +392,8 @@ func createBlockData(collectionSchema *schemapb.CollectionSchema, fieldsData map blockData[fieldID].(*storage.StringFieldData).Data = append(blockData[fieldID].(*storage.StringFieldData).Data, fieldsData[fieldID].([]string)...) case schemapb.DataType_JSON: blockData[fieldID].(*storage.JSONFieldData).Data = append(blockData[fieldID].(*storage.JSONFieldData).Data, fieldsData[fieldID].([][]byte)...) + case schemapb.DataType_Array: + blockData[fieldID].(*storage.ArrayFieldData).Data = append(blockData[fieldID].(*storage.ArrayFieldData).Data, fieldsData[fieldID].([]*schemapb.ScalarField)...) default: return nil } @@ -586,6 +608,7 @@ func Test_InitValidators(t *testing.T) { checkConvertFunc("FieldFloatVector", validVal, invalidVal) invalidVal = []interface{}{jsonNumber("1"), jsonNumber("2"), jsonNumber("3"), true} checkConvertFunc("FieldFloatVector", validVal, invalidVal) + checkConvertFunc("FieldArray", validVal, invalidVal) }) t.Run("init error cases", func(t *testing.T) { @@ -673,6 +696,230 @@ func Test_InitValidators(t *testing.T) { assert.Error(t, err) assert.Equal(t, 2, fieldData.RowNum()) }) + + t.Run("array field", func(t *testing.T) { + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Bool, + }, + }, + } + validators = make(map[storage.FieldID]*Validator) + err = initValidators(schema, validators) + assert.NoError(t, err) + + v, ok := validators[113] + assert.True(t, ok) + + fields := initBlockData(schema) + assert.NotNil(t, fields) + fieldData := fields[113] + + err = v.convertFunc([]interface{}{true, false}, fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + err = v.convertFunc([]interface{}{1, 2}, fieldData) + assert.Error(t, err) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Int32, + }, + }, + } + validators = make(map[storage.FieldID]*Validator) + err = initValidators(schema, validators) + assert.NoError(t, err) + + v, ok = validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc([]interface{}{jsonNumber("1"), jsonNumber("2"), jsonNumber("3"), jsonNumber("4")}, fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + err = v.convertFunc([]interface{}{true, false}, fieldData) + assert.Error(t, err) + + err = v.convertFunc([]interface{}{jsonNumber("1.1"), jsonNumber("2.2")}, fieldData) + assert.Error(t, err) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Int64, + }, + }, + } + validators = make(map[storage.FieldID]*Validator) + err = initValidators(schema, validators) + assert.NoError(t, err) + + v, ok = validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc([]interface{}{jsonNumber("1"), jsonNumber("2"), jsonNumber("3"), jsonNumber("4")}, fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + err = v.convertFunc([]interface{}{true, false}, fieldData) + assert.Error(t, err) + + err = v.convertFunc([]interface{}{jsonNumber("1.1"), jsonNumber("2.2")}, fieldData) + assert.Error(t, err) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Float, + }, + }, + } + validators = make(map[storage.FieldID]*Validator) + err = initValidators(schema, validators) + assert.NoError(t, err) + + v, ok = validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc([]interface{}{jsonNumber("1.1"), jsonNumber("2.2"), jsonNumber("3.3"), jsonNumber("4.4")}, fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + err = v.convertFunc([]interface{}{true, false}, fieldData) + assert.Error(t, err) + + err = v.convertFunc([]interface{}{jsonNumber("1.1.1"), jsonNumber("2.2.2")}, fieldData) + assert.Error(t, err) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_Double, + }, + }, + } + validators = make(map[storage.FieldID]*Validator) + err = initValidators(schema, validators) + assert.NoError(t, err) + + v, ok = validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc([]interface{}{jsonNumber("1.2"), jsonNumber("2.3"), jsonNumber("3.4"), jsonNumber("4.5")}, fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + err = v.convertFunc([]interface{}{true, false}, fieldData) + assert.Error(t, err) + + err = v.convertFunc([]interface{}{jsonNumber("1.1.1"), jsonNumber("2.2.2")}, fieldData) + assert.Error(t, err) + + schema = &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 113, + Name: "FieldArray", + IsPrimaryKey: false, + DataType: schemapb.DataType_Array, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_capacity", Value: "100"}, + }, + ElementType: schemapb.DataType_VarChar, + }, + }, + } + validators = make(map[storage.FieldID]*Validator) + err = initValidators(schema, validators) + assert.NoError(t, err) + + v, ok = validators[113] + assert.True(t, ok) + + fields = initBlockData(schema) + assert.NotNil(t, fields) + fieldData = fields[113] + + err = v.convertFunc([]interface{}{"abc", "def"}, fieldData) + assert.NoError(t, err) + assert.Equal(t, 1, fieldData.RowNum()) + + err = v.convertFunc([]interface{}{true, false}, fieldData) + assert.Error(t, err) + }) } func Test_GetFileNameAndExt(t *testing.T) { @@ -829,7 +1076,7 @@ func Test_TryFlushBlocks(t *testing.T) { return nil } - blockSize := int64(1024) + blockSize := int64(2048) maxTotalSize := int64(4096) shardNum := int32(3) schema := sampleSchema() diff --git a/internal/util/importutil/import_wrapper.go b/internal/util/importutil/import_wrapper.go index d136d164fcf57..e2e970b38e44f 100644 --- a/internal/util/importutil/import_wrapper.go +++ b/internal/util/importutil/import_wrapper.go @@ -31,6 +31,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/timerecord" ) @@ -149,17 +150,17 @@ func NewImportWrapper(ctx context.Context, collectionInfo *CollectionInfo, segme func (p *ImportWrapper) SetCallbackFunctions(assignSegmentFunc AssignSegmentFunc, createBinlogsFunc CreateBinlogsFunc, saveSegmentFunc SaveSegmentFunc) error { if assignSegmentFunc == nil { log.Warn("import wrapper: callback function AssignSegmentFunc is nil") - return fmt.Errorf("callback function AssignSegmentFunc is nil") + return merr.WrapErrImportFailed("callback function AssignSegmentFunc is nil") } if createBinlogsFunc == nil { log.Warn("import wrapper: callback function CreateBinlogsFunc is nil") - return fmt.Errorf("callback function CreateBinlogsFunc is nil") + return merr.WrapErrImportFailed("callback function CreateBinlogsFunc is nil") } if saveSegmentFunc == nil { log.Warn("import wrapper: callback function SaveSegmentFunc is nil") - return fmt.Errorf("callback function SaveSegmentFunc is nil") + return merr.WrapErrImportFailed("callback function SaveSegmentFunc is nil") } p.assignSegmentFunc = assignSegmentFunc @@ -190,7 +191,7 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) { // only allow json file, numpy file and csv file if fileType != JSONFileExt && fileType != NumpyFileExt && fileType != CSVFileExt { log.Warn("import wrapper: unsupported file type", zap.String("filePath", filePath)) - return false, fmt.Errorf("unsupported file type: '%s'", filePath) + return false, merr.WrapErrImportFailed(fmt.Sprintf("unsupported file type: '%s'", filePath)) } // we use the first file to determine row-based or column-based @@ -203,12 +204,12 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) { if rowBased { if fileType != JSONFileExt && fileType != CSVFileExt { log.Warn("import wrapper: unsupported file type for row-based mode", zap.String("filePath", filePath)) - return rowBased, fmt.Errorf("unsupported file type for row-based mode: '%s'", filePath) + return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("unsupported file type for row-based mode: '%s'", filePath)) } } else { if fileType != NumpyFileExt { log.Warn("import wrapper: unsupported file type for column-based mode", zap.String("filePath", filePath)) - return rowBased, fmt.Errorf("unsupported file type for column-based mode: '%s'", filePath) + return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("unsupported file type for column-based mode: '%s'", filePath)) } } @@ -216,7 +217,7 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) { _, ok := fileNames[name] if ok { log.Warn("import wrapper: duplicate file name", zap.String("filePath", filePath)) - return rowBased, fmt.Errorf("duplicate file: '%s'", filePath) + return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("duplicate file: '%s'", filePath)) } fileNames[name] = struct{}{} @@ -224,20 +225,20 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) { size, err := p.chunkManager.Size(p.ctx, filePath) if err != nil { log.Warn("import wrapper: failed to get file size", zap.String("filePath", filePath), zap.Error(err)) - return rowBased, fmt.Errorf("failed to get file size of '%s', error:%w", filePath, err) + return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("failed to get file size of '%s', error:%v", filePath, err)) } // empty file if size == 0 { log.Warn("import wrapper: file size is zero", zap.String("filePath", filePath)) - return rowBased, fmt.Errorf("the file '%s' size is zero", filePath) + return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("the file '%s' size is zero", filePath)) } if size > params.Params.CommonCfg.ImportMaxFileSize.GetAsInt64() { log.Warn("import wrapper: file size exceeds the maximum size", zap.String("filePath", filePath), zap.Int64("fileSize", size), zap.String("MaxFileSize", params.Params.CommonCfg.ImportMaxFileSize.GetValue())) - return rowBased, fmt.Errorf("the file '%s' size exceeds the maximum size: %s bytes", - filePath, params.Params.CommonCfg.ImportMaxFileSize.GetValue()) + return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("the file '%s' size exceeds the maximum size: %s bytes", + filePath, params.Params.CommonCfg.ImportMaxFileSize.GetValue())) } totalSize += size } @@ -567,7 +568,7 @@ func (p *ImportWrapper) flushFunc(fields BlockData, shardID int, partitionID int if err != nil { logFields = append(logFields, zap.Error(err)) log.Warn("import wrapper: failed to assign a new segment", logFields...) - return fmt.Errorf("failed to assign a new segment for shard id %d, error: %w", shardID, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to assign a new segment for shard id %d, error: %v", shardID, err)) } segment = &WorkingSegment{ @@ -589,8 +590,8 @@ func (p *ImportWrapper) flushFunc(fields BlockData, shardID int, partitionID int logFields = append(logFields, zap.Error(err), zap.Int64("segmentID", segment.segmentID), zap.String("targetChannel", segment.targetChName)) log.Warn("import wrapper: failed to save binlogs", logFields...) - return fmt.Errorf("failed to save binlogs, shard id %d, segment id %d, channel '%s', error: %w", - shardID, segment.segmentID, segment.targetChName, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to save binlogs, shard id %d, segment id %d, channel '%s', error: %v", + shardID, segment.segmentID, segment.targetChName, err)) } segment.fieldsInsert = append(segment.fieldsInsert, fieldsInsert...) @@ -630,8 +631,8 @@ func (p *ImportWrapper) closeWorkingSegment(segment *WorkingSegment) error { if err != nil { logFields = append(logFields, zap.Error(err)) log.Warn("import wrapper: failed to seal segment", logFields...) - return fmt.Errorf("failed to seal segment, shard id %d, segment id %d, channel '%s', error: %w", - segment.shardID, segment.segmentID, segment.targetChName, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to seal segment, shard id %d, segment id %d, channel '%s', error: %v", + segment.shardID, segment.segmentID, segment.targetChName, err)) } return nil diff --git a/internal/util/importutil/import_wrapper_test.go b/internal/util/importutil/import_wrapper_test.go index 8f08cd1437036..7ec785e8e3eaf 100644 --- a/internal/util/importutil/import_wrapper_test.go +++ b/internal/util/importutil/import_wrapper_test.go @@ -250,11 +250,11 @@ func Test_ImportWrapperRowBased(t *testing.T) { content := []byte(`{ "rows":[ - {"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": {"x": 2}, "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 7, "b": true}}, - {"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{\"k\": 2.5}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4], "FieldJSON": {"a": 8, "b": 2}}, - {"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": {"y": "hello"}, "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4], "FieldJSON": {"a": 9, "b": false}}, - {"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4], "FieldJSON": {"a": 10, "b": 2.15}}, - {"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4], "FieldJSON": {"a": 11, "b": "s"}} + {"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": {"x": 2}, "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 7, "b": true}, "FieldArray": [1, 2, 3, 4]}, + {"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{\"k\": 2.5}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4], "FieldJSON": {"a": 8, "b": 2}, "FieldArray": [5, 6, 7, 8]}, + {"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": {"y": "hello"}, "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4], "FieldJSON": {"a": 9, "b": false}, "FieldArray": [11, 22, 33, 44]}, + {"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4], "FieldJSON": {"a": 10, "b": 2.15}, "FieldArray": [10, 12, 13, 14]}, + {"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4], "FieldJSON": {"a": 11, "b": "s"}, "FieldArray": [21, 22, 23, 24]} ] }`) @@ -342,10 +342,10 @@ func Test_ImportWrapperRowBased_CSV(t *testing.T) { idAllocator := newIDAllocator(ctx, t, nil) content := []byte( - `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector - true,10,101,1001,10001,3.14,1.56,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]" - false,11,102,1002,10002,3.15,1.57,No.1,"{""x"": 1}","[201,0]","[0.1,0.2,0.3,0.4]" - true,12,103,1003,10003,3.16,1.58,No.2,"{""x"": 2}","[202,0]","[0.1,0.2,0.3,0.4]"`) + `FieldBool,FieldInt8,FieldInt16,FieldInt32,FieldInt64,FieldFloat,FieldDouble,FieldString,FieldJSON,FieldBinaryVector,FieldFloatVector,FieldArray + true,10,101,1001,10001,3.14,1.56,No.0,"{""x"": 0}","[200,0]","[0.1,0.2,0.3,0.4]","[1,2,3,4]" + false,11,102,1002,10002,3.15,1.57,No.1,"{""x"": 1}","[201,0]","[0.1,0.2,0.3,0.4]","[5,6,7,8]" + true,12,103,1003,10003,3.16,1.58,No.2,"{""x"": 2}","[202,0]","[0.1,0.2,0.3,0.4]","[9,10,11,12]"`) filePath := TempFilesPath + "rows_1.csv" err = cm.Write(ctx, filePath, content) @@ -443,7 +443,8 @@ func Test_ImportWrapperColumnBased_numpy(t *testing.T) { reportFunc := func(res *rootcoordpb.ImportResult) error { return nil } - collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1}) + schema := createNumpySchema() + collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1}) assert.NoError(t, err) files := createSampleNumpyFiles(t, cm) @@ -741,11 +742,11 @@ func Test_ImportWrapperReportFailRowBased(t *testing.T) { content := []byte(`{ "rows":[ - {"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": "{\"x\": \"aaa\"}", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 9, "b": false}}, - {"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4], "FieldJSON": {"a": 9, "b": false}}, - {"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": "{\"x\": 2, \"y\": 5}", "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4], "FieldJSON": {"a": 9, "b": false}}, - {"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4], "FieldJSON": {"a": 9, "b": false}}, - {"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4], "FieldJSON": {"a": 9, "b": false}} + {"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": "{\"x\": \"aaa\"}", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 9, "b": false}, "FieldArray": [1, 2, 3, 4]}, + {"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4], "FieldJSON": {"a": 9, "b": false}, "FieldArray": [1, 2, 3, 4]}, + {"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": "{\"x\": 2, \"y\": 5}", "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4], "FieldJSON": {"a": 9, "b": false}, "FieldArray": [1, 2, 3, 4]}, + {"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4], "FieldJSON": {"a": 9, "b": false}, "FieldArray": [1, 2, 3, 4]}, + {"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4], "FieldJSON": {"a": 9, "b": false}, "FieldArray": [1, 2, 3, 4]} ] }`) @@ -817,7 +818,7 @@ func Test_ImportWrapperReportFailColumnBased_numpy(t *testing.T) { reportFunc := func(res *rootcoordpb.ImportResult) error { return nil } - collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1}) + collectionInfo, err := NewCollectionInfo(createNumpySchema(), 2, []int64{1}) assert.NoError(t, err) wrapper := NewImportWrapper(ctx, collectionInfo, 1, ReadBufferSize, idAllocator, cm, importResult, reportFunc) wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc) diff --git a/internal/util/importutil/json_handler.go b/internal/util/importutil/json_handler.go index eec84d3d849ac..c9afdc3cfa65c 100644 --- a/internal/util/importutil/json_handler.go +++ b/internal/util/importutil/json_handler.go @@ -22,13 +22,13 @@ import ( "fmt" "strconv" - "github.com/cockroachdb/errors" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -43,14 +43,14 @@ func getKeyValue(obj interface{}, fieldName string, isString bool) (string, erro if value, ok := obj.(string); ok { return value, nil } - return "", fmt.Errorf("illegal value '%v' for varchar type key field '%s'", obj, fieldName) + return "", merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for varchar type key field '%s'", obj, fieldName)) } // int64 type primary field, the value must be json.Number if num, ok := obj.(json.Number); ok { return string(num), nil } - return "", fmt.Errorf("illegal value '%v' for int64 type key field '%s'", obj, fieldName) + return "", merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int64 type key field '%s'", obj, fieldName)) } // JSONRowConsumer is row-based json format consumer class @@ -75,7 +75,7 @@ func NewJSONRowConsumer(ctx context.Context, ) (*JSONRowConsumer, error) { if collectionInfo == nil { log.Warn("JSON row consumer: collection schema is nil") - return nil, errors.New("collection schema is nil") + return nil, merr.WrapErrImportFailed("collection schema is nil") } v := &JSONRowConsumer{ @@ -92,7 +92,7 @@ func NewJSONRowConsumer(ctx context.Context, err := initValidators(collectionInfo.Schema, v.validators) if err != nil { log.Warn("JSON row consumer: fail to initialize json row-based consumer", zap.Error(err)) - return nil, fmt.Errorf("fail to initialize json row-based consumer, error: %w", err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("fail to initialize json row-based consumer, error: %v", err)) } v.shardsData = make([]ShardData, 0, collectionInfo.ShardNum) @@ -100,7 +100,7 @@ func NewJSONRowConsumer(ctx context.Context, shardData := initShardData(collectionInfo.Schema, collectionInfo.PartitionIDs) if shardData == nil { log.Warn("JSON row consumer: fail to initialize in-memory segment data", zap.Int("shardID", i)) - return nil, fmt.Errorf("fail to initialize in-memory segment data for shard id %d", i) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("fail to initialize in-memory segment data for shard id %d", i)) } v.shardsData = append(v.shardsData, shardData) } @@ -108,7 +108,7 @@ func NewJSONRowConsumer(ctx context.Context, // primary key is autoid, id generator is required if v.collectionInfo.PrimaryKey.GetAutoID() && idAlloc == nil { log.Warn("JSON row consumer: ID allocator is nil") - return nil, errors.New("ID allocator is nil") + return nil, merr.WrapErrImportFailed("ID allocator is nil") } return v, nil @@ -125,7 +125,7 @@ func (v *JSONRowConsumer) RowCount() int64 { func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if v == nil || v.validators == nil || len(v.validators) == 0 { log.Warn("JSON row consumer is not initialized") - return errors.New("JSON row consumer is not initialized") + return merr.WrapErrImportFailed("JSON row consumer is not initialized") } // if rows is nil, that means read to end of file, force flush all data @@ -141,7 +141,7 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { err := tryFlushBlocks(v.ctx, v.shardsData, v.collectionInfo.Schema, v.callFlushFunc, v.blockSize, MaxTotalSizeInMemory, false) if err != nil { log.Warn("JSON row consumer: try flush data but failed", zap.Error(err)) - return fmt.Errorf("try flush data but failed, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("try flush data but failed, error: %v", err)) } // prepare autoid, no matter int64 or varchar pk, we always generate autoid since the hidden field RowIDField requires them @@ -152,18 +152,18 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if primaryValidator.autoID { if v.rowIDAllocator == nil { log.Warn("JSON row consumer: primary keys is auto-generated but IDAllocator is nil") - return fmt.Errorf("primary keys is auto-generated but IDAllocator is nil") + return merr.WrapErrImportFailed("primary keys is auto-generated but IDAllocator is nil") } var err error rowIDBegin, rowIDEnd, err = v.rowIDAllocator.Alloc(uint32(len(rows))) if err != nil { log.Warn("JSON row consumer: failed to generate primary keys", zap.Int("count", len(rows)), zap.Error(err)) - return fmt.Errorf("failed to generate %d primary keys, error: %w", len(rows), err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to generate %d primary keys, error: %v", len(rows), err)) } if rowIDEnd-rowIDBegin != int64(len(rows)) { log.Warn("JSON row consumer: try to generate primary keys but allocated ids are not enough", zap.Int("count", len(rows)), zap.Int64("generated", rowIDEnd-rowIDBegin)) - return fmt.Errorf("try to generate %d primary keys but only %d keys were allocated", len(rows), rowIDEnd-rowIDBegin) + return merr.WrapErrImportFailed(fmt.Sprintf("try to generate %d primary keys but only %d keys were allocated", len(rows), rowIDEnd-rowIDBegin)) } log.Info("JSON row consumer: auto-generate primary keys", zap.Int64("begin", rowIDBegin), zap.Int64("end", rowIDEnd)) if !primaryValidator.isString { @@ -183,7 +183,7 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if primaryValidator.isString { if primaryValidator.autoID { log.Warn("JSON row consumer: string type primary key cannot be auto-generated") - return errors.New("string type primary key cannot be auto-generated") + return merr.WrapErrImportFailed("string type primary key cannot be auto-generated") } value := row[primaryKeyID] @@ -191,7 +191,7 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if err != nil { log.Warn("JSON row consumer: failed to parse primary key at the row", zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to parse primary key at the row %d, error: %w", rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key at the row %d, error: %v", rowNumber, err)) } // hash to shard based on pk, hash to partition if partition key exist @@ -215,7 +215,7 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if err != nil { log.Warn("JSON row consumer: failed to parse primary key at the row", zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to parse primary key at the row %d, error: %w", rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key at the row %d, error: %v", rowNumber, err)) } // parse the pk from a string @@ -223,8 +223,8 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if err != nil { log.Warn("JSON row consumer: failed to parse primary key at the row", zap.String("value", strValue), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to parse primary key '%s' at the row %d, error: %w", - strValue, rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key '%s' at the row %d, error: %v", + strValue, rowNumber, err)) } } @@ -232,7 +232,7 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if err != nil { log.Warn("JSON row consumer: failed to hash primary key at the row", zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to hash primary key %d at the row %d, error: %w", pk, rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to hash primary key %d at the row %d, error: %v", pk, rowNumber, err)) } // hash to shard based on pk, hash to partition if partition key exist @@ -259,8 +259,8 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { if err := validator.convertFunc(value, v.shardsData[shard][partitionID][fieldID]); err != nil { log.Warn("JSON row consumer: failed to convert value for field at the row", zap.String("fieldName", validator.fieldName), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return fmt.Errorf("failed to convert value for field '%s' at the row %d, error: %w", - validator.fieldName, rowNumber, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert value for field '%s' at the row %d, error: %v", + validator.fieldName, rowNumber, err)) } } } @@ -275,7 +275,7 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error { func (v *JSONRowConsumer) hashToPartition(row map[storage.FieldID]interface{}, rowNumber int64) (int64, error) { if v.collectionInfo.PartitionKey == nil { if len(v.collectionInfo.PartitionIDs) != 1 { - return 0, fmt.Errorf("collection '%s' partition list is empty", v.collectionInfo.Schema.Name) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("collection '%s' partition list is empty", v.collectionInfo.Schema.Name)) } // no partition key, directly return the target partition id return v.collectionInfo.PartitionIDs[0], nil @@ -288,7 +288,7 @@ func (v *JSONRowConsumer) hashToPartition(row map[storage.FieldID]interface{}, r if err != nil { log.Warn("JSON row consumer: failed to parse partition key at the row", zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return 0, fmt.Errorf("failed to parse partition key at the row %d, error: %w", rowNumber, err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse partition key at the row %d, error: %v", rowNumber, err)) } var hashValue uint32 @@ -300,15 +300,15 @@ func (v *JSONRowConsumer) hashToPartition(row map[storage.FieldID]interface{}, r if err != nil { log.Warn("JSON row consumer: failed to parse partition key at the row", zap.String("value", strValue), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return 0, fmt.Errorf("failed to parse partition key '%s' at the row %d, error: %w", - strValue, rowNumber, err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse partition key '%s' at the row %d, error: %v", + strValue, rowNumber, err)) } hashValue, err = typeutil.Hash32Int64(pk) if err != nil { log.Warn("JSON row consumer: failed to hash partition key at the row", zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err)) - return 0, fmt.Errorf("failed to hash partition key %d at the row %d, error: %w", pk, rowNumber, err) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to hash partition key %d at the row %d, error: %v", pk, rowNumber, err)) } } diff --git a/internal/util/importutil/json_parser.go b/internal/util/importutil/json_parser.go index 06df18a023d38..a946b1bfb56ec 100644 --- a/internal/util/importutil/json_parser.go +++ b/internal/util/importutil/json_parser.go @@ -23,13 +23,13 @@ import ( "io" "strings" - "github.com/cockroachdb/errors" "go.uber.org/zap" "golang.org/x/exp/maps" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -115,7 +115,7 @@ func (p *JSONParser) combineDynamicRow(dynamicValues map[string]interface{}, row err := desc.Decode(&mp) if err != nil { // invalid input - return errors.New("illegal value for dynamic field, not a JSON format string") + return merr.WrapErrImportFailed("illegal value for dynamic field, not a JSON format string") } maps.Copy(dynamicValues, mp) @@ -124,7 +124,7 @@ func (p *JSONParser) combineDynamicRow(dynamicValues map[string]interface{}, row maps.Copy(dynamicValues, mp) } else { // invalid input - return errors.New("illegal value for dynamic field, not a JSON object") + return merr.WrapErrImportFailed("illegal value for dynamic field, not a JSON object") } row[dynamicFieldID] = dynamicValues } @@ -146,7 +146,7 @@ func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{} stringMap, ok := raw.(map[string]interface{}) if !ok { log.Warn("JSON parser: invalid JSON format, each row should be a key-value map") - return nil, errors.New("invalid JSON format, each row should be a key-value map") + return nil, merr.WrapErrImportFailed("invalid JSON format, each row should be a key-value map") } dynamicValues := make(map[string]interface{}) @@ -157,7 +157,7 @@ func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{} if (fieldID == p.collectionInfo.PrimaryKey.GetFieldID()) && p.collectionInfo.PrimaryKey.GetAutoID() { // primary key is auto-id, no need to provide log.Warn("JSON parser: the primary key is auto-generated, no need to provide", zap.String("fieldName", k)) - return nil, fmt.Errorf("the primary key '%s' is auto-generated, no need to provide", k) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("the primary key '%s' is auto-generated, no need to provide", k)) } if ok { @@ -168,7 +168,7 @@ func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{} } else { // no dynamic field. if user provided redundant field, return error log.Warn("JSON parser: the field is not defined in collection schema", zap.String("fieldName", k)) - return nil, fmt.Errorf("the field '%s' is not defined in collection schema", k) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("the field '%s' is not defined in collection schema", k)) } } @@ -189,7 +189,7 @@ func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{} if !ok { // not auto-id primary key, no dynamic field, must provide value log.Warn("JSON parser: a field value is missed", zap.String("fieldName", k)) - return nil, fmt.Errorf("value of field '%s' is missed", k) + return nil, merr.WrapErrImportFailed(fmt.Sprintf("value of field '%s' is missed", k)) } } } @@ -207,7 +207,7 @@ func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{} func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error { if handler == nil || reader == nil { log.Warn("JSON parse handler is nil") - return errors.New("JSON parse handler is nil") + return merr.WrapErrImportFailed("JSON parse handler is nil") } dec := json.NewDecoder(reader.r) @@ -232,42 +232,42 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error { t, err := dec.Token() if err != nil { log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err)) - return fmt.Errorf("failed to decode the JSON file, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err)) } if t != json.Delim('{') && t != json.Delim('[') { log.Warn("JSON parser: invalid JSON format, the content should be started with '{' or '['") - return errors.New("invalid JSON format, the content should be started with '{' or '['") + return merr.WrapErrImportFailed("invalid JSON format, the content should be started with '{' or '['") } // read the first level isEmpty := true - isOldFormat := (t == json.Delim('{')) + isOldFormat := t == json.Delim('{') for dec.More() { if isOldFormat { // read the key t, err := dec.Token() if err != nil { log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err)) - return fmt.Errorf("failed to decode the JSON file, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err)) } key := t.(string) keyLower := strings.ToLower(key) // the root key should be RowRootNode if keyLower != RowRootNode { log.Warn("JSON parser: invalid JSON format, the root key is not found", zap.String("RowRootNode", RowRootNode), zap.String("key", key)) - return fmt.Errorf("invalid JSON format, the root key should be '%s', but get '%s'", RowRootNode, key) + return merr.WrapErrImportFailed(fmt.Sprintf("invalid JSON format, the root key should be '%s', but get '%s'", RowRootNode, key)) } // started by '[' t, err = dec.Token() if err != nil { log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err)) - return fmt.Errorf("failed to decode the JSON file, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err)) } if t != json.Delim('[') { log.Warn("JSON parser: invalid JSON format, rows list should begin with '['") - return errors.New("invalid JSON format, rows list should begin with '['") + return merr.WrapErrImportFailed("invalid JSON format, rows list should begin with '['") } } @@ -277,7 +277,7 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error { var value interface{} if err := dec.Decode(&value); err != nil { log.Warn("JSON parser: failed to parse row value", zap.Error(err)) - return fmt.Errorf("failed to parse row value, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse row value, error: %v", err)) } row, err := p.verifyRow(value) @@ -292,7 +292,7 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error { isEmpty = false if err = handler.Handle(buf); err != nil { log.Warn("JSON parser: failed to convert row value to entity", zap.Error(err)) - return fmt.Errorf("failed to convert row value to entity, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert row value to entity, error: %v", err)) } // clear the buffer @@ -305,7 +305,7 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error { isEmpty = false if err = handler.Handle(buf); err != nil { log.Warn("JSON parser: failed to convert row value to entity", zap.Error(err)) - return fmt.Errorf("failed to convert row value to entity, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert row value to entity, error: %v", err)) } } @@ -313,18 +313,18 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error { t, err = dec.Token() if err != nil { log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err)) - return fmt.Errorf("failed to decode the JSON file, error: %w", err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err)) } if t != json.Delim(']') { log.Warn("JSON parser: invalid JSON format, rows list should end with a ']'") - return errors.New("invalid JSON format, rows list should end with a ']'") + return merr.WrapErrImportFailed("invalid JSON format, rows list should end with a ']'") } // outside context might be canceled(service stop, or future enhancement for canceling import task) if isCanceled(p.ctx) { log.Warn("JSON parser: import task was canceled") - return errors.New("import task was canceled") + return merr.WrapErrImportFailed("import task was canceled") } // nolint diff --git a/internal/util/importutil/json_parser_test.go b/internal/util/importutil/json_parser_test.go index 3a18a1e03b6c3..a8aae5671528f 100644 --- a/internal/util/importutil/json_parser_test.go +++ b/internal/util/importutil/json_parser_test.go @@ -112,6 +112,7 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) { FieldJSON: fmt.Sprintf("{\"x\": %d}", i), FieldBinaryVector: []int{(200 + i) % math.MaxUint8, 0}, FieldFloatVector: []float32{float32(i) + 0.1, float32(i) + 0.2, float32(i) + 0.3, float32(i) + 0.4}, + FieldArray: []int32{1, 2, 3}, } content.Rows = append(content.Rows, row) } @@ -187,6 +188,17 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) { assert.NoError(t, err) assert.InDelta(t, contenctRow.FieldFloatVector[k], float32(fval), 10e-6) } + + v11, ok := parsedRow[113].([]interface{}) + assert.True(t, ok) + assert.Equal(t, len(contenctRow.FieldArray), len(v11)) + for k := 0; k < len(v11); k++ { + val, ok := v11[k].(json.Number) + assert.True(t, ok) + ival, err := strconv.ParseInt(string(val), 0, 32) + assert.NoError(t, err) + assert.Equal(t, contenctRow.FieldArray[k], int32(ival)) + } } } diff --git a/internal/util/importutil/numpy_adapter.go b/internal/util/importutil/numpy_adapter.go index 8ebe1c4d6eee5..f1a049d576117 100644 --- a/internal/util/importutil/numpy_adapter.go +++ b/internal/util/importutil/numpy_adapter.go @@ -27,7 +27,6 @@ import ( "strconv" "unicode/utf8" - "github.com/cockroachdb/errors" "github.com/sbinet/npyio" "github.com/sbinet/npyio/npy" "go.uber.org/zap" @@ -35,6 +34,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/merr" ) var ( @@ -139,7 +139,7 @@ func convertNumpyType(typeStr string) (schemapb.DataType, error) { return schemapb.DataType_VarChar, nil } log.Warn("Numpy adapter: the numpy file data type is not supported", zap.String("dtype", typeStr)) - return schemapb.DataType_None, fmt.Errorf("the numpy file dtype '%s' is not supported", typeStr) + return schemapb.DataType_None, merr.WrapErrImportFailed(fmt.Sprintf("the numpy file dtype '%s' is not supported", typeStr)) } } @@ -182,7 +182,7 @@ func stringLen(dtype string) (int, bool, error) { } log.Warn("Numpy adapter: the numpy file dtype is not varchar data type", zap.String("dtype", dtype)) - return 0, false, fmt.Errorf("dtype '%s' of numpy file is not varchar data type", dtype) + return 0, false, merr.WrapErrImportFailed(fmt.Sprintf("dtype '%s' of numpy file is not varchar data type", dtype)) } func isStringType(typeStr string) bool { @@ -255,13 +255,13 @@ func (n *NumpyAdapter) checkCount(count int) int { func (n *NumpyAdapter) ReadBool(count int) ([]bool, error) { if count <= 0 { log.Warn("Numpy adapter: cannot read bool data with a zero or nagative count") - return nil, errors.New("cannot read bool data with a zero or nagative count") + return nil, merr.WrapErrImportFailed("cannot read bool data with a zero or nagative count") } // incorrect type if n.dataType != schemapb.DataType_Bool { log.Warn("Numpy adapter: numpy data is not bool type") - return nil, errors.New("numpy data is not bool type") + return nil, merr.WrapErrImportFailed("numpy data is not bool type") } // avoid read overflow @@ -277,7 +277,7 @@ func (n *NumpyAdapter) ReadBool(count int) ([]bool, error) { err := binary.Read(n.reader, n.order, &data) if err != nil { log.Warn("Numpy adapter: failed to read bool data", zap.Int("count", count), zap.Error(err)) - return nil, fmt.Errorf(" failed to read bool data with count %d, error: %w", readSize, err) + return nil, merr.WrapErrImportFailed(fmt.Sprintf(" failed to read bool data with count %d, error: %v", readSize, err)) } // update read position after successfully read @@ -289,7 +289,7 @@ func (n *NumpyAdapter) ReadBool(count int) ([]bool, error) { func (n *NumpyAdapter) ReadUint8(count int) ([]uint8, error) { if count <= 0 { log.Warn("Numpy adapter: cannot read uint8 data with a zero or nagative count") - return nil, errors.New("cannot read uint8 data with a zero or nagative count") + return nil, merr.WrapErrImportFailed("cannot read uint8 data with a zero or nagative count") } // incorrect type @@ -298,7 +298,7 @@ func (n *NumpyAdapter) ReadUint8(count int) ([]uint8, error) { case "u1", " rowCount { @@ -747,7 +747,7 @@ func (p *NumpyParser) checkRowCount(fieldsData BlockData) (int, error) { if count != rowCount { log.Warn("Numpy parser: field row count is not equal to other fields row count", zap.String("fieldName", name), zap.Int("rowCount", count), zap.Int("otherRowCount", rowCount)) - return 0, fmt.Errorf("field '%s' row count %d is not equal to other fields row count: %d", name, count, rowCount) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("field '%s' row count %d is not equal to other fields row count: %d", name, count, rowCount)) } } @@ -758,13 +758,13 @@ func (p *NumpyParser) checkRowCount(fieldsData BlockData) (int, error) { func (p *NumpyParser) splitFieldsData(fieldsData BlockData, shards []ShardData) error { if len(fieldsData) == 0 { log.Warn("Numpy parser: fields data to split is empty") - return fmt.Errorf("fields data to split is empty") + return merr.WrapErrImportFailed("fields data to split is empty") } if len(shards) != int(p.collectionInfo.ShardNum) { log.Warn("Numpy parser: block count is not equal to collection shard number", zap.Int("shardsLen", len(shards)), zap.Int32("shardNum", p.collectionInfo.ShardNum)) - return fmt.Errorf("block count %d is not equal to collection shard number %d", len(shards), p.collectionInfo.ShardNum) + return merr.WrapErrImportFailed(fmt.Sprintf("block count %d is not equal to collection shard number %d", len(shards), p.collectionInfo.ShardNum)) } rowCount, err := p.checkRowCount(fieldsData) @@ -776,7 +776,7 @@ func (p *NumpyParser) splitFieldsData(fieldsData BlockData, shards []ShardData) rowIDBegin, rowIDEnd, err := p.rowIDAllocator.Alloc(uint32(rowCount)) if err != nil { log.Warn("Numpy parser: failed to alloc row ID", zap.Int("rowCount", rowCount), zap.Error(err)) - return fmt.Errorf("failed to alloc %d rows ID, error: %w", rowCount, err) + return merr.WrapErrImportFailed(fmt.Sprintf("failed to alloc %d rows ID, error: %v", rowCount, err)) } rowIDField, ok := fieldsData[common.RowIDField] @@ -797,7 +797,7 @@ func (p *NumpyParser) splitFieldsData(fieldsData BlockData, shards []ShardData) log.Info("Numpy parser: generating auto-id", zap.Int("rowCount", rowCount), zap.Int64("rowIDBegin", rowIDBegin)) if primaryKey.GetDataType() != schemapb.DataType_Int64 { log.Warn("Numpy parser: primary key field is auto-generated but the field type is not int64") - return fmt.Errorf("primary key field is auto-generated but the field type is not int64") + return merr.WrapErrImportFailed("primary key field is auto-generated but the field type is not int64") } primaryDataArr := &storage.Int64FieldData{ @@ -815,7 +815,7 @@ func (p *NumpyParser) splitFieldsData(fieldsData BlockData, shards []ShardData) primaryData, ok := fieldsData[primaryKey.GetFieldID()] if !ok || primaryData.RowNum() <= 0 { log.Warn("Numpy parser: primary key field is not provided", zap.String("keyName", primaryKey.GetName())) - return fmt.Errorf("primary key '%s' field data is not provided", primaryKey.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("primary key '%s' field data is not provided", primaryKey.GetName())) } // prepare append functions @@ -855,8 +855,8 @@ func (p *NumpyParser) splitFieldsData(fieldsData BlockData, shards []ShardData) log.Warn("Numpy parser: cannot append data since source or target field data is nil", zap.String("FieldName", schema.GetName()), zap.Bool("sourceNil", srcData == nil), zap.Bool("targetNil", targetData == nil)) - return fmt.Errorf("cannot append data for field '%s', possibly no any fields corresponding to this numpy file, or a required numpy file is not provided", - schema.GetName()) + return merr.WrapErrImportFailed(fmt.Sprintf("cannot append data for field '%s', possibly no any fields corresponding to this numpy file, or a required numpy file is not provided", + schema.GetName())) } appendFunc := appendFunctions[schema.GetName()] err := appendFunc(srcData, i, targetData) @@ -869,13 +869,13 @@ func (p *NumpyParser) splitFieldsData(fieldsData BlockData, shards []ShardData) return nil } -// hashToPartition hash partition key to get an partition ID, return the first partition ID if no partition key exist +// hashToPartition hash partition key to get a partition ID, return the first partition ID if no partition key exist // CollectionInfo ensures only one partition ID in the PartitionIDs if no partition key exist func (p *NumpyParser) hashToPartition(fieldsData BlockData, rowNumber int) (int64, error) { if p.collectionInfo.PartitionKey == nil { // no partition key, directly return the target partition id if len(p.collectionInfo.PartitionIDs) != 1 { - return 0, fmt.Errorf("collection '%s' partition list is empty", p.collectionInfo.Schema.Name) + return 0, merr.WrapErrImportFailed(fmt.Sprintf("collection '%s' partition list is empty", p.collectionInfo.Schema.Name)) } return p.collectionInfo.PartitionIDs[0], nil } diff --git a/internal/util/importutil/numpy_parser_test.go b/internal/util/importutil/numpy_parser_test.go index be545fe48a0c4..4e20130274d9d 100644 --- a/internal/util/importutil/numpy_parser_test.go +++ b/internal/util/importutil/numpy_parser_test.go @@ -44,9 +44,21 @@ func createLocalChunkManager(t *testing.T) storage.ChunkManager { return cm } +func createNumpySchema() *schemapb.CollectionSchema { + schema := sampleSchema() + fields := make([]*schemapb.FieldSchema, 0) + for _, field := range schema.GetFields() { + if field.GetDataType() != schemapb.DataType_Array { + fields = append(fields, field) + } + } + schema.Fields = fields + return schema +} + func createNumpyParser(t *testing.T) *NumpyParser { ctx := context.Background() - schema := sampleSchema() + schema := createNumpySchema() idAllocator := newIDAllocator(ctx, t, nil) cm := createLocalChunkManager(t) @@ -660,7 +672,7 @@ func Test_NumpyParserPrepareAppendFunctions(t *testing.T) { // succeed appendFuncs, err := parser.prepareAppendFunctions() assert.NoError(t, err) - assert.Equal(t, len(sampleSchema().Fields), len(appendFuncs)) + assert.Equal(t, len(createNumpySchema().Fields), len(appendFuncs)) // schema has unsupported data type schema := &schemapb.CollectionSchema{ @@ -868,7 +880,7 @@ func Test_NumpyParserSplitFieldsData(t *testing.T) { }) t.Run("primary key auto-generated", func(t *testing.T) { - parser.collectionInfo.resetSchema(sampleSchema()) + parser.collectionInfo.resetSchema(createNumpySchema()) schema := findSchema(parser.collectionInfo.Schema, schemapb.DataType_Int64) schema.AutoID = true @@ -982,7 +994,7 @@ func Test_NumpyParserConsume(t *testing.T) { files := createSampleNumpyFiles(t, cm) readers, err := parser.createReaders(files) assert.NoError(t, err) - assert.Equal(t, len(sampleSchema().Fields), len(readers)) + assert.Equal(t, len(createNumpySchema().Fields), len(readers)) // succeed err = parser.consume(readers) @@ -1043,7 +1055,7 @@ func Test_NumpyParserParse(t *testing.T) { assert.Error(t, err) }) - parser.collectionInfo.resetSchema(sampleSchema()) + parser.collectionInfo.resetSchema(createNumpySchema()) t.Run("succeed", func(t *testing.T) { cm := createLocalChunkManager(t) diff --git a/pkg/util/merr/errors.go b/pkg/util/merr/errors.go index 790fb20696d8e..bbea9ebacd235 100644 --- a/pkg/util/merr/errors.go +++ b/pkg/util/merr/errors.go @@ -142,6 +142,9 @@ var ( // Do NOT export this, // never allow programmer using this, keep only for converting unknown error to milvusError errUnexpected = newMilvusError("unexpected error", (1<<16)-1, false) + + // import + ErrImportFailed = newMilvusError("deserializing import data failed", 2100, false) ) type milvusError struct { diff --git a/pkg/util/merr/utils.go b/pkg/util/merr/utils.go index 1fd3afe3c2e02..fa0ac4f052265 100644 --- a/pkg/util/merr/utils.go +++ b/pkg/util/merr/utils.go @@ -873,3 +873,11 @@ func bound(name string, value, lower, upper any) boundField { func (f boundField) String() string { return fmt.Sprintf("%v out of range %v <= %s <= %v", f.value, f.lower, f.name, f.upper) } + +func WrapErrImportFailed(msg ...string) error { + err := error(ErrImportFailed) + if len(msg) > 0 { + err = errors.Wrap(err, strings.Join(msg, "->")) + } + return err +}