diff --git a/internal/util/importutil/binlog_adapter.go b/internal/util/importutil/binlog_adapter.go new file mode 100644 index 0000000000..5993479b7f --- /dev/null +++ b/internal/util/importutil/binlog_adapter.go @@ -0,0 +1,1093 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package importutil + +import ( + "encoding/json" + "errors" + "fmt" + "strconv" + "strings" + + "github.com/milvus-io/milvus/api/schemapb" + "github.com/milvus-io/milvus/internal/common" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/util/typeutil" + "go.uber.org/zap" +) + +// 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 +// This class do the following works: +// 1. read insert log of each field, then constructs map[storage.FieldID]storage.FieldData in memory. +// 2. read delta log to remove deleted entities(TimeStampField is used to apply or skip the operation). +// 3. split data according to shard number +// 4. call the callFlushFunc function to flush data into new segment if data size reaches segmentSize. +type BinlogAdapter struct { + collectionSchema *schemapb.CollectionSchema // collection schema + chunkManager storage.ChunkManager // storage interfaces to read binlog files + callFlushFunc ImportFlushFunc // call back function to flush segment + shardNum int32 // sharding number of the collection + segmentSize int64 // maximum size of a segment(unit:byte) + maxTotalSize int64 // maximum size of in-memory segments(unit:byte) + primaryKey storage.FieldID // id of primary key + primaryType schemapb.DataType // data type of primary key + + // a timestamp to define the end point of restore, data after this point will be ignored + // set this value to 0, all the data will be ignored + // set this value to math.MaxUint64, all the data will be imported + tsEndPoint uint64 +} + +func NewBinlogAdapter(collectionSchema *schemapb.CollectionSchema, + shardNum int32, + segmentSize int64, + maxTotalSize int64, + chunkManager storage.ChunkManager, + flushFunc ImportFlushFunc, + tsEndPoint uint64) (*BinlogAdapter, error) { + if collectionSchema == nil { + log.Error("Binlog adapter: collection schema is nil") + return nil, errors.New("collection schema is nil") + } + + if chunkManager == nil { + log.Error("Binlog adapter: chunk manager pointer is nil") + return nil, errors.New("chunk manager pointer is nil") + } + + if flushFunc == nil { + log.Error("Binlog adapter: flush function is nil") + return nil, errors.New("flush function is nil") + } + + adapter := &BinlogAdapter{ + collectionSchema: collectionSchema, + chunkManager: chunkManager, + callFlushFunc: flushFunc, + shardNum: shardNum, + segmentSize: segmentSize, + maxTotalSize: maxTotalSize, + tsEndPoint: tsEndPoint, + } + + // amend the segment size to avoid portential OOM risk + if adapter.segmentSize > MaxSegmentSizeInMemory { + adapter.segmentSize = MaxSegmentSizeInMemory + } + + // find out the primary key ID and its data type + adapter.primaryKey = -1 + for i := 0; i < len(collectionSchema.Fields); i++ { + schema := collectionSchema.Fields[i] + if schema.GetIsPrimaryKey() { + adapter.primaryKey = schema.GetFieldID() + adapter.primaryType = schema.GetDataType() + break + } + } + // primary key not found + if adapter.primaryKey == -1 { + log.Error("Binlog adapter: collection schema has no primary key") + return nil, errors.New("collection schema has no primary key") + } + + return adapter, nil +} + +func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error { + if segmentHolder == nil { + log.Error("Binlog adapter: segment files holder is nil") + return errors.New("segment files holder is nil") + } + + log.Info("Binlog adapter: read segment", zap.Int64("segmentID", segmentHolder.segmentID)) + + // step 1: verify the file count by collection schema + err := p.verify(segmentHolder) + if err != nil { + return err + } + + // step 2: read the delta log to prepare delete list, and combine lists into one dict + intDeletedList, strDeletedList, err := p.readDeltalogs(segmentHolder) + if err != nil { + return err + } + + // step 3: read binlog files batch by batch + // Assume the collection has 2 fields: a and b + // a has these binlog files: a_1, a_2, a_3 ... + // b has these binlog files: b_1, b_2, b_3 ... + // Then first round read a_1 and b_1, second round read a_2 and b_2, etc... + // deleted list will be used to remove deleted entities + // if accumulate data exceed segmentSize, call callFlushFunc to generate new segment + batchCount := 0 + for _, files := range segmentHolder.fieldFiles { + batchCount = len(files) + break + } + + // prepare FieldData list + segmentsData := make([]map[storage.FieldID]storage.FieldData, 0, p.shardNum) + for i := 0; i < int(p.shardNum); i++ { + segmentData := initSegmentData(p.collectionSchema) + if segmentData == nil { + log.Error("Binlog adapter: failed to initialize FieldData list") + return errors.New("failed to initialize FieldData list") + } + segmentsData = append(segmentsData, segmentData) + } + + // read binlog files batch by batch + for i := 0; i < batchCount; i++ { + // 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) + for fieldID, files := range segmentHolder.fieldFiles { + if fieldID == p.primaryKey || fieldID == common.TimeStampField { + continue + } + batchFiles[fieldID] = files[i] + } + log.Info("Binlog adapter: batch files to read", zap.Any("batchFiles", batchFiles)) + + // read primary keys firstly + primaryLog := segmentHolder.fieldFiles[p.primaryKey][i] // no need to check existence, already verified + log.Info("Binlog adapter: prepare to read primary key binglog", zap.Int64("pk", p.primaryKey), zap.String("logPath", primaryLog)) + intList, strList, err := p.readPrimaryKeys(primaryLog) + if err != nil { + return err + } + + // read timestamps list + timestampLog := segmentHolder.fieldFiles[common.TimeStampField][i] // no need to check existence, already verified + log.Info("Binlog adapter: prepare to read timestamp binglog", zap.Any("logPath", timestampLog)) + timestampList, err := p.readTimestamp(timestampLog) + if err != nil { + return err + } + + var shardList []int32 + if p.primaryType == schemapb.DataType_Int64 { + // calculate a shard num list by primary keys and deleted entities + shardList, err = p.getShardingListByPrimaryInt64(intList, timestampList, segmentsData, intDeletedList) + if err != nil { + return err + } + } else if p.primaryType == schemapb.DataType_VarChar { + // calculate a shard num list by primary keys and deleted entities + shardList, err = p.getShardingListByPrimaryVarchar(strList, timestampList, segmentsData, strDeletedList) + if err != nil { + return err + } + } else { + log.Error("Binlog adapter: unknow primary key type", zap.Int("type", int(p.primaryType))) + return errors.New("unknow primary key type") + } + + // if shardList is empty, that means all the primary keys have been deleted(or skipped), no need to read other files + if len(shardList) == 0 { + continue + } + + // read other insert logs and use the shardList to do sharding + for fieldID, file := range batchFiles { + err = p.readInsertlog(fieldID, file, segmentsData, shardList) + if err != nil { + return err + } + } + + // flush segment whose size exceed segmentSize + err = p.tryFlushSegments(segmentsData, false) + if err != nil { + return err + } + } + + // finally, force to flush + return p.tryFlushSegments(segmentsData, true) +} + +// This method verify the schema and binlog files +// 1. each field must has 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. +func (p *BinlogAdapter) verify(segmentHolder *SegmentFilesHolder) error { + if segmentHolder == nil { + log.Error("Binlog adapter: segment files holder is nil") + return errors.New("segment files holder is nil") + } + + firstFieldFileCount := 0 + // each field must has binlog file + for i := 0; i < len(p.collectionSchema.Fields); i++ { + schema := p.collectionSchema.Fields[i] + + files, ok := segmentHolder.fieldFiles[schema.FieldID] + if !ok { + log.Error("Binlog adapter: a field has no binlog file", zap.Int64("fieldID", schema.FieldID)) + return errors.New("the field " + strconv.Itoa(int(schema.FieldID)) + " has no binlog file") + } + + if i == 0 { + firstFieldFileCount = len(files) + } + } + + // the segmentHolder.fieldFiles need to contains RowIDField + _, ok := segmentHolder.fieldFiles[common.RowIDField] + if !ok { + log.Error("Binlog adapter: the binlog files of RowIDField is missed") + return errors.New("the binlog files of RowIDField is missed") + } + + // the segmentHolder.fieldFiles need to contains TimeStampField + _, ok = segmentHolder.fieldFiles[common.TimeStampField] + if !ok { + log.Error("Binlog adapter: the binlog files of TimeStampField is missed") + return errors.New("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.Error("Binlog adapter: file count of each field must be equal", zap.Int("firstFieldFileCount", firstFieldFileCount)) + return errors.New("binlog file count of each field must be equal") + } + } + + return nil +} + +// This method read data from deltalog, and convert to a dict +// The deltalog data is a list, to improve performance of next step, we convert it to a dict, +// key is the deleted ID, value is operation timestamp which is used to apply or skip the delete operation. +func (p *BinlogAdapter) readDeltalogs(segmentHolder *SegmentFilesHolder) (map[int64]uint64, map[string]uint64, error) { + deleteLogs, err := p.decodeDeleteLogs(segmentHolder) + if err != nil { + return nil, nil, err + } + + if len(deleteLogs) == 0 { + log.Info("Binlog adapter: no deletion for segment", zap.Int64("segmentID", segmentHolder.segmentID)) + return nil, nil, nil // no deletion + } + + if p.primaryType == schemapb.DataType_Int64 { + deletedIDDict := make(map[int64]uint64) + for _, deleteLog := range deleteLogs { + deletedIDDict[deleteLog.Pk.GetValue().(int64)] = deleteLog.Ts + } + log.Info("Binlog adapter: count of deleted entities", zap.Int("deletedCount", len(deletedIDDict))) + return deletedIDDict, nil, nil + } else if p.primaryType == schemapb.DataType_VarChar { + deletedIDDict := make(map[string]uint64) + for _, deleteLog := range deleteLogs { + deletedIDDict[deleteLog.Pk.GetValue().(string)] = deleteLog.Ts + } + log.Info("Binlog adapter: count of deleted entities", zap.Int("deletedCount", len(deletedIDDict))) + return nil, deletedIDDict, nil + } else { + log.Error("Binlog adapter: primary key is neither int64 nor varchar") + return nil, nil, errors.New("primary key is neither int64 nor varchar") + } +} + +// Decode string array(read from delta log) to storage.DeleteLog array +func (p *BinlogAdapter) decodeDeleteLogs(segmentHolder *SegmentFilesHolder) ([]*storage.DeleteLog, error) { + // step 1: read all delta logs to construct a string array, each string is marshaled from storage.DeleteLog + stringArray := make([]string, 0) + for _, deltalog := range segmentHolder.deltaFiles { + deltaStrings, err := p.readDeltalog(deltalog) + if err != nil { + return nil, err + } + stringArray = append(stringArray, deltaStrings...) + } + + if len(stringArray) == 0 { + return nil, nil // no delete log, return directly + } + + // print out the first deletion information for diagnose purpose + log.Info("Binlog adapter: total deletion count", zap.Int("count", len(stringArray)), zap.String("firstDeletion", stringArray[0])) + + // step 2: decode each string to a storage.DeleteLog object + deleteLogs := make([]*storage.DeleteLog, 0) + for i := 0; i < len(stringArray); i++ { + deleteLog, err := p.decodeDeleteLog(stringArray[i]) + if err != nil { + return nil, err + } + + // ignore deletions whose timestamp is larger than the tsEndPoint + if deleteLog.Ts <= p.tsEndPoint { + deleteLogs = append(deleteLogs, deleteLog) + } + } + log.Info("Binlog adapter: deletion count after filtering", zap.Int("count", len(deleteLogs))) + + // step 3: verify the current collection primary key type and the delete logs data type + for i := 0; i < len(deleteLogs); i++ { + if deleteLogs[i].PkType != int64(p.primaryType) { + log.Error("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(p.primaryType))) + return nil, errors.New("delta log data type is not equal to collection's primary key data type") + } + } + + return deleteLogs, nil +} + +// Decode a string to storage.DeleteLog +// Note: the following code is mainly come from data_codec.go, I suppose the code can compatible with old version 2.0 +func (p *BinlogAdapter) decodeDeleteLog(deltaStr string) (*storage.DeleteLog, error) { + deleteLog := &storage.DeleteLog{} + if err := json.Unmarshal([]byte(deltaStr), deleteLog); err != nil { + // compatible with versions that only support int64 type primary keys + // compatible with fmt.Sprintf("%d,%d", pk, ts) + // compatible error info (unmarshal err invalid character ',' after top-level value) + splits := strings.Split(deltaStr, ",") + if len(splits) != 2 { + log.Error("Binlog adapter: the format of deletion string is incorrect", zap.String("deltaStr", deltaStr)) + return nil, fmt.Errorf("the format of deletion string is incorrect, %v can not be split", deltaStr) + } + pk, err := strconv.ParseInt(splits[0], 10, 64) + if err != nil { + log.Error("Binlog adapter: failed to parse primary key of deletion string from old version", + zap.String("deltaStr", deltaStr), zap.Error(err)) + return nil, err + } + deleteLog.Pk = &storage.Int64PrimaryKey{ + Value: pk, + } + deleteLog.PkType = int64(schemapb.DataType_Int64) + deleteLog.Ts, err = strconv.ParseUint(splits[1], 10, 64) + if err != nil { + log.Error("Binlog adapter: failed to parse timestamp of deletion string from old version", + zap.String("deltaStr", deltaStr), zap.Error(err)) + return nil, err + } + } + + return deleteLog, nil +} + +// Each delta log data type is varchar, marshaled from an array of storage.DeleteLog objects. +func (p *BinlogAdapter) readDeltalog(logPath string) ([]string, error) { + // open the delta log file + binlogFile, err := NewBinlogFile(p.chunkManager) + if err != nil { + log.Error("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) + return nil, err + } + + err = binlogFile.Open(logPath) + if err != nil { + log.Error("Binlog adapter: failed to open delta log", zap.String("logPath", logPath), zap.Error(err)) + return nil, err + } + defer binlogFile.Close() + + // delta log type is varchar, return a string array(marshaled from an array of storage.DeleteLog objects) + data, err := binlogFile.ReadVarchar() + if err != nil { + log.Error("Binlog adapter: failed to read delta log", zap.String("logPath", logPath), zap.Error(err)) + return nil, err + } + log.Info("Binlog adapter: successfully read deltalog", zap.Int("deleteCount", len(data))) + + return data, nil +} + +// This method read data from int64 field, currently we use it to read the timestamp field. +func (p *BinlogAdapter) readTimestamp(logPath string) ([]int64, error) { + // open the log file + binlogFile, err := NewBinlogFile(p.chunkManager) + if err != nil { + log.Error("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) + return nil, err + } + + err = binlogFile.Open(logPath) + if err != nil { + log.Error("Binlog adapter: failed to open timestamp log file", zap.String("logPath", logPath)) + return nil, err + } + defer binlogFile.Close() + + // read int64 data + int64List, err := binlogFile.ReadInt64() + if err != nil { + log.Error("Binlog adapter: failed to read timestamp data from log file", zap.String("logPath", logPath)) + return nil, err + } + + log.Info("Binlog adapter: read timestamp from log file", zap.Int("tsCount", len(int64List))) + + return int64List, nil +} + +// This method read primary keys from insert log. +func (p *BinlogAdapter) readPrimaryKeys(logPath string) ([]int64, []string, error) { + // open the delta log file + binlogFile, err := NewBinlogFile(p.chunkManager) + if err != nil { + log.Error("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) + return nil, nil, err + } + + err = binlogFile.Open(logPath) + if err != nil { + log.Error("Binlog adapter: failed to open primary key binlog", zap.String("logPath", logPath)) + return nil, nil, err + } + defer binlogFile.Close() + + // primary key can be int64 or varchar, we need to handle the two cases + if p.primaryType == schemapb.DataType_Int64 { + idList, err := binlogFile.ReadInt64() + if err != nil { + log.Error("Binlog adapter: failed to read int64 primary key from binlog", zap.String("logPath", logPath), zap.Error(err)) + return nil, nil, err + } + log.Info("Binlog adapter: succeed to read int64 primary key binlog", zap.Int("len", len(idList))) + return idList, nil, nil + } else if p.primaryType == schemapb.DataType_VarChar { + idList, err := binlogFile.ReadVarchar() + if err != nil { + log.Error("Binlog adapter: failed to read varchar primary key from binlog", zap.String("logPath", logPath), zap.Error(err)) + return nil, nil, err + } + log.Info("Binlog adapter: succeed to read varchar primary key binlog", zap.Int("len", len(idList))) + return nil, idList, nil + } else { + log.Error("Binlog adapter: primary key is neither int64 nor varchar") + return nil, nil, errors.New("primary key is neither int64 nor varchar") + } +} + +// This method generate a shard id list by primary key(int64) list and deleted list. +// For example, an insert log has 10 rows, the no.3 and no.7 has been deleted, shardNum=2, the shardList could be: +// [0, 1, -1, 1, 0, 1, -1, 1, 0, 1] +// Compare timestampList with tsEndPoint to skip some rows. +func (p *BinlogAdapter) getShardingListByPrimaryInt64(primaryKeys []int64, + timestampList []int64, + memoryData []map[storage.FieldID]storage.FieldData, + intDeletedList map[int64]uint64) ([]int32, error) { + if len(timestampList) != len(primaryKeys) { + log.Error("Binlog adapter: primary key length is not equal to timestamp list length", + zap.Int("primaryKeysLen", len(primaryKeys)), zap.Int("timestampLen", len(timestampList))) + return nil, errors.New("primary key length is not equal to timestamp list length") + } + + log.Info("Binlog adapter: building shard list", zap.Int("pkLen", len(primaryKeys)), zap.Int("tsLen", len(timestampList))) + + actualDeleted := 0 + excluded := 0 + shardList := make([]int32, 0, len(primaryKeys)) + for i, key := range primaryKeys { + // if this entity's timestamp is greater than the tsEndPoint, set shardID = -1 to skip this entity + ts := timestampList[i] + if ts > int64(p.tsEndPoint) { + shardList = append(shardList, -1) + excluded++ + continue + } + + _, deleted := intDeletedList[key] + // if the key exists in intDeletedList, that means this entity has been deleted + if deleted { + shardList = append(shardList, -1) // this entity has been deleted, set shardID = -1 and skip this entity + actualDeleted++ + } else { + hash, _ := typeutil.Hash32Int64(key) + shardID := hash % uint32(p.shardNum) + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[p.primaryKey] // initSegmentData() can ensure the existence, no need to check here + + // append the entity to primary key's FieldData + field.(*storage.Int64FieldData).Data = append(field.(*storage.Int64FieldData).Data, key) + field.(*storage.Int64FieldData).NumRows[0]++ + + shardList = append(shardList, int32(shardID)) + } + } + log.Info("Binlog adapter: succeed to calculate a shard list", zap.Int("actualDeleted", actualDeleted), + zap.Int("excluded", excluded), zap.Int("len", len(shardList))) + + return shardList, nil +} + +// This method generate a shard id list by primary key(varchar) list and deleted list. +// For example, an insert log has 10 rows, the no.3 and no.7 has been deleted, shardNum=2, the shardList could be: +// [0, 1, -1, 1, 0, 1, -1, 1, 0, 1] +func (p *BinlogAdapter) getShardingListByPrimaryVarchar(primaryKeys []string, + timestampList []int64, + memoryData []map[storage.FieldID]storage.FieldData, + strDeletedList map[string]uint64) ([]int32, error) { + if len(timestampList) != len(primaryKeys) { + log.Error("Binlog adapter: primary key length is not equal to timestamp list length", + zap.Int("primaryKeysLen", len(primaryKeys)), zap.Int("timestampLen", len(timestampList))) + return nil, errors.New("primary key length is not equal to timestamp list length") + } + + log.Info("Binlog adapter: building shard list", zap.Int("pkLen", len(primaryKeys)), zap.Int("tsLen", len(timestampList))) + + actualDeleted := 0 + excluded := 0 + shardList := make([]int32, 0, len(primaryKeys)) + for i, key := range primaryKeys { + // if this entity's timestamp is greater than the tsEndPoint, set shardID = -1 to skip this entity + ts := timestampList[i] + if ts > int64(p.tsEndPoint) { + shardList = append(shardList, -1) + excluded++ + continue + } + + _, deleted := strDeletedList[key] + // if exists in strDeletedList, that means this entity has been deleted + if deleted { + shardList = append(shardList, -1) // this entity has been deleted, set shardID = -1 and skip this entity + actualDeleted++ + } else { + hash := typeutil.HashString2Uint32(key) + shardID := hash % uint32(p.shardNum) + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[p.primaryKey] // initSegmentData() can ensure the existence, no need to check existence here + + // append the entity to primary key's FieldData + field.(*storage.StringFieldData).Data = append(field.(*storage.StringFieldData).Data, key) + field.(*storage.StringFieldData).NumRows[0]++ + + shardList = append(shardList, int32(shardID)) + } + } + log.Info("Binlog adapter: succeed to calculate a shard list", zap.Int("actualDeleted", actualDeleted), + zap.Int("excluded", excluded), zap.Int("len", len(shardList))) + + return shardList, nil +} + +// This method read an insert log, and split the data into different shards according to a shard list +// The shardList is a list to tell which row belong to which shard, returned by getShardingListByPrimaryXXX() +// For deleted rows, we say its shard id is -1. +// For example, an insert log has 10 rows, the no.3 and no.7 has been deleted, shardNum=2, the shardList could be: +// [0, 1, -1, 1, 0, 1, -1, 1, 0, 1] +// This method put each row into different FieldData according to its shard id and field id, +// so, the no.1, no.5, no.9 will be put into shard_0 +// the no.2, no.4, no.6, no.8, no.10 will be put into shard_1 +// Note: the row count of insert log need to be equal to length of shardList +func (p *BinlogAdapter) readInsertlog(fieldID storage.FieldID, logPath string, + memoryData []map[storage.FieldID]storage.FieldData, shardList []int32) error { + // open the insert log file + binlogFile, err := NewBinlogFile(p.chunkManager) + if err != nil { + log.Error("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err)) + return err + } + + err = binlogFile.Open(logPath) + if err != nil { + log.Error("Binlog adapter: failed to open insert log", zap.String("logPath", logPath), zap.Error(err)) + return err + } + defer binlogFile.Close() + + // read data according to data type + switch binlogFile.DataType() { + case schemapb.DataType_Bool: + data, err := binlogFile.ReadBool() + if err != nil { + return err + } + + err = p.dispatchBoolToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_Int8: + data, err := binlogFile.ReadInt8() + if err != nil { + return err + } + + err = p.dispatchInt8ToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_Int16: + data, err := binlogFile.ReadInt16() + if err != nil { + return err + } + + err = p.dispatchInt16ToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_Int32: + data, err := binlogFile.ReadInt32() + if err != nil { + return err + } + + err = p.dispatchInt32ToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_Int64: + data, err := binlogFile.ReadInt64() + if err != nil { + return err + } + + err = p.dispatchInt64ToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_Float: + data, err := binlogFile.ReadFloat() + if err != nil { + return err + } + + err = p.dispatchFloatToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_Double: + data, err := binlogFile.ReadDouble() + if err != nil { + return err + } + + err = p.dispatchDoubleToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_String, schemapb.DataType_VarChar: + data, err := binlogFile.ReadVarchar() + if err != nil { + return err + } + + err = p.dispatchVarcharToShards(data, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_BinaryVector: + data, dim, err := binlogFile.ReadBinaryVector() + if err != nil { + return err + } + + err = p.dispatchBinaryVecToShards(data, dim, memoryData, shardList, fieldID) + if err != nil { + return err + } + case schemapb.DataType_FloatVector: + data, dim, err := binlogFile.ReadFloatVector() + if err != nil { + return err + } + + err = p.dispatchFloatVecToShards(data, dim, memoryData, shardList, fieldID) + if err != nil { + return err + } + default: + return errors.New("unsupported data type") + } + log.Info("Binlog adapter: read data into shard list", zap.Int("dataType", int(binlogFile.DataType())), zap.Int("shardLen", len(shardList))) + + return nil +} + +func (p *BinlogAdapter) dispatchBoolToShards(data []bool, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: bool field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("bool field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.BoolFieldData).Data = append(field.(*storage.BoolFieldData).Data, val) + field.(*storage.BoolFieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchInt8ToShards(data []int8, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: int8 field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("int8 field row count is not equal to primary key") + } + + // dispatch entity acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.Int8FieldData).Data = append(field.(*storage.Int8FieldData).Data, val) + field.(*storage.Int8FieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchInt16ToShards(data []int16, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: int16 field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("int16 field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.Int16FieldData).Data = append(field.(*storage.Int16FieldData).Data, val) + field.(*storage.Int16FieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchInt32ToShards(data []int32, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: int32 field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("int32 field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.Int32FieldData).Data = append(field.(*storage.Int32FieldData).Data, val) + field.(*storage.Int32FieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchInt64ToShards(data []int64, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: int64 field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("int64 field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.Int64FieldData).Data = append(field.(*storage.Int64FieldData).Data, val) + field.(*storage.Int64FieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchFloatToShards(data []float32, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: float field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("float field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.FloatFieldData).Data = append(field.(*storage.FloatFieldData).Data, val) + field.(*storage.FloatFieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchDoubleToShards(data []float64, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: double field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("double field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.DoubleFieldData).Data = append(field.(*storage.DoubleFieldData).Data, val) + field.(*storage.DoubleFieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchVarcharToShards(data []string, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + if len(data) != len(shardList) { + log.Error("Binlog adapter: varchar field row count is not equal to primary key", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList))) + return errors.New("varchar field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i, val := range data { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + field.(*storage.StringFieldData).Data = append(field.(*storage.StringFieldData).Data, val) + field.(*storage.StringFieldData).NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchBinaryVecToShards(data []byte, dim int, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + bytesPerVector := dim / 8 + count := len(data) / bytesPerVector + if count != len(shardList) { + log.Error("Binlog adapter: binary vector field row count is not equal to primary key", zap.Int("dataLen", count), zap.Int("shardLen", len(shardList))) + return errors.New("binary vector field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i := 0; i < count; i++ { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + binVecField := field.(*storage.BinaryVectorFieldData) + if binVecField == nil { + log.Error("Binlog adapter: the in-memory field is not a binary vector field") + return errors.New("the in-memory field is not a binary vector field") + } + if binVecField.Dim != dim { + log.Error("Binlog adapter: binary vector dimension mismatch", zap.Int("sourceDim", dim), zap.Int("schemaDim", binVecField.Dim)) + return errors.New("binary vector dimension mismatch") + } + for j := 0; j < bytesPerVector; j++ { + val := data[bytesPerVector*i+j] + + binVecField.Data = append(binVecField.Data, val) + } + binVecField.NumRows[0]++ + } + + return nil +} + +func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memoryData []map[storage.FieldID]storage.FieldData, + shardList []int32, fieldID storage.FieldID) error { + // verify row count + count := len(data) / dim + if count != len(shardList) { + log.Error("Binlog adapter: float vector field row count is not equal to primary key", zap.Int("dataLen", count), zap.Int("shardLen", len(shardList))) + return errors.New("float vector field row count is not equal to primary key") + } + + // dispatch entities acoording to shard list + for i := 0; i < count; i++ { + shardID := shardList[i] + if shardID < 0 { + continue // this entity has been deleted or excluded by timestamp + } + + fields := memoryData[shardID] // initSegmentData() can ensure the existence, no need to check bound here + field := fields[fieldID] // initSegmentData() can ensure the existence, no need to check existence here + floatVecField := field.(*storage.FloatVectorFieldData) + if floatVecField == nil { + log.Error("Binlog adapter: the in-memory field is not a float vector field") + return errors.New("the in-memory field is not a float vector field") + } + if floatVecField.Dim != dim { + log.Error("Binlog adapter: float vector dimension mismatch", zap.Int("sourceDim", dim), zap.Int("schemaDim", floatVecField.Dim)) + return errors.New("float vector dimension mismatch") + } + for j := 0; j < dim; j++ { + val := data[dim*i+j] + floatVecField.Data = append(floatVecField.Data, val) + } + floatVecField.NumRows[0]++ + } + + return nil +} + +// This method do the two things: +// 1. if accumulate data of a segment exceed segmentSize, call callFlushFunc to generate new segment +// 2. if total accumulate data exceed maxTotalSize, call callFlushFUnc to flush the biggest segment +func (p *BinlogAdapter) tryFlushSegments(segmentsData []map[storage.FieldID]storage.FieldData, force bool) error { + totalSize := 0 + biggestSize := 0 + biggestItem := -1 + + // 1. if accumulate data of a segment exceed segmentSize, call callFlushFunc to generate new segment + for i := 0; i < len(segmentsData); i++ { + segmentData := segmentsData[i] + // Note: even rowCount is 0, the size is still non-zero + size := 0 + rowCount := 0 + for _, fieldData := range segmentData { + size += fieldData.GetMemorySize() + rowCount = fieldData.RowNum() + } + + // force to flush, called at the end of Read() + if force && rowCount > 0 { + err := p.callFlushFunc(segmentData, i) + if err != nil { + log.Error("Binlog adapter: failed to force flush segment data", zap.Int("shardID", i)) + return err + } + log.Info("Binlog adapter: force flush", zap.Int("rowCount", rowCount), zap.Int("size", size), zap.Int("shardID", i)) + + segmentsData[i] = initSegmentData(p.collectionSchema) + if segmentsData[i] == nil { + log.Error("Binlog adapter: failed to initialize FieldData list") + return errors.New("failed to initialize FieldData list") + } + continue + } + + // if segment size is larger than predefined segmentSize, flush to create a new segment + // initialize a new FieldData list for next round batch read + if size > int(p.segmentSize) && rowCount > 0 { + err := p.callFlushFunc(segmentData, i) + if err != nil { + log.Error("Binlog adapter: failed to flush segment data", zap.Int("shardID", i)) + return err + } + log.Info("Binlog adapter: segment size exceed limit and flush", zap.Int("rowCount", rowCount), zap.Int("size", size), zap.Int("shardID", i)) + + segmentsData[i] = initSegmentData(p.collectionSchema) + if segmentsData[i] == nil { + log.Error("Binlog adapter: failed to initialize FieldData list") + return errors.New("failed to initialize FieldData list") + } + continue + } + + // calculate the total size(ignore the flushed segments) + // find out the biggest segment for the step 2 + totalSize += size + if size > biggestSize { + biggestSize = size + biggestItem = i + } + } + + // 2. if total accumulate data exceed maxTotalSize, call callFlushFUnc to flush the biggest segment + if totalSize > int(p.maxTotalSize) && biggestItem >= 0 { + segmentData := segmentsData[biggestItem] + size := 0 + rowCount := 0 + for _, fieldData := range segmentData { + size += fieldData.GetMemorySize() + rowCount = fieldData.RowNum() + } + + if rowCount > 0 { + err := p.callFlushFunc(segmentData, biggestItem) + if err != nil { + log.Error("Binlog adapter: failed to flush biggest segment data", zap.Int("shardID", biggestItem)) + return err + } + log.Info("Binlog adapter: total size exceed limit and flush", zap.Int("rowCount", rowCount), + zap.Int("size", size), zap.Int("totalSize", totalSize), zap.Int("shardID", biggestItem)) + + segmentsData[biggestItem] = initSegmentData(p.collectionSchema) + if segmentsData[biggestItem] == nil { + log.Error("Binlog adapter: failed to initialize FieldData list") + return errors.New("failed to initialize FieldData list") + } + } + } + + return nil +} diff --git a/internal/util/importutil/binlog_adapter_test.go b/internal/util/importutil/binlog_adapter_test.go new file mode 100644 index 0000000000..0d88fde10d --- /dev/null +++ b/internal/util/importutil/binlog_adapter_test.go @@ -0,0 +1,1132 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package importutil + +import ( + "encoding/json" + "errors" + "strconv" + "testing" + + "github.com/milvus-io/milvus/api/schemapb" + "github.com/milvus-io/milvus/internal/common" + "github.com/milvus-io/milvus/internal/storage" + "github.com/stretchr/testify/assert" +) + +const ( + baseTimestamp = 43757345 +) + +func createDeltalogBuf(t *testing.T, deleteList interface{}, varcharType bool) []byte { + deleteData := &storage.DeleteData{ + Pks: make([]storage.PrimaryKey, 0), + Tss: make([]storage.Timestamp, 0), + RowCount: 0, + } + + if varcharType { + deltaData := deleteList.([]string) + assert.NotNil(t, deltaData) + for i, id := range deltaData { + deleteData.Pks = append(deleteData.Pks, storage.NewVarCharPrimaryKey(id)) + deleteData.Tss = append(deleteData.Tss, baseTimestamp+uint64(i)) + deleteData.RowCount++ + } + } else { + deltaData := deleteList.([]int64) + assert.NotNil(t, deltaData) + for i, id := range deltaData { + deleteData.Pks = append(deleteData.Pks, storage.NewInt64PrimaryKey(id)) + deleteData.Tss = append(deleteData.Tss, baseTimestamp+uint64(i)) + deleteData.RowCount++ + } + } + + deleteCodec := storage.NewDeleteCodec() + blob, err := deleteCodec.Serialize(1, 1, 1, deleteData) + assert.Nil(t, err) + assert.NotNil(t, blob) + + return blob.Value +} + +// this function create fields data for the sampleSchame() +func createFieldsData(rowCount int) map[storage.FieldID]interface{} { + fieldsData := make(map[storage.FieldID]interface{}) + + rowIDData := make([]int64, 0) + timestampData := make([]int64, 0) + boolData := make([]bool, 0) + int8Data := make([]int8, 0) + int16Data := make([]int16, 0) + int32Data := make([]int32, 0) + int64Data := make([]int64, 0) + floatData := make([]float32, 0) + doubleData := make([]float64, 0) + varcharData := make([]string, 0) + binVecData := make([][]byte, 0) + floatVecData := make([][]float32, 0) + + boolFunc := func(i int) bool { + return i%3 != 0 + } + + for i := 0; i < rowCount; i++ { + rowIDData = append(rowIDData, int64(i)) + timestampData = append(timestampData, baseTimestamp+int64(i)) + boolData = append(boolData, boolFunc(i)) + int8Data = append(int8Data, int8(i%256)) + int16Data = append(int16Data, int16(i%65536)) + int32Data = append(int32Data, int32(i%1000)) + int64Data = append(int64Data, int64(i)) // this is primary key + floatData = append(floatData, float32(i/2)) + doubleData = append(doubleData, float64(i/5)) + varcharData = append(varcharData, "no."+strconv.Itoa(i)) + binVecData = append(binVecData, []byte{byte(i % 256), byte(i % 256)}) // dim = 16 + floatVecData = append(floatVecData, []float32{float32(i / 2), float32(i / 4), float32(i / 5), float32(i / 8)}) // dim = 4 + } + + fieldsData[0] = rowIDData + fieldsData[1] = timestampData + fieldsData[102] = boolData + fieldsData[103] = int8Data + fieldsData[104] = int16Data + fieldsData[105] = int32Data + fieldsData[106] = int64Data + fieldsData[107] = floatData + fieldsData[108] = doubleData + fieldsData[109] = varcharData + fieldsData[110] = binVecData + fieldsData[111] = floatVecData + + return fieldsData +} + +func createSegmentsData(fieldsData map[storage.FieldID]interface{}, shardNum int32) []map[storage.FieldID]storage.FieldData { + segmentsData := make([]map[storage.FieldID]storage.FieldData, 0, shardNum) + for i := 0; i < int(shardNum); i++ { + segData := initSegmentData(sampleSchema()) + fieldID := int64(0) + segData[fieldID].(*storage.Int64FieldData).Data = append(segData[fieldID].(*storage.Int64FieldData).Data, fieldsData[fieldID].([]int64)...) // RoeIDField + fieldID = int64(102) + segData[fieldID].(*storage.BoolFieldData).Data = append(segData[fieldID].(*storage.BoolFieldData).Data, fieldsData[fieldID].([]bool)...) + fieldID = int64(103) + segData[fieldID].(*storage.Int8FieldData).Data = append(segData[fieldID].(*storage.Int8FieldData).Data, fieldsData[fieldID].([]int8)...) + fieldID = int64(104) + segData[fieldID].(*storage.Int16FieldData).Data = append(segData[fieldID].(*storage.Int16FieldData).Data, fieldsData[fieldID].([]int16)...) + fieldID = int64(105) + segData[fieldID].(*storage.Int32FieldData).Data = append(segData[fieldID].(*storage.Int32FieldData).Data, fieldsData[fieldID].([]int32)...) + fieldID = int64(106) + segData[fieldID].(*storage.Int64FieldData).Data = append(segData[fieldID].(*storage.Int64FieldData).Data, fieldsData[fieldID].([]int64)...) // primary key + fieldID = int64(107) + segData[fieldID].(*storage.FloatFieldData).Data = append(segData[fieldID].(*storage.FloatFieldData).Data, fieldsData[fieldID].([]float32)...) + fieldID = int64(108) + segData[fieldID].(*storage.DoubleFieldData).Data = append(segData[fieldID].(*storage.DoubleFieldData).Data, fieldsData[fieldID].([]float64)...) + fieldID = int64(109) + segData[fieldID].(*storage.StringFieldData).Data = append(segData[fieldID].(*storage.StringFieldData).Data, fieldsData[fieldID].([]string)...) + fieldID = int64(110) + binVectors := fieldsData[fieldID].([][]byte) + for _, vec := range binVectors { + segData[fieldID].(*storage.BinaryVectorFieldData).Data = append(segData[fieldID].(*storage.BinaryVectorFieldData).Data, vec...) + } + fieldID = int64(111) + floatVectors := fieldsData[fieldID].([][]float32) + for _, vec := range floatVectors { + segData[fieldID].(*storage.FloatVectorFieldData).Data = append(segData[fieldID].(*storage.FloatVectorFieldData).Data, vec...) + } + segmentsData = append(segmentsData, segData) + } + return segmentsData +} + +func Test_NewBinlogAdapter(t *testing.T) { + // nil schema + adapter, err := NewBinlogAdapter(nil, 2, 1024, 2048, nil, nil, 0) + assert.Nil(t, adapter) + assert.NotNil(t, err) + + // nil chunkmanager + adapter, err = NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, nil, nil, 0) + assert.Nil(t, adapter) + assert.NotNil(t, err) + + // nil flushfunc + adapter, err = NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, &MockChunkManager{}, nil, 0) + assert.Nil(t, adapter) + assert.NotNil(t, err) + + // succeed + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + adapter, err = NewBinlogAdapter(sampleSchema(), 2, 2048, 1024, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // no primary key + schema := &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 101, + Name: "id", + IsPrimaryKey: false, + DataType: schemapb.DataType_Int64, + }, + }, + } + adapter, err = NewBinlogAdapter(schema, 2, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.Nil(t, adapter) + assert.NotNil(t, err) +} + +func Test_BinlogAdapterVerify(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // nil input + err = adapter.verify(nil) + assert.NotNil(t, err) + + // empty holder + holder := &SegmentFilesHolder{} + err = adapter.verify(holder) + assert.NotNil(t, err) + + // row id field missed + holder.fieldFiles = make(map[int64][]string) + for i := int64(102); i <= 111; i++ { + holder.fieldFiles[i] = make([]string, 0) + } + err = adapter.verify(holder) + assert.NotNil(t, err) + + // timestamp field missed + holder.fieldFiles[common.RowIDField] = []string{ + "a", + } + + err = adapter.verify(holder) + assert.NotNil(t, err) + + // binlog file count of each field must be equal + holder.fieldFiles[common.TimeStampField] = []string{ + "a", + } + err = adapter.verify(holder) + assert.NotNil(t, err) + + // succeed + for i := int64(102); i <= 111; i++ { + holder.fieldFiles[i] = []string{ + "a", + } + } + err = adapter.verify(holder) + assert.Nil(t, err) +} + +func Test_BinlogAdapterReadDeltalog(t *testing.T) { + deleteItems := []int64{1001, 1002, 1003} + buf := createDeltalogBuf(t, deleteItems, false) + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": buf, + }, + } + + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // succeed + deleteLogs, err := adapter.readDeltalog("dummy") + assert.Nil(t, err) + assert.Equal(t, len(deleteItems), len(deleteLogs)) + + // failed to init BinlogFile + adapter.chunkManager = nil + deleteLogs, err = adapter.readDeltalog("dummy") + assert.NotNil(t, err) + assert.Nil(t, deleteLogs) + + // failed to open binlog file + chunkManager.readErr = errors.New("error") + adapter.chunkManager = chunkManager + deleteLogs, err = adapter.readDeltalog("dummy") + assert.NotNil(t, err) + assert.Nil(t, deleteLogs) +} + +func Test_BinlogAdapterDecodeDeleteLogs(t *testing.T) { + deleteItems := []int64{1001, 1002, 1003, 1004, 1005} + buf := createDeltalogBuf(t, deleteItems, false) + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": buf, + }, + } + + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + holder := &SegmentFilesHolder{ + deltaFiles: []string{ + "dummy", + }, + } + + // use timetamp to filter the no.1 and no.2 deletions + adapter.tsEndPoint = baseTimestamp + 1 + deletions, err := adapter.decodeDeleteLogs(holder) + assert.Nil(t, err) + assert.Equal(t, 2, len(deletions)) + + // wrong data type of delta log + chunkManager.readBuf = map[string][]byte{ + "dummy": createDeltalogBuf(t, []string{"1001", "1002"}, true), + } + + adapter, err = NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + adapter.tsEndPoint = baseTimestamp + deletions, err = adapter.decodeDeleteLogs(holder) + assert.NotNil(t, err) + assert.Nil(t, deletions) +} + +func Test_BinlogAdapterDecodeDeleteLog(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // v2.1 format + st := &storage.DeleteLog{ + Pk: &storage.Int64PrimaryKey{ + Value: 100, + }, + Ts: uint64(450000), + PkType: 5, + } + + m, _ := json.Marshal(st) + + del, err := adapter.decodeDeleteLog(string(m)) + assert.Nil(t, err) + assert.NotNil(t, del) + assert.True(t, del.Pk.EQ(st.Pk)) + assert.Equal(t, st.Ts, del.Ts) + assert.Equal(t, st.PkType, del.PkType) + + // v2.0 format + del, err = adapter.decodeDeleteLog("") + assert.Nil(t, del) + assert.NotNil(t, err) + + del, err = adapter.decodeDeleteLog("a,b") + assert.Nil(t, del) + assert.NotNil(t, err) + + del, err = adapter.decodeDeleteLog("5,b") + assert.Nil(t, del) + assert.NotNil(t, err) + + del, err = adapter.decodeDeleteLog("5,1000") + assert.Nil(t, err) + assert.NotNil(t, del) + assert.True(t, del.Pk.EQ(&storage.Int64PrimaryKey{ + Value: 5, + })) + tt, _ := strconv.ParseUint("1000", 10, 64) + assert.Equal(t, del.Ts, tt) + assert.Equal(t, del.PkType, int64(schemapb.DataType_Int64)) +} + +func Test_BinlogAdapterReadDeltalogs(t *testing.T) { + deleteItems := []int64{1001, 1002, 1003, 1004, 1005} + buf := createDeltalogBuf(t, deleteItems, false) + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": buf, + }, + } + + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + holder := &SegmentFilesHolder{ + deltaFiles: []string{ + "dummy", + }, + } + + // 1. int64 primary key, succeed, return the no.1 and no.2 deletion + adapter.tsEndPoint = baseTimestamp + 1 + intDeletions, strDeletions, err := adapter.readDeltalogs(holder) + assert.Nil(t, err) + assert.Nil(t, strDeletions) + assert.NotNil(t, intDeletions) + + ts, ok := intDeletions[deleteItems[0]] + assert.True(t, ok) + assert.Equal(t, uint64(baseTimestamp), ts) + + ts, ok = intDeletions[deleteItems[1]] + assert.True(t, ok) + assert.Equal(t, uint64(baseTimestamp+1), ts) + + // 2. varchar primary key, succeed, return the no.1 and no.2 deletetion + schema := &schemapb.CollectionSchema{ + Name: "schema", + Fields: []*schemapb.FieldSchema{ + { + FieldID: 101, + Name: "ID", + IsPrimaryKey: true, + DataType: schemapb.DataType_VarChar, + }, + }, + } + + chunkManager.readBuf = map[string][]byte{ + "dummy": createDeltalogBuf(t, []string{"1001", "1002"}, true), + } + + adapter, err = NewBinlogAdapter(schema, 2, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // 2.1 all deletion have been filtered out + intDeletions, strDeletions, err = adapter.readDeltalogs(holder) + assert.Nil(t, err) + assert.Nil(t, intDeletions) + assert.Nil(t, strDeletions) + + // 2.2 filter the no.1 and no.2 deletion + adapter.tsEndPoint = baseTimestamp + 1 + intDeletions, strDeletions, err = adapter.readDeltalogs(holder) + assert.Nil(t, err) + assert.Nil(t, intDeletions) + assert.NotNil(t, strDeletions) + + ts, ok = strDeletions["1001"] + assert.True(t, ok) + assert.Equal(t, uint64(baseTimestamp), ts) + + ts, ok = strDeletions["1002"] + assert.True(t, ok) + assert.Equal(t, uint64(baseTimestamp+1), ts) + + // 3. unsupported primary key type + schema = &schemapb.CollectionSchema{ + Name: "schema", + Fields: []*schemapb.FieldSchema{ + { + FieldID: 101, + Name: "ID", + IsPrimaryKey: true, + DataType: schemapb.DataType_Float, + }, + }, + } + + adapter, err = NewBinlogAdapter(schema, 2, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + adapter.tsEndPoint = baseTimestamp + 1 + intDeletions, strDeletions, err = adapter.readDeltalogs(holder) + assert.NotNil(t, err) + assert.Nil(t, intDeletions) + assert.Nil(t, strDeletions) +} + +func Test_BinlogAdapterReadTimestamp(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // new BinglogFile error + adapter.chunkManager = nil + ts, err := adapter.readTimestamp("dummy") + assert.Nil(t, ts) + assert.NotNil(t, err) + + // open binlog file error + chunkManager := &MockChunkManager{ + readBuf: make(map[string][]byte), + } + adapter.chunkManager = chunkManager + ts, err = adapter.readTimestamp("dummy") + assert.Nil(t, ts) + assert.NotNil(t, err) + + // succeed + rowCount := 10 + fieldsData := createFieldsData(rowCount) + chunkManager.readBuf["dummy"] = createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)) + ts, err = adapter.readTimestamp("dummy") + assert.Nil(t, err) + assert.NotNil(t, ts) + assert.Equal(t, rowCount, len(ts)) +} + +func Test_BinlogAdapterReadPrimaryKeys(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // new BinglogFile error + adapter.chunkManager = nil + intList, strList, err := adapter.readPrimaryKeys("dummy") + assert.Nil(t, intList) + assert.Nil(t, strList) + assert.NotNil(t, err) + + // open binlog file error + chunkManager := &MockChunkManager{ + readBuf: make(map[string][]byte), + } + adapter.chunkManager = chunkManager + intList, strList, err = adapter.readPrimaryKeys("dummy") + assert.Nil(t, intList) + assert.Nil(t, strList) + assert.NotNil(t, err) + + // wrong primary key type + rowCount := 10 + fieldsData := createFieldsData(rowCount) + chunkManager.readBuf["dummy"] = createBinlogBuf(t, schemapb.DataType_Bool, fieldsData[102].([]bool)) + + adapter.primaryType = schemapb.DataType_Bool + intList, strList, err = adapter.readPrimaryKeys("dummy") + assert.Nil(t, intList) + assert.Nil(t, strList) + assert.NotNil(t, err) + + // succeed int64 + adapter.primaryType = schemapb.DataType_Int64 + chunkManager.readBuf["dummy"] = createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)) + intList, strList, err = adapter.readPrimaryKeys("dummy") + assert.NotNil(t, intList) + assert.Nil(t, strList) + assert.Nil(t, err) + assert.Equal(t, rowCount, len(intList)) + + // succeed varchar + adapter.primaryType = schemapb.DataType_VarChar + chunkManager.readBuf["dummy"] = createBinlogBuf(t, schemapb.DataType_VarChar, fieldsData[109].([]string)) + intList, strList, err = adapter.readPrimaryKeys("dummy") + assert.Nil(t, intList) + assert.NotNil(t, strList) + assert.Nil(t, err) + assert.Equal(t, rowCount, len(strList)) +} + +func Test_BinlogAdapterShardListInt64(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + shardNum := int32(2) + adapter, err := NewBinlogAdapter(sampleSchema(), shardNum, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + segmentsData := make([]map[storage.FieldID]storage.FieldData, 0, shardNum) + for i := 0; i < int(shardNum); i++ { + segmentData := initSegmentData(sampleSchema()) + segmentsData = append(segmentsData, segmentData) + } + + // wrong input + shardList, err := adapter.getShardingListByPrimaryInt64([]int64{1}, []int64{1, 2}, segmentsData, map[int64]uint64{}) + assert.Nil(t, shardList) + assert.NotNil(t, err) + + // succeed + // 5 ids, delete two items, the ts end point is 25, there shardList should be [-1, 0, 1, -1, -1] + adapter.tsEndPoint = 30 + idList := []int64{1, 2, 3, 4, 5} + tsList := []int64{10, 20, 30, 40, 50} + deletion := map[int64]uint64{ + 1: 23, + 4: 36, + } + shardList, err = adapter.getShardingListByPrimaryInt64(idList, tsList, segmentsData, deletion) + assert.Nil(t, err) + assert.NotNil(t, shardList) + correctShardList := []int32{-1, 0, 1, -1, -1} + assert.Equal(t, len(correctShardList), len(shardList)) + for i := 0; i < len(shardList); i++ { + assert.Equal(t, correctShardList[i], shardList[i]) + } +} + +func Test_BinlogAdapterShardListVarchar(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + shardNum := int32(2) + adapter, err := NewBinlogAdapter(strKeySchema(), shardNum, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + segmentsData := make([]map[storage.FieldID]storage.FieldData, 0, shardNum) + for i := 0; i < int(shardNum); i++ { + segmentData := initSegmentData(strKeySchema()) + segmentsData = append(segmentsData, segmentData) + } + + // wrong input + shardList, err := adapter.getShardingListByPrimaryVarchar([]string{"1"}, []int64{1, 2}, segmentsData, map[string]uint64{}) + assert.Nil(t, shardList) + assert.NotNil(t, err) + + // succeed + // 5 ids, delete two items, the ts end point is 25, there shardList should be [-1, 1, 1, -1, -1] + adapter.tsEndPoint = 30 + idList := []string{"1", "2", "3", "4", "5"} + tsList := []int64{10, 20, 30, 40, 50} + deletion := map[string]uint64{ + "1": 23, + "4": 36, + } + shardList, err = adapter.getShardingListByPrimaryVarchar(idList, tsList, segmentsData, deletion) + assert.Nil(t, err) + assert.NotNil(t, shardList) + correctShardList := []int32{-1, 1, 1, -1, -1} + assert.Equal(t, len(correctShardList), len(shardList)) + for i := 0; i < len(shardList); i++ { + assert.Equal(t, correctShardList[i], shardList[i]) + } +} + +func Test_BinlogAdapterReadInt64PK(t *testing.T) { + chunkManager := &MockChunkManager{} + + flushCounter := 0 + flushRowCount := 0 + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + flushCounter++ + rowCount := 0 + for _, v := range fields { + rowCount = v.RowNum() + break + } + flushRowCount += rowCount + for _, v := range fields { + assert.Equal(t, rowCount, v.RowNum()) + } + return nil + } + + shardNum := int32(2) + adapter, err := NewBinlogAdapter(sampleSchema(), shardNum, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + adapter.tsEndPoint = baseTimestamp + 1 + + // nil holder + err = adapter.Read(nil) + assert.NotNil(t, err) + + // verify failed + holder := &SegmentFilesHolder{} + err = adapter.Read(holder) + assert.NotNil(t, err) + + // failed to read delta log + holder.fieldFiles = map[int64][]string{ + int64(0): {"0_insertlog"}, + int64(1): {"1_insertlog"}, + int64(102): {"102_insertlog"}, + int64(103): {"103_insertlog"}, + int64(104): {"104_insertlog"}, + int64(105): {"105_insertlog"}, + int64(106): {"106_insertlog"}, + int64(107): {"107_insertlog"}, + int64(108): {"108_insertlog"}, + int64(109): {"109_insertlog"}, + int64(110): {"110_insertlog"}, + int64(111): {"111_insertlog"}, + } + holder.deltaFiles = []string{"deltalog"} + err = adapter.Read(holder) + assert.NotNil(t, err) + + // prepare binlog data + rowCount := 1000 + fieldsData := createFieldsData(rowCount) + deletedItems := []int64{41, 51, 100, 400, 600} + + chunkManager.readBuf = map[string][]byte{ + "102_insertlog": createBinlogBuf(t, schemapb.DataType_Bool, fieldsData[102].([]bool)), + "103_insertlog": createBinlogBuf(t, schemapb.DataType_Int8, fieldsData[103].([]int8)), + "104_insertlog": createBinlogBuf(t, schemapb.DataType_Int16, fieldsData[104].([]int16)), + "105_insertlog": createBinlogBuf(t, schemapb.DataType_Int32, fieldsData[105].([]int32)), + "106_insertlog": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)), // this is primary key + "107_insertlog": createBinlogBuf(t, schemapb.DataType_Float, fieldsData[107].([]float32)), + "108_insertlog": createBinlogBuf(t, schemapb.DataType_Double, fieldsData[108].([]float64)), + "109_insertlog": createBinlogBuf(t, schemapb.DataType_VarChar, fieldsData[109].([]string)), + "110_insertlog": createBinlogBuf(t, schemapb.DataType_BinaryVector, fieldsData[110].([][]byte)), + "111_insertlog": createBinlogBuf(t, schemapb.DataType_FloatVector, fieldsData[111].([][]float32)), + "deltalog": createDeltalogBuf(t, deletedItems, false), + } + + // failed to read primary keys + err = adapter.Read(holder) + assert.NotNil(t, err) + + // failed to read timestamp field + chunkManager.readBuf["0_insertlog"] = createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[0].([]int64)) + err = adapter.Read(holder) + assert.NotNil(t, err) + + // succeed flush + chunkManager.readBuf["1_insertlog"] = createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[1].([]int64)) + + adapter.tsEndPoint = baseTimestamp + uint64(499) // 4 entities deleted, 500 entities excluded + err = adapter.Read(holder) + assert.Nil(t, err) + assert.Equal(t, shardNum, int32(flushCounter)) + assert.Equal(t, rowCount-4-500, flushRowCount) +} + +func Test_BinlogAdapterReadVarcharPK(t *testing.T) { + chunkManager := &MockChunkManager{} + + flushCounter := 0 + flushRowCount := 0 + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + flushCounter++ + rowCount := 0 + for _, v := range fields { + rowCount = v.RowNum() + break + } + flushRowCount += rowCount + for _, v := range fields { + assert.Equal(t, rowCount, v.RowNum()) + } + return nil + } + + // prepare data + holder := &SegmentFilesHolder{} + holder.fieldFiles = map[int64][]string{ + int64(0): {"0_insertlog"}, + int64(1): {"1_insertlog"}, + int64(101): {"101_insertlog"}, + int64(102): {"102_insertlog"}, + int64(103): {"103_insertlog"}, + int64(104): {"104_insertlog"}, + int64(105): {"105_insertlog"}, + int64(106): {"106_insertlog"}, + } + holder.deltaFiles = []string{"deltalog"} + + rowIDData := make([]int64, 0) + timestampData := make([]int64, 0) + pkData := make([]string, 0) + int32Data := make([]int32, 0) + floatData := make([]float32, 0) + varcharData := make([]string, 0) + boolData := make([]bool, 0) + floatVecData := make([][]float32, 0) + + boolFunc := func(i int) bool { + return i%3 != 0 + } + + rowCount := 1000 + for i := 0; i < rowCount; i++ { + rowIDData = append(rowIDData, int64(i)) + timestampData = append(timestampData, baseTimestamp+int64(i)) + pkData = append(pkData, strconv.Itoa(i)) // primary key + int32Data = append(int32Data, int32(i%1000)) + floatData = append(floatData, float32(i/2)) + varcharData = append(varcharData, "no."+strconv.Itoa(i)) + boolData = append(boolData, boolFunc(i)) + floatVecData = append(floatVecData, []float32{float32(i / 2), float32(i / 4), float32(i / 5), float32(i / 8)}) // dim = 4 + } + + deletedItems := []string{"1", "100", "999"} + + chunkManager.readBuf = map[string][]byte{ + "0_insertlog": createBinlogBuf(t, schemapb.DataType_Int64, rowIDData), + "1_insertlog": createBinlogBuf(t, schemapb.DataType_Int64, timestampData), + "101_insertlog": createBinlogBuf(t, schemapb.DataType_VarChar, pkData), + "102_insertlog": createBinlogBuf(t, schemapb.DataType_Int32, int32Data), + "103_insertlog": createBinlogBuf(t, schemapb.DataType_Float, floatData), + "104_insertlog": createBinlogBuf(t, schemapb.DataType_VarChar, varcharData), + "105_insertlog": createBinlogBuf(t, schemapb.DataType_Bool, boolData), + "106_insertlog": createBinlogBuf(t, schemapb.DataType_FloatVector, floatVecData), + "deltalog": createDeltalogBuf(t, deletedItems, true), + } + + // succeed + shardNum := int32(3) + adapter, err := NewBinlogAdapter(strKeySchema(), shardNum, 1024, 2048, chunkManager, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + adapter.tsEndPoint = baseTimestamp + uint64(499) // 3 entities deleted, 500 entities excluded, the "999" is excluded, so totally 502 entities skipped + err = adapter.Read(holder) + assert.Nil(t, err) + assert.Equal(t, shardNum, int32(flushCounter)) + assert.Equal(t, rowCount-502, flushRowCount) +} + +func Test_BinlogAdapterTryFlush(t *testing.T) { + flushCounter := 0 + flushRowCount := 0 + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + flushCounter++ + rowCount := 0 + for _, v := range fields { + rowCount = v.RowNum() + break + } + flushRowCount += rowCount + for _, v := range fields { + assert.Equal(t, rowCount, v.RowNum()) + } + return nil + } + + segmentSize := int64(1024) + maxTotalSize := int64(2048) + shardNum := int32(3) + adapter, err := NewBinlogAdapter(sampleSchema(), shardNum, segmentSize, maxTotalSize, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // prepare flush data, 3 shards, each shard 10 rows + rowCount := 10 + fieldsData := createFieldsData(rowCount) + + // non-force flush + segmentsData := createSegmentsData(fieldsData, shardNum) + err = adapter.tryFlushSegments(segmentsData, false) + assert.Nil(t, err) + assert.Equal(t, 0, flushCounter) + assert.Equal(t, 0, flushRowCount) + + // force flush + err = adapter.tryFlushSegments(segmentsData, true) + assert.Nil(t, err) + assert.Equal(t, int(shardNum), flushCounter) + assert.Equal(t, rowCount*int(shardNum), flushRowCount) + + // after force flush, no data left + flushCounter = 0 + flushRowCount = 0 + err = adapter.tryFlushSegments(segmentsData, true) + assert.Nil(t, err) + assert.Equal(t, 0, flushCounter) + assert.Equal(t, 0, flushRowCount) + + // flush when segment size exceeds segmentSize + segmentsData = createSegmentsData(fieldsData, shardNum) + adapter.segmentSize = 100 // segmentSize is 100 bytes, less than the 10 rows size + err = adapter.tryFlushSegments(segmentsData, false) + assert.Nil(t, err) + assert.Equal(t, int(shardNum), flushCounter) + assert.Equal(t, rowCount*int(shardNum), flushRowCount) + + flushCounter = 0 + flushRowCount = 0 + err = adapter.tryFlushSegments(segmentsData, true) // no data left + assert.Nil(t, err) + assert.Equal(t, 0, flushCounter) + assert.Equal(t, 0, flushRowCount) + + // flush when segments total size exceeds maxTotalSize + segmentsData = createSegmentsData(fieldsData, shardNum) + adapter.segmentSize = 4096 // segmentSize is 4096 bytes, larger than the 10 rows size + adapter.maxTotalSize = 100 // maxTotalSize is 100 bytes, less than the 30 rows size + err = adapter.tryFlushSegments(segmentsData, false) + assert.Nil(t, err) + assert.Equal(t, 1, flushCounter) // only the max segment is flushed + assert.Equal(t, 10, flushRowCount) + + flushCounter = 0 + flushRowCount = 0 + err = adapter.tryFlushSegments(segmentsData, true) // two segments left + assert.Nil(t, err) + assert.Equal(t, 2, flushCounter) + assert.Equal(t, 20, flushRowCount) +} + +func Test_BinlogAdapterDispatch(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + shardNum := int32(3) + adapter, err := NewBinlogAdapter(sampleSchema(), shardNum, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // prepare empty in-memory segments data + segmentsData := make([]map[storage.FieldID]storage.FieldData, 0, shardNum) + for i := 0; i < int(shardNum); i++ { + segmentData := initSegmentData(sampleSchema()) + segmentsData = append(segmentsData, segmentData) + } + + // dispatch bool data + fieldID := int64(102) + shardList := []int32{0, -1, 1} + err = adapter.dispatchBoolToShards([]bool{true}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchBoolToShards([]bool{true, false, false}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch int8 data + fieldID = int64(103) + err = adapter.dispatchInt8ToShards([]int8{1, 2, 3, 4}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchInt8ToShards([]int8{1, 2, 3}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch int16 data + fieldID = int64(104) + err = adapter.dispatchInt16ToShards([]int16{1, 2, 3, 4}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchInt16ToShards([]int16{1, 2, 3}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch int32 data + fieldID = int64(105) + err = adapter.dispatchInt32ToShards([]int32{1, 2, 3, 4}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchInt32ToShards([]int32{1, 2, 3}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch int64 data + fieldID = int64(106) + err = adapter.dispatchInt64ToShards([]int64{1, 2, 3, 4}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchInt64ToShards([]int64{1, 2, 3}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch float data + fieldID = int64(107) + err = adapter.dispatchFloatToShards([]float32{1, 2, 3, 4}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchFloatToShards([]float32{1, 2, 3}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch double data + fieldID = int64(108) + err = adapter.dispatchDoubleToShards([]float64{1, 2, 3, 4}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchDoubleToShards([]float64{1, 2, 3}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch varchar data + fieldID = int64(109) + err = adapter.dispatchVarcharToShards([]string{"a", "b", "c", "d"}, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchVarcharToShards([]string{"a", "b", "c"}, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch binary vector data + fieldID = int64(110) + err = adapter.dispatchBinaryVecToShards([]byte{1, 2, 3, 4}, 16, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchBinaryVecToShards([]byte{1, 2, 3, 4, 5, 6}, 16, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) + + // dispatch float vector data + fieldID = int64(111) + err = adapter.dispatchFloatVecToShards([]float32{1, 2, 3, 4}, 4, segmentsData, shardList, fieldID) // row count mismatch + assert.NotNil(t, err) + for _, segment := range segmentsData { + assert.Equal(t, 0, segment[fieldID].RowNum()) + } + err = adapter.dispatchFloatVecToShards([]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12}, 4, segmentsData, shardList, fieldID) // succeed + assert.Nil(t, err) + assert.Equal(t, 1, segmentsData[0][fieldID].RowNum()) + assert.Equal(t, 1, segmentsData[1][fieldID].RowNum()) + assert.Equal(t, 0, segmentsData[2][fieldID].RowNum()) +} + +func Test_BinlogAdapterReadInsertlog(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + adapter, err := NewBinlogAdapter(sampleSchema(), 2, 1024, 2048, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, adapter) + assert.Nil(t, err) + + // new BinglogFile error + segmentsData := make([]map[storage.FieldID]storage.FieldData, 0, 2) + adapter.chunkManager = nil + err = adapter.readInsertlog(1, "dummy", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // open binlog file error + chunkManager := &MockChunkManager{ + readBuf: make(map[string][]byte), + } + adapter.chunkManager = chunkManager + err = adapter.readInsertlog(1, "dummy", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // prepare binlog data + rowCount := 10 + fieldsData := createFieldsData(rowCount) + + // failed to dispatch bool data + chunkManager.readBuf["bool"] = createBinlogBuf(t, schemapb.DataType_Bool, fieldsData[102].([]bool)) + err = adapter.readInsertlog(1, "bool", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch int8 data + chunkManager.readBuf["int8"] = createBinlogBuf(t, schemapb.DataType_Int8, fieldsData[103].([]int8)) + err = adapter.readInsertlog(1, "int8", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch int16 data + chunkManager.readBuf["int16"] = createBinlogBuf(t, schemapb.DataType_Int16, fieldsData[104].([]int16)) + err = adapter.readInsertlog(1, "int16", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch int32 data + chunkManager.readBuf["int32"] = createBinlogBuf(t, schemapb.DataType_Int32, fieldsData[105].([]int32)) + err = adapter.readInsertlog(1, "int32", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch int64 data + chunkManager.readBuf["int64"] = createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)) + err = adapter.readInsertlog(1, "int64", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch float data + chunkManager.readBuf["float"] = createBinlogBuf(t, schemapb.DataType_Float, fieldsData[107].([]float32)) + err = adapter.readInsertlog(1, "float", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch double data + chunkManager.readBuf["double"] = createBinlogBuf(t, schemapb.DataType_Double, fieldsData[108].([]float64)) + err = adapter.readInsertlog(1, "double", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch varchar data + chunkManager.readBuf["varchar"] = createBinlogBuf(t, schemapb.DataType_VarChar, fieldsData[109].([]string)) + err = adapter.readInsertlog(1, "varchar", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch binvector data + chunkManager.readBuf["binvector"] = createBinlogBuf(t, schemapb.DataType_BinaryVector, fieldsData[110].([][]byte)) + err = adapter.readInsertlog(1, "binvector", segmentsData, []int32{1}) + assert.NotNil(t, err) + + // failed to dispatch floatvector data + chunkManager.readBuf["floatvector"] = createBinlogBuf(t, schemapb.DataType_FloatVector, fieldsData[111].([][]float32)) + err = adapter.readInsertlog(1, "floatvector", segmentsData, []int32{1}) + assert.NotNil(t, err) +} diff --git a/internal/util/importutil/binlog_file.go b/internal/util/importutil/binlog_file.go new file mode 100644 index 0000000000..385b67bf0b --- /dev/null +++ b/internal/util/importutil/binlog_file.go @@ -0,0 +1,526 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package importutil + +import ( + "context" + "errors" + + "github.com/milvus-io/milvus/api/schemapb" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/storage" + "go.uber.org/zap" +) + +// This class is a wrapper of storage.BinlogReader, to read binlog file, block by block. +// Note: for bulkoad function, we only handle normal insert log and delta log. +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// Typically, an insert log file size is 16MB. +type BinlogFile struct { + chunkManager storage.ChunkManager // storage interfaces to read binlog files + reader *storage.BinlogReader // binlog reader +} + +func NewBinlogFile(chunkManager storage.ChunkManager) (*BinlogFile, error) { + if chunkManager == nil { + log.Error("Binlog file: chunk manager pointer is nil") + return nil, errors.New("chunk manager pointer is nil") + } + + binlogFile := &BinlogFile{ + chunkManager: chunkManager, + } + + return binlogFile, nil +} + +func (p *BinlogFile) Open(filePath string) error { + p.Close() + if len(filePath) == 0 { + log.Error("Binlog file: binlog path is empty") + return errors.New("binlog path is empty") + } + + // TODO add context + bytes, err := p.chunkManager.Read(context.TODO(), filePath) + if err != nil { + log.Error("Binlog file: failed to open binlog", zap.String("filePath", filePath), zap.Error(err)) + return err + } + + p.reader, err = storage.NewBinlogReader(bytes) + if err != nil { + log.Error("Binlog file: failed to initialize binlog reader", zap.String("filePath", filePath), zap.Error(err)) + return err + } + + log.Info("Binlog file: open binlog successfully", zap.String("filePath", filePath)) + return nil +} + +// The outer caller must call this method in defer +func (p *BinlogFile) Close() { + if p.reader != nil { + p.reader.Close() + p.reader = nil + } +} + +func (p *BinlogFile) DataType() schemapb.DataType { + if p.reader == nil { + return schemapb.DataType_None + } + + return p.reader.PayloadDataType +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadBool() ([]bool, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]bool, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Bool { + log.Error("Binlog file: binlog data type is not bool") + return nil, errors.New("binlog data type is not bool") + } + + data, err := event.PayloadReaderInterface.GetBoolFromPayload() + if err != nil { + log.Error("Binlog file: failed to read bool data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadInt8() ([]int8, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]int8, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Int8 { + log.Error("Binlog file: binlog data type is not int8") + return nil, errors.New("binlog data type is not int8") + } + + data, err := event.PayloadReaderInterface.GetInt8FromPayload() + if err != nil { + log.Error("Binlog file: failed to read int8 data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadInt16() ([]int16, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]int16, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Int16 { + log.Error("Binlog file: binlog data type is not int16") + return nil, errors.New("binlog data type is not int16") + } + + data, err := event.PayloadReaderInterface.GetInt16FromPayload() + if err != nil { + log.Error("Binlog file: failed to read int16 data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadInt32() ([]int32, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]int32, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Int32 { + log.Error("Binlog file: binlog data type is not int32") + return nil, errors.New("binlog data type is not int32") + } + + data, err := event.PayloadReaderInterface.GetInt32FromPayload() + if err != nil { + log.Error("Binlog file: failed to read int32 data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadInt64() ([]int64, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]int64, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Int64 { + log.Error("Binlog file: binlog data type is not int64") + return nil, errors.New("binlog data type is not int64") + } + + data, err := event.PayloadReaderInterface.GetInt64FromPayload() + if err != nil { + log.Error("Binlog file: failed to read int64 data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadFloat() ([]float32, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]float32, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Float { + log.Error("Binlog file: binlog data type is not float") + return nil, errors.New("binlog data type is not float") + } + + data, err := event.PayloadReaderInterface.GetFloatFromPayload() + if err != nil { + log.Error("Binlog file: failed to read float data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadDouble() ([]float64, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]float64, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_Double { + log.Error("Binlog file: binlog data type is not double") + return nil, errors.New("binlog data type is not double") + } + + data, err := event.PayloadReaderInterface.GetDoubleFromPayload() + if err != nil { + log.Error("Binlog file: failed to read double data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +func (p *BinlogFile) ReadVarchar() ([]string, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, errors.New("binlog reader not yet initialized") + } + + result := make([]string, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, err + } + + // end of the file + if event == nil { + break + } + + // special case: delete event data type is varchar + if event.TypeCode != storage.InsertEventType && event.TypeCode != storage.DeleteEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, errors.New("binlog file is not insert log") + } + + if (p.DataType() != schemapb.DataType_VarChar) && (p.DataType() != schemapb.DataType_String) { + log.Error("Binlog file: binlog data type is not varchar") + return nil, errors.New("binlog data type is not varchar") + } + + data, err := event.PayloadReaderInterface.GetStringFromPayload() + if err != nil { + log.Error("Binlog file: failed to read varchar data", zap.Error(err)) + return nil, err + } + + result = append(result, data...) + } + + return result, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +// return vectors data and the dimension +func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, 0, errors.New("binlog reader not yet initialized") + } + + dim := 0 + result := make([]byte, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, 0, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, 0, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_BinaryVector { + log.Error("Binlog file: binlog data type is not binary vector") + return nil, 0, errors.New("binlog data type is not binary vector") + } + + data, dimenson, err := event.PayloadReaderInterface.GetBinaryVectorFromPayload() + if err != nil { + log.Error("Binlog file: failed to read binary vector data", zap.Error(err)) + return nil, 0, err + } + + dim = dimenson + result = append(result, data...) + } + + return result, dim, nil +} + +// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block. +// This method read all the blocks of a binlog by a data type. +// return vectors data and the dimension +func (p *BinlogFile) ReadFloatVector() ([]float32, int, error) { + if p.reader == nil { + log.Error("Binlog file: binlog reader not yet initialized") + return nil, 0, errors.New("binlog reader not yet initialized") + } + + dim := 0 + result := make([]float32, 0) + for { + event, err := p.reader.NextEventReader() + if err != nil { + log.Error("Binlog file: failed to iterate events reader", zap.Error(err)) + return nil, 0, err + } + + // end of the file + if event == nil { + break + } + + if event.TypeCode != storage.InsertEventType { + log.Error("Binlog file: binlog file is not insert log") + return nil, 0, errors.New("binlog file is not insert log") + } + + if p.DataType() != schemapb.DataType_FloatVector { + log.Error("Binlog file: binlog data type is not float vector") + return nil, 0, errors.New("binlog data type is not float vector") + } + + data, dimension, err := event.PayloadReaderInterface.GetFloatVectorFromPayload() + if err != nil { + log.Error("Binlog file: failed to read float vector data", zap.Error(err)) + return nil, 0, err + } + + dim = dimension + result = append(result, data...) + } + + return result, dim, nil +} diff --git a/internal/util/importutil/binlog_file_test.go b/internal/util/importutil/binlog_file_test.go new file mode 100644 index 0000000000..4e8f145501 --- /dev/null +++ b/internal/util/importutil/binlog_file_test.go @@ -0,0 +1,761 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package importutil + +import ( + "encoding/binary" + "errors" + "fmt" + "testing" + + "github.com/milvus-io/milvus/api/schemapb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/stretchr/testify/assert" +) + +func createBinlogBuf(t *testing.T, dataType schemapb.DataType, data interface{}) []byte { + w := storage.NewInsertBinlogWriter(dataType, 10, 20, 30, 40) + assert.NotNil(t, w) + defer w.Close() + + dim := 0 + if dataType == schemapb.DataType_BinaryVector { + vectors := data.([][]byte) + if len(vectors) > 0 { + dim = len(vectors[0]) * 8 + } + } else if dataType == schemapb.DataType_FloatVector { + vectors := data.([][]float32) + if len(vectors) > 0 { + dim = len(vectors[0]) + } + } + + evt, err := w.NextInsertEventWriter(dim) + assert.Nil(t, err) + assert.NotNil(t, evt) + + evt.SetEventTimestamp(100, 200) + w.SetEventTimeStamp(1000, 2000) + + switch dataType { + case schemapb.DataType_Bool: + err = evt.AddBoolToPayload(data.([]bool)) + assert.Nil(t, err) + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(data.([]bool)) + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_Int8: + err = evt.AddInt8ToPayload(data.([]int8)) + assert.Nil(t, err) + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(data.([]int8)) + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_Int16: + err = evt.AddInt16ToPayload(data.([]int16)) + assert.Nil(t, err) + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(data.([]int16)) * 2 + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_Int32: + err = evt.AddInt32ToPayload(data.([]int32)) + assert.Nil(t, err) + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(data.([]int32)) * 4 + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_Int64: + err = evt.AddInt64ToPayload(data.([]int64)) + assert.Nil(t, err) + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(data.([]int64)) * 8 + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_Float: + err = evt.AddFloatToPayload(data.([]float32)) + assert.Nil(t, err) + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(data.([]float32)) * 4 + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_Double: + err = evt.AddDoubleToPayload(data.([]float64)) + assert.Nil(t, err) + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(data.([]float64)) * 8 + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_VarChar: + values := data.([]string) + sizeTotal := 0 + for _, val := range values { + err = evt.AddOneStringToPayload(val) + assert.Nil(t, err) + sizeTotal += binary.Size(val) + } + // 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++ { + err = evt.AddBinaryVectorToPayload(vectors[i], dim) + assert.Nil(t, err) + } + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(vectors) * dim / 8 + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + case schemapb.DataType_FloatVector: + vectors := data.([][]float32) + for i := 0; i < len(vectors); i++ { + err = evt.AddFloatVectorToPayload(vectors[i], dim) + assert.Nil(t, err) + } + // without the two lines, the case will crash at here. + // the "original_size" is come from storage.originalSizeKey + sizeTotal := len(vectors) * dim * 4 + w.AddExtra("original_size", fmt.Sprintf("%v", sizeTotal)) + default: + assert.True(t, false) + return nil + } + + err = w.Finish() + assert.Nil(t, err) + + buf, err := w.GetBuffer() + assert.Nil(t, err) + assert.NotNil(t, buf) + + return buf +} + +func Test_NewBinlogFile(t *testing.T) { + // nil chunkManager + file, err := NewBinlogFile(nil) + assert.NotNil(t, err) + assert.Nil(t, file) + + // succeed + file, err = NewBinlogFile(&MockChunkManager{}) + assert.Nil(t, err) + assert.NotNil(t, file) +} + +func Test_BinlogFileOpen(t *testing.T) { + chunkManager := &MockChunkManager{ + readBuf: nil, + readErr: nil, + } + + // read succeed + chunkManager.readBuf = map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Bool, []bool{true}), + } + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.NotNil(t, binlogFile.reader) + + dt := binlogFile.DataType() + assert.Equal(t, schemapb.DataType_Bool, dt) + + // failed to read + err = binlogFile.Open("") + assert.NotNil(t, err) + + chunkManager.readErr = errors.New("error") + err = binlogFile.Open("dummy") + assert.NotNil(t, err) + + // failed to create new BinlogReader + chunkManager.readBuf["dummy"] = []byte{} + chunkManager.readErr = nil + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.NotNil(t, err) + assert.Nil(t, binlogFile.reader) + + dt = binlogFile.DataType() + assert.Equal(t, schemapb.DataType_None, dt) + + // nil reader protect + dataBool, err := binlogFile.ReadBool() + assert.Nil(t, dataBool) + assert.NotNil(t, err) + + dataInt8, err := binlogFile.ReadInt8() + assert.Nil(t, dataInt8) + assert.NotNil(t, err) + + dataInt16, err := binlogFile.ReadInt16() + assert.Nil(t, dataInt16) + assert.NotNil(t, err) + + dataInt32, err := binlogFile.ReadInt32() + assert.Nil(t, dataInt32) + assert.NotNil(t, err) + + dataInt64, err := binlogFile.ReadInt64() + assert.Nil(t, dataInt64) + assert.NotNil(t, err) + + dataFloat, err := binlogFile.ReadFloat() + assert.Nil(t, dataFloat) + assert.NotNil(t, err) + + dataDouble, err := binlogFile.ReadDouble() + assert.Nil(t, dataDouble) + assert.NotNil(t, err) + + dataVarchar, err := binlogFile.ReadVarchar() + assert.Nil(t, dataVarchar) + assert.NotNil(t, err) + + dataBinaryVector, dim, err := binlogFile.ReadBinaryVector() + assert.Nil(t, dataBinaryVector) + assert.Equal(t, 0, dim) + assert.NotNil(t, err) + + dataFloatVector, dim, err := binlogFile.ReadFloatVector() + assert.Nil(t, dataFloatVector) + assert.Equal(t, 0, dim) + assert.NotNil(t, err) +} + +func Test_BinlogFileBool(t *testing.T) { + source := []bool{true, false, true, false} + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Bool, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_Bool, binlogFile.DataType()) + + data, err := binlogFile.ReadBool() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, err := binlogFile.ReadInt8() + assert.Zero(t, len(d)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, err = binlogFile.ReadBool() + assert.Zero(t, len(data)) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileInt8(t *testing.T) { + source := []int8{2, 4, 6, 8} + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Int8, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_Int8, binlogFile.DataType()) + + data, err := binlogFile.ReadInt8() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, err := binlogFile.ReadInt16() + assert.Zero(t, len(d)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, err = binlogFile.ReadInt8() + assert.Zero(t, len(data)) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileInt16(t *testing.T) { + source := []int16{2, 4, 6, 8} + + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Int16, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_Int16, binlogFile.DataType()) + + data, err := binlogFile.ReadInt16() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, err := binlogFile.ReadInt32() + assert.Zero(t, len(d)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, err = binlogFile.ReadInt16() + assert.Zero(t, len(data)) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileInt32(t *testing.T) { + source := []int32{2, 4, 6, 8} + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Int32, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_Int32, binlogFile.DataType()) + + data, err := binlogFile.ReadInt32() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, err := binlogFile.ReadInt64() + assert.Zero(t, len(d)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, err = binlogFile.ReadInt32() + assert.Zero(t, len(data)) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileInt64(t *testing.T) { + source := []int64{2, 4, 6, 8} + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Int64, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_Int64, binlogFile.DataType()) + + data, err := binlogFile.ReadInt64() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, err := binlogFile.ReadFloat() + assert.Zero(t, len(d)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, err = binlogFile.ReadInt64() + assert.Zero(t, len(data)) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileFloat(t *testing.T) { + source := []float32{2, 4, 6, 8} + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Float, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_Float, binlogFile.DataType()) + + data, err := binlogFile.ReadFloat() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, err := binlogFile.ReadDouble() + assert.Zero(t, len(d)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, err = binlogFile.ReadFloat() + assert.Zero(t, len(data)) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileDouble(t *testing.T) { + source := []float64{2, 4, 6, 8} + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_Double, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_Double, binlogFile.DataType()) + + data, err := binlogFile.ReadDouble() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, err := binlogFile.ReadVarchar() + assert.Zero(t, len(d)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, err = binlogFile.ReadDouble() + assert.Zero(t, len(data)) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileVarchar(t *testing.T) { + source := []string{"a", "b", "c", "d"} + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_VarChar, source), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_VarChar, binlogFile.DataType()) + + data, err := binlogFile.ReadVarchar() + assert.Nil(t, err) + assert.NotNil(t, data) + assert.Equal(t, len(source), len(data)) + for i := 0; i < len(source); i++ { + assert.Equal(t, source[i], data[i]) + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + d, dim, err := binlogFile.ReadBinaryVector() + assert.Zero(t, len(d)) + assert.Zero(t, dim) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileBinaryVector(t *testing.T) { + vectors := make([][]byte, 0) + vectors = append(vectors, []byte{1, 3, 5, 7}) + vectors = append(vectors, []byte{2, 4, 6, 8}) + dim := len(vectors[0]) * 8 + vecCount := len(vectors) + + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_BinaryVector, vectors), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_BinaryVector, binlogFile.DataType()) + + data, d, err := binlogFile.ReadBinaryVector() + assert.Nil(t, err) + assert.Equal(t, dim, d) + assert.NotNil(t, data) + assert.Equal(t, vecCount*dim/8, len(data)) + for i := 0; i < vecCount; i++ { + for j := 0; j < dim/8; j++ { + assert.Equal(t, vectors[i][j], data[i*dim/8+j]) + } + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + dt, d, err := binlogFile.ReadFloatVector() + assert.Zero(t, len(dt)) + assert.Zero(t, d) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, d, err = binlogFile.ReadBinaryVector() + assert.Zero(t, len(data)) + assert.Zero(t, d) + assert.NotNil(t, err) + + binlogFile.Close() +} + +func Test_BinlogFileFloatVector(t *testing.T) { + vectors := make([][]float32, 0) + vectors = append(vectors, []float32{1, 3, 5, 7}) + vectors = append(vectors, []float32{2, 4, 6, 8}) + dim := len(vectors[0]) + vecCount := len(vectors) + + chunkManager := &MockChunkManager{ + readBuf: map[string][]byte{ + "dummy": createBinlogBuf(t, schemapb.DataType_FloatVector, vectors), + }, + } + + binlogFile, err := NewBinlogFile(chunkManager) + assert.Nil(t, err) + assert.NotNil(t, binlogFile) + + // correct reading + err = binlogFile.Open("dummy") + assert.Nil(t, err) + assert.Equal(t, schemapb.DataType_FloatVector, binlogFile.DataType()) + + data, d, err := binlogFile.ReadFloatVector() + assert.Nil(t, err) + assert.Equal(t, dim, d) + assert.NotNil(t, data) + assert.Equal(t, vecCount*dim, len(data)) + for i := 0; i < vecCount; i++ { + for j := 0; j < dim; j++ { + assert.Equal(t, vectors[i][j], data[i*dim+j]) + } + } + + binlogFile.Close() + + // wrong data type reading + binlogFile, err = NewBinlogFile(chunkManager) + assert.Nil(t, err) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + dt, err := binlogFile.ReadBool() + assert.Zero(t, len(dt)) + assert.NotNil(t, err) + + binlogFile.Close() + + // wrong log type + chunkManager.readBuf["dummy"] = createDeltalogBuf(t, []int64{1}, false) + err = binlogFile.Open("dummy") + assert.Nil(t, err) + + data, d, err = binlogFile.ReadFloatVector() + assert.Zero(t, len(data)) + assert.Zero(t, d) + assert.NotNil(t, err) + + binlogFile.Close() +} diff --git a/internal/util/importutil/binlog_parser.go b/internal/util/importutil/binlog_parser.go new file mode 100644 index 0000000000..0099e10d3e --- /dev/null +++ b/internal/util/importutil/binlog_parser.go @@ -0,0 +1,238 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package importutil + +import ( + "context" + "errors" + "path" + "sort" + "strconv" + + "github.com/milvus-io/milvus/api/schemapb" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/storage" + "go.uber.org/zap" +) + +type BinlogParser struct { + collectionSchema *schemapb.CollectionSchema // collection schema + shardNum int32 // sharding number of the collection + segmentSize int64 // maximum size of a segment(unit:byte) + chunkManager storage.ChunkManager // storage interfaces to browse/read the files + callFlushFunc ImportFlushFunc // call back function to flush segment + + // a timestamp to define the end point of restore, data after this point will be ignored + // set this value to 0, all the data will be ignored + // set this value to math.MaxUint64, all the data will be imported + tsEndPoint uint64 +} + +func NewBinlogParser(collectionSchema *schemapb.CollectionSchema, + shardNum int32, + segmentSize int64, + chunkManager storage.ChunkManager, + flushFunc ImportFlushFunc, + tsEndPoint uint64) (*BinlogParser, error) { + if collectionSchema == nil { + log.Error("Binlog parser: collection schema is nil") + return nil, errors.New("collection schema is nil") + } + + if chunkManager == nil { + log.Error("Binlog parser: chunk manager pointer is nil") + return nil, errors.New("chunk manager pointer is nil") + } + + if flushFunc == nil { + log.Error("Binlog parser: flush function is nil") + return nil, errors.New("flush function is nil") + } + + v := &BinlogParser{ + collectionSchema: collectionSchema, + shardNum: shardNum, + segmentSize: segmentSize, + chunkManager: chunkManager, + callFlushFunc: flushFunc, + tsEndPoint: tsEndPoint, + } + + return v, nil +} + +// For instance, the insertlogRoot is "backup/bak1/data/insert_log/435978159196147009/435978159196147010". +// 435978159196147009 is a collection id, 435978159196147010 is a partition id, +// there is a segment(id is 435978159261483009) under this partition. +// ListWithPrefix() will return all the insert logs under this partition: +// +// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/0/435978159903735811" +// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/1/435978159903735812" +// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/100/435978159903735809" +// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/101/435978159903735810" +// +// The deltalogRoot is "backup/bak1/data/delta_log/435978159196147009/435978159196147010". +// Then we get all the delta logs under this partition: +// +// "backup/bak1/data/delta_log/435978159196147009/435978159196147010/435978159261483009/434574382554415105" +// +// In this function, we will constuct a list of SegmentFilesHolder objects, each SegmentFilesHolder holds the +// insert logs and delta logs of a segment. +func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoot string) ([]*SegmentFilesHolder, error) { + holders := make(map[int64]*SegmentFilesHolder) + // TODO add context + insertlogs, _, err := p.chunkManager.ListWithPrefix(context.TODO(), insertlogRoot, true) + if err != nil { + log.Error("Binlog parser: list insert logs error", zap.Error(err)) + return nil, err + } + + // collect insert log paths + log.Info("Binlog parser: list insert logs", zap.Int("logsCount", len(insertlogs))) + for _, insertlog := range insertlogs { + log.Info("Binlog parser: mapping insert log to segment", zap.String("insertlog", insertlog)) + fieldPath := path.Dir(insertlog) + fieldStrID := path.Base(fieldPath) + fieldID, err := strconv.ParseInt(fieldStrID, 10, 64) + if err != nil { + log.Error("Binlog parser: parse field id error", zap.String("fieldPath", fieldPath), zap.Error(err)) + return nil, err + } + + segmentPath := path.Dir(fieldPath) + segmentStrID := path.Base(segmentPath) + segmentID, err := strconv.ParseInt(segmentStrID, 10, 64) + if err != nil { + log.Error("Binlog parser: parse segment id error", zap.String("segmentPath", segmentPath), zap.Error(err)) + return nil, err + } + + holder, ok := holders[segmentID] + if ok { + holder.fieldFiles[fieldID] = append(holder.fieldFiles[fieldID], insertlog) + } else { + holder = &SegmentFilesHolder{ + segmentID: segmentID, + fieldFiles: make(map[int64][]string), + deltaFiles: make([]string, 0), + } + holder.fieldFiles[fieldID] = make([]string, 0) + holder.fieldFiles[fieldID] = append(holder.fieldFiles[fieldID], insertlog) + holders[segmentID] = holder + } + } + + // sort the insert log paths of each field by ascendent sequence + // there might be several insert logs under a field, for example: + // 2 insert logs under field a: a_1, a_2 + // 2 insert logs under field b: b_1, b_2 + // the row count of a_1 is equal to b_1, the row count of a_2 is equal to b_2 + // when we read these logs, we firstly read a_1 and b_1, then read a_2 and b_2 + // so, here we must ensure the paths are arranged correctly + for _, holder := range holders { + for _, v := range holder.fieldFiles { + sort.Strings(v) + } + } + + // collect delta log paths + if len(deltalogRoot) > 0 { + // TODO add context + deltalogs, _, err := p.chunkManager.ListWithPrefix(context.TODO(), deltalogRoot, true) + if err != nil { + log.Error("Binlog parser: list delta logs error", zap.Error(err)) + return nil, err + } + + log.Info("Binlog parser: list delta logs", zap.Int("logsCount", len(deltalogs))) + for _, deltalog := range deltalogs { + log.Info("Binlog parser: mapping delta log to segment", zap.String("deltalog", deltalog)) + segmentPath := path.Dir(deltalog) + segmentStrID := path.Base(segmentPath) + segmentID, err := strconv.ParseInt(segmentStrID, 10, 64) + if err != nil { + log.Error("Binlog parser: parse segment id error", zap.String("segmentPath", segmentPath), zap.Error(err)) + return nil, err + } + + // if the segment id doesn't exist, no need to process this deltalog + holder, ok := holders[segmentID] + if ok { + holder.deltaFiles = append(holder.deltaFiles, deltalog) + } + } + } + + holdersList := make([]*SegmentFilesHolder, 0) + for _, holder := range holders { + holdersList = append(holdersList, holder) + } + + return holdersList, nil +} + +func (p *BinlogParser) parseSegmentFiles(segmentHolder *SegmentFilesHolder) error { + if segmentHolder == nil { + log.Error("Binlog parser: segment files holder is nil") + return errors.New("segment files holder is nil") + } + + adapter, err := NewBinlogAdapter(p.collectionSchema, p.shardNum, p.segmentSize, + MaxTotalSizeInMemory, p.chunkManager, p.callFlushFunc, p.tsEndPoint) + if err != nil { + log.Error("Binlog parser: failed to create binlog adapter", zap.Error(err)) + return err + } + + return adapter.Read(segmentHolder) +} + +// This functions requires two paths: +// 1. the insert log path of a partition +// 2. the delta log path of a partiion (optional) +func (p *BinlogParser) Parse(filePaths []string) error { + if len(filePaths) != 1 && len(filePaths) != 2 { + log.Error("Binlog parser: illegal paths for binlog import") + return errors.New("illegal paths for binlog import, partition binlog path and partition delta path are required") + } + + insertlogPath := filePaths[0] + deltalogPath := "" + if len(filePaths) == 2 { + deltalogPath = filePaths[1] + } + log.Info("Binlog parser: target paths", + zap.String("insertlogPath", insertlogPath), + zap.String("deltalogPath", deltalogPath)) + + segmentHolders, err := p.constructSegmentHolders(insertlogPath, deltalogPath) + if err != nil { + return err + } + + for _, segmentHolder := range segmentHolders { + err = p.parseSegmentFiles(segmentHolder) + if err != nil { + return err + } + + // trigger gb after each segment finished + triggerGC() + } + + return nil +} diff --git a/internal/util/importutil/binlog_parser_test.go b/internal/util/importutil/binlog_parser_test.go new file mode 100644 index 0000000000..c951d3c981 --- /dev/null +++ b/internal/util/importutil/binlog_parser_test.go @@ -0,0 +1,290 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package importutil + +import ( + "errors" + "path" + "strconv" + "testing" + + "github.com/milvus-io/milvus/api/schemapb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/stretchr/testify/assert" +) + +func Test_NewBinlogParser(t *testing.T) { + // nil schema + parser, err := NewBinlogParser(nil, 2, 1024, nil, nil, 0) + assert.Nil(t, parser) + assert.NotNil(t, err) + + // nil chunkmanager + parser, err = NewBinlogParser(sampleSchema(), 2, 1024, nil, nil, 0) + assert.Nil(t, parser) + assert.NotNil(t, err) + + // nil flushfunc + parser, err = NewBinlogParser(sampleSchema(), 2, 1024, &MockChunkManager{}, nil, 0) + assert.Nil(t, parser) + assert.NotNil(t, err) + + // succeed + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + parser, err = NewBinlogParser(sampleSchema(), 2, 1024, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, parser) + assert.Nil(t, err) +} + +func Test_BinlogParserConstructHolders(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + chunkManager := &MockChunkManager{ + listResult: make(map[string][]string), + } + + insertPath := "insertPath" + deltaPath := "deltaPath" + + // the first segment has 12 fields, each field has 2 binlog files + seg1Files := []string{ + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/0/435978159903735800", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/1/435978159903735801", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/102/435978159903735802", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/103/435978159903735803", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/104/435978159903735804", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/105/435978159903735805", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/106/435978159903735806", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/107/435978159903735807", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/108/435978159903735808", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/109/435978159903735809", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/110/435978159903735810", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/111/435978159903735811", + + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/0/425978159903735800", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/1/425978159903735801", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/102/425978159903735802", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/103/425978159903735803", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/104/425978159903735804", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/105/425978159903735805", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/106/425978159903735806", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/107/425978159903735807", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/108/425978159903735808", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/109/425978159903735809", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/110/425978159903735810", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/111/425978159903735811", + } + + // the second segment has 12 fields, each field has 1 binlog file + seg2Files := []string{ + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/0/435978159903735811", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/1/435978159903735812", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/102/435978159903735802", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/103/435978159903735803", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/104/435978159903735804", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/105/435978159903735805", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/106/435978159903735806", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/107/435978159903735807", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/108/435978159903735808", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/109/435978159903735809", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/110/435978159903735810", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/111/435978159903735811", + } + + chunkManager.listResult[insertPath] = append(chunkManager.listResult[insertPath], seg1Files...) + chunkManager.listResult[insertPath] = append(chunkManager.listResult[insertPath], seg2Files...) + + // the segment has a delta log file + chunkManager.listResult[deltaPath] = []string{ + "backup/bak1/data/delta_log/435978159196147009/435978159196147010/435978159261483009/434574382554415105", + } + + parser, err := NewBinlogParser(sampleSchema(), 2, 1024, chunkManager, flushFunc, 0) + assert.NotNil(t, parser) + assert.Nil(t, err) + + holders, err := parser.constructSegmentHolders(insertPath, deltaPath) + assert.Nil(t, err) + assert.Equal(t, 2, len(holders)) + + // verify the first segment + holder := holders[0] + assert.Equal(t, int64(435978159261483008), holder.segmentID) + assert.Equal(t, 12, len(holder.fieldFiles)) + for i := 0; i < 12; i++ { + fieldPath := path.Dir(seg1Files[i]) + fieldStrID := path.Base(fieldPath) + fieldID, _ := strconv.ParseInt(fieldStrID, 10, 64) + logFiles, ok := holder.fieldFiles[fieldID] + assert.True(t, ok) + assert.Equal(t, 2, len(logFiles)) + + // verify logs under each field is sorted + log1 := logFiles[0] + logID1 := path.Base(log1) + ID1, _ := strconv.ParseInt(logID1, 10, 64) + log2 := logFiles[1] + logID2 := path.Base(log2) + ID2, _ := strconv.ParseInt(logID2, 10, 64) + assert.LessOrEqual(t, ID1, ID2) + } + assert.Equal(t, 0, len(holder.deltaFiles)) + + // verify the second segment + holder = holders[1] + assert.Equal(t, int64(435978159261483009), holder.segmentID) + assert.Equal(t, len(seg2Files), len(holder.fieldFiles)) + for i := 0; i < len(seg2Files); i++ { + fieldPath := path.Dir(seg2Files[i]) + fieldStrID := path.Base(fieldPath) + fieldID, _ := strconv.ParseInt(fieldStrID, 10, 64) + logFiles, ok := holder.fieldFiles[fieldID] + assert.True(t, ok) + assert.Equal(t, 1, len(logFiles)) + assert.Equal(t, seg2Files[i], logFiles[0]) + } + assert.Equal(t, 1, len(holder.deltaFiles)) + assert.Equal(t, chunkManager.listResult[deltaPath][0], holder.deltaFiles[0]) +} + +func Test_BinlogParserConstructHoldersFailed(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + chunkManager := &MockChunkManager{ + listErr: errors.New("error"), + listResult: make(map[string][]string), + } + + parser, err := NewBinlogParser(sampleSchema(), 2, 1024, chunkManager, flushFunc, 0) + assert.NotNil(t, parser) + assert.Nil(t, err) + + insertPath := "insertPath" + deltaPath := "deltaPath" + + // chunkManager return error + holders, err := parser.constructSegmentHolders(insertPath, deltaPath) + assert.NotNil(t, err) + assert.Nil(t, holders) + + // parse field id error(insert log) + chunkManager.listErr = nil + chunkManager.listResult[insertPath] = []string{ + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/illegal/435978159903735811", + } + holders, err = parser.constructSegmentHolders(insertPath, deltaPath) + assert.NotNil(t, err) + assert.Nil(t, holders) + + // parse segment id error(insert log) + chunkManager.listResult[insertPath] = []string{ + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/illegal/0/435978159903735811", + } + holders, err = parser.constructSegmentHolders(insertPath, deltaPath) + assert.NotNil(t, err) + assert.Nil(t, holders) + + // parse segment id error(delta log) + chunkManager.listResult[insertPath] = []string{} + chunkManager.listResult[deltaPath] = []string{ + "backup/bak1/data/delta_log/435978159196147009/435978159196147010/illegal/434574382554415105", + } + holders, err = parser.constructSegmentHolders(insertPath, deltaPath) + assert.NotNil(t, err) + assert.Nil(t, holders) +} + +func Test_BinlogParserParseFilesFailed(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + parser, err := NewBinlogParser(sampleSchema(), 2, 1024, &MockChunkManager{}, flushFunc, 0) + assert.NotNil(t, parser) + assert.Nil(t, err) + + err = parser.parseSegmentFiles(nil) + assert.NotNil(t, err) + + parser.collectionSchema = nil + err = parser.parseSegmentFiles(&SegmentFilesHolder{}) + assert.NotNil(t, err) +} + +func Test_BinlogParserParse(t *testing.T) { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + chunkManager := &MockChunkManager{} + + schema := &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: 101, + Name: "id", + IsPrimaryKey: true, + DataType: schemapb.DataType_Int64, + }, + }, + } + + parser, err := NewBinlogParser(schema, 2, 1024, chunkManager, flushFunc, 0) + assert.NotNil(t, parser) + assert.Nil(t, err) + + // zero paths + err = parser.Parse(nil) + assert.NotNil(t, err) + + // one empty path + paths := []string{ + "insertPath", + } + err = parser.Parse(paths) + assert.Nil(t, err) + + // two empty paths + paths = append(paths, "deltaPath") + err = parser.Parse(paths) + assert.Nil(t, err) + + // wrong path + chunkManager.listResult = make(map[string][]string) + chunkManager.listResult["insertPath"] = []string{ + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/illegal/101/435978159903735811", + } + err = parser.Parse(paths) + assert.NotNil(t, err) + + // file not found + chunkManager.listResult["insertPath"] = []string{ + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/0/435978159903735811", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/1/435978159903735811", + "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/101/435978159903735811", + } + err = parser.Parse(paths) + assert.NotNil(t, err) +} diff --git a/internal/util/importutil/import_wrapper.go b/internal/util/importutil/import_wrapper.go index d5c2a792df..ce7e79789d 100644 --- a/internal/util/importutil/import_wrapper.go +++ b/internal/util/importutil/import_wrapper.go @@ -1,9 +1,26 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( "bufio" "context" "errors" + "math" "path" "runtime/debug" "strconv" @@ -27,7 +44,21 @@ import ( const ( JSONFileExt = ".json" NumpyFileExt = ".npy" - MaxFileSize = 1 * 1024 * 1024 * 1024 // maximum size of each file + + // this limitation is to avoid this OOM risk: + // for column-based file, we read all its data into memory, if user input a large file, the read() method may + // cost extra memory and lear to OOM. + MaxFileSize = 1 * 1024 * 1024 * 1024 // 1GB + + // this limitation is to avoid this OOM risk: + // simetimes system segment max size is a large number, a single segment fields data might cause OOM. + // flush the segment when its data reach this limitation, let the compaction to compact it later. + MaxSegmentSizeInMemory = 512 * 1024 * 1024 // 512MB + + // this limitation is to avoid this OOM risk: + // if the shard number is a large number, although single segment size is small, but there are lot of in-memory segments, + // the total memory size might cause OOM. + MaxTotalSizeInMemory = 2 * 1024 * 1024 * 1024 // 2GB ) type ImportWrapper struct { @@ -111,10 +142,17 @@ func getFileNameAndExt(filePath string) (string, string) { return fileNameWithoutExt, fileType } +// trigger golang gc to return all free memory back to the underlying system at once, +// Note: this operation is expensive, and can lead to latency spikes as it holds the heap lock through the whole process +func triggerGC() { + debug.FreeOSMemory() +} + func (p *ImportWrapper) fileValidation(filePaths []string, rowBased bool) error { // use this map to check duplicate file name(only for numpy file) fileNames := make(map[string]struct{}) + totalSize := int64(0) for i := 0; i < len(filePaths); i++ { filePath := filePaths[i] name, fileType := getFileNameAndExt(filePath) @@ -122,6 +160,7 @@ func (p *ImportWrapper) fileValidation(filePaths []string, rowBased bool) error if ok { // only check dupliate numpy file if fileType == NumpyFileExt { + log.Error("import wrapper: duplicate file name", zap.String("fileName", name+"."+fileType)) return errors.New("duplicate file: " + name + "." + fileType) } } else { @@ -129,25 +168,44 @@ func (p *ImportWrapper) fileValidation(filePaths []string, rowBased bool) error } // check file type + // row-based only support json type, column-based can support json and numpy type if rowBased { if fileType != JSONFileExt { + log.Error("import wrapper: unsupported file type for row-based mode", zap.String("filePath", filePath)) return errors.New("unsupported file type for row-based mode: " + filePath) } } else { if fileType != JSONFileExt && fileType != NumpyFileExt { + log.Error("import wrapper: unsupported file type for column-based mode", zap.String("filePath", filePath)) return errors.New("unsupported file type for column-based mode: " + filePath) } } - // check file size + // check file size, single file size cannot exceed MaxFileSize // TODO add context - size, _ := p.chunkManager.Size(context.TODO(), filePath) + size, err := p.chunkManager.Size(context.TODO(), filePath) + if err != nil { + log.Error("import wrapper: failed to get file size", zap.String("filePath", filePath), zap.Any("err", err)) + return errors.New("failed to ") + } + if size == 0 { + log.Error("import wrapper: file path is empty", zap.String("filePath", filePath)) return errors.New("the file " + filePath + " is empty") } + if size > MaxFileSize { - return errors.New("the file " + filePath + " size exceeds the maximum file size: " + strconv.FormatInt(MaxFileSize, 10) + " bytes") + log.Error("import wrapper: file size exceeds the maximum size", zap.String("filePath", filePath), + zap.Int64("fileSize", size), zap.Int64("MaxFileSize", MaxFileSize)) + return errors.New("the file " + filePath + " size exceeds the maximum size: " + strconv.FormatInt(MaxFileSize, 10) + " bytes") } + totalSize += size + } + + // especially for column-base, total size of files cannot exceed MaxTotalSizeInMemory + if totalSize > MaxTotalSizeInMemory { + log.Error("import wrapper: total size of files exceeds the maximum size", zap.Int64("totalSize", totalSize), zap.Int64("MaxTotalSize", MaxTotalSizeInMemory)) + return errors.New("the total size of all files exceeds the maximum size: " + strconv.FormatInt(MaxTotalSizeInMemory, 10) + " bytes") } return nil @@ -157,9 +215,17 @@ func (p *ImportWrapper) fileValidation(filePaths []string, rowBased bool) error // filePath and rowBased are from ImportTask // if onlyValidate is true, this process only do validation, no data generated, callFlushFunc will not be called func (p *ImportWrapper) Import(filePaths []string, rowBased bool, onlyValidate bool) error { + log.Info("import wrapper: filePaths", zap.Any("filePaths", filePaths)) + // data restore function to import milvus native binlog files(for backup/restore tools) + // the backup/restore tool provide two paths for a partition, the first path is binlog path, the second is deltalog path + if p.isBinlogImport(filePaths) { + // TODO: handle the timestamp end point passed from client side, currently use math.MaxUint64 + return p.doBinlogImport(filePaths, math.MaxUint64) + } + + // normal logic for import general data files err := p.fileValidation(filePaths, rowBased) if err != nil { - log.Error("import error: " + err.Error()) return err } @@ -178,8 +244,10 @@ func (p *ImportWrapper) Import(filePaths []string, rowBased bool, onlyValidate b log.Error("import error: "+err.Error(), zap.String("filePath", filePath)) return err } - } - // no need to check else, since the fileValidation() already do this + } // no need to check else, since the fileValidation() already do this + + // trigger gc after each file finished + triggerGC() } } else { // parse and consume column-based files @@ -187,6 +255,11 @@ func (p *ImportWrapper) Import(filePaths []string, rowBased bool, onlyValidate b // after all columns are parsed/consumed, we need to combine map[string]storage.FieldData into one // and use splitFieldsData() to split fields data into segments according to shard number fieldsData := initSegmentData(p.collectionSchema) + if fieldsData == nil { + log.Error("import wrapper: failed to initialize FieldData list") + return errors.New("failed to initialize FieldData list") + } + rowCount := 0 // function to combine column data into fieldsData @@ -247,15 +320,23 @@ func (p *ImportWrapper) Import(filePaths []string, rowBased bool, onlyValidate b // no need to check else, since the fileValidation() already do this } + // trigger after read finished + triggerGC() + // split fields data into segments err := p.splitFieldsData(fieldsData, filePaths) if err != nil { - log.Error("import error: " + err.Error()) return err } + + // trigger after write finished + triggerGC() } - debug.FreeOSMemory() + return p.reportPersisted() +} + +func (p *ImportWrapper) reportPersisted() error { // report file process state p.importResult.State = commonpb.ImportState_ImportPersisted // persist state task is valuable, retry more times in case fail this task only because of network error @@ -263,12 +344,55 @@ func (p *ImportWrapper) Import(filePaths []string, rowBased bool, onlyValidate b return p.reportFunc(p.importResult) }, retry.Attempts(10)) if reportErr != nil { - log.Warn("fail to report import state to root coord", zap.Error(err)) + log.Warn("import wrapper: fail to report import state to root coord", zap.Error(reportErr)) return reportErr } return nil } +// For internal usage by the restore tool: https://github.com/zilliztech/milvus-backup +// This tool exports data from a milvus service, and call bulkload interface to import native data into another milvus service. +// This tool provides two paths: one is data log path of a partition,the other is delta log path of this partition. +// This method checks the filePaths, if the file paths is exist and not a file, we say it is native import. +func (p *ImportWrapper) isBinlogImport(filePaths []string) bool { + // must contains the insert log path, and the delta log path is optional + if len(filePaths) != 1 && len(filePaths) != 2 { + log.Info("import wrapper: paths count is not 1 or 2", zap.Int("len", len(filePaths))) + return false + } + + for i := 0; i < len(filePaths); i++ { + filePath := filePaths[i] + _, fileType := getFileNameAndExt(filePath) + // contains file extension, is not a path + if len(fileType) != 0 { + log.Info("import wrapper: not a path", zap.String("filePath", filePath), zap.String("fileType", fileType)) + return false + } + } + + log.Info("import wrapper: do binlog import") + return true +} + +func (p *ImportWrapper) doBinlogImport(filePaths []string, tsEndPoint uint64) error { + flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + p.printFieldsDataInfo(fields, "import wrapper: prepare to flush binlog data", filePaths) + return p.callFlushFunc(fields, shardID) + } + parser, err := NewBinlogParser(p.collectionSchema, p.shardNum, p.segmentSize, p.chunkManager, flushFunc, tsEndPoint) + if err != nil { + return err + } + + err = parser.Parse(filePaths) + if err != nil { + return err + } + + return p.reportPersisted() +} + func (p *ImportWrapper) parseRowBasedJSON(filePath string, onlyValidate bool) error { tr := timerecord.NewTimeRecorder("json row-based parser: " + filePath) @@ -483,6 +607,7 @@ func (p *ImportWrapper) appendFunc(schema *schemapb.FieldSchema) func(src storag func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.FieldData, files []string) error { if len(fieldsData) == 0 { + log.Error("import wrapper: fields data is empty") return errors.New("import error: fields data is empty") } @@ -504,6 +629,7 @@ func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.F if !schema.GetAutoID() { v, ok := fieldsData[schema.GetFieldID()] if !ok { + log.Error("import wrapper: field not provided", zap.String("fieldName", schema.GetName())) return errors.New("import error: field " + schema.GetName() + " not provided") } rowCounter[schema.GetName()] = v.RowNum() @@ -513,17 +639,21 @@ func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.F } } if primaryKey == nil { + log.Error("import wrapper: primary key field is not found") return errors.New("import error: primary key field is not found") } for name, count := range rowCounter { if count != rowCount { + log.Error("import wrapper: field row count is not equal to other fields row count", zap.String("fieldName", name), + zap.Int("rowCount", count), zap.Int("otherRowCount", rowCount)) return errors.New("import error: field " + name + " row count " + strconv.Itoa(count) + " is not equal to other fields row count " + strconv.Itoa(rowCount)) } } primaryData, ok := fieldsData[primaryKey.GetFieldID()] if !ok { + log.Error("import wrapper: primary key field is not provided", zap.String("keyName", primaryKey.GetName())) return errors.New("import error: primary key field is not provided") } @@ -550,7 +680,8 @@ func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.F } if primaryData.RowNum() <= 0 { - return errors.New("import error: primary key " + primaryKey.GetName() + " not provided") + log.Error("import wrapper: primary key not provided", zap.String("keyName", primaryKey.GetName())) + return errors.New("import wrapper: primary key " + primaryKey.GetName() + " not provided") } // prepare segemnts @@ -558,7 +689,8 @@ func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.F for i := 0; i < int(p.shardNum); i++ { segmentData := initSegmentData(p.collectionSchema) if segmentData == nil { - return nil + log.Error("import wrapper: failed to initialize FieldData list") + return errors.New("failed to initialize FieldData list") } segmentsData = append(segmentsData, segmentData) } @@ -569,7 +701,8 @@ func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.F schema := p.collectionSchema.Fields[i] appendFuncErr := p.appendFunc(schema) if appendFuncErr == nil { - return errors.New("import error: unsupported field data type") + log.Error("import wrapper: unsupported field data type") + return errors.New("import wrapper: unsupported field data type") } appendFunctions[schema.GetName()] = appendFuncErr } @@ -586,6 +719,7 @@ func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.F } else { intPK, ok := interface{}(pk).(int64) if !ok { + log.Error("import wrapper: primary key field must be int64 or varchar") return errors.New("import error: primary key field must be int64 or varchar") } hash, _ := typeutil.Hash32Int64(intPK) @@ -615,6 +749,7 @@ func (p *ImportWrapper) splitFieldsData(fieldsData map[storage.FieldID]storage.F p.printFieldsDataInfo(segmentData, "import wrapper: prepare to flush segment", files) err := p.callFlushFunc(segmentData, i) if err != nil { + log.Error("import wrapper: flush callback function failed", zap.Any("err", err)) return err } } diff --git a/internal/util/importutil/import_wrapper_test.go b/internal/util/importutil/import_wrapper_test.go index 340c613fd5..e4adf82a85 100644 --- a/internal/util/importutil/import_wrapper_test.go +++ b/internal/util/importutil/import_wrapper_test.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( @@ -29,7 +45,12 @@ const ( ) type MockChunkManager struct { - size int64 + size int64 + sizeErr error + readBuf map[string][]byte + readErr error + listResult map[string][]string + listErr error } func (mc *MockChunkManager) RootPath() string { @@ -57,7 +78,16 @@ func (mc *MockChunkManager) Exist(ctx context.Context, filePath string) (bool, e } func (mc *MockChunkManager) Read(ctx context.Context, filePath string) ([]byte, error) { - return nil, nil + if mc.readErr != nil { + return nil, mc.readErr + } + + val, ok := mc.readBuf[filePath] + if !ok { + return nil, errors.New("mock chunk manager: file path not found: " + filePath) + } + + return val, nil } func (mc *MockChunkManager) MultiRead(ctx context.Context, filePaths []string) ([][]byte, error) { @@ -65,6 +95,15 @@ func (mc *MockChunkManager) MultiRead(ctx context.Context, filePaths []string) ( } func (mc *MockChunkManager) ListWithPrefix(ctx context.Context, prefix string, recursive bool) ([]string, []time.Time, error) { + if mc.listErr != nil { + return nil, nil, mc.listErr + } + + result, ok := mc.listResult[prefix] + if ok { + return result, nil, nil + } + return nil, nil, nil } @@ -81,6 +120,10 @@ func (mc *MockChunkManager) Mmap(ctx context.Context, filePath string) (*mmap.Re } func (mc *MockChunkManager) Size(ctx context.Context, filePath string) (int64, error) { + if mc.sizeErr != nil { + return 0, mc.sizeErr + } + return mc.size, nil } @@ -101,7 +144,6 @@ func Test_NewImportWrapper(t *testing.T) { ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) assert.NoError(t, err) - wrapper := NewImportWrapper(ctx, nil, 2, 1, nil, cm, nil, nil, nil) assert.Nil(t, wrapper) @@ -127,7 +169,7 @@ func Test_NewImportWrapper(t *testing.T) { assert.Nil(t, err) } -func Test_ImportRowBased(t *testing.T) { +func Test_ImportWrapperRowBased(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -214,7 +256,7 @@ func Test_ImportRowBased(t *testing.T) { assert.NotNil(t, err) } -func Test_ImportColumnBased_json(t *testing.T) { +func Test_ImportWrapperColumnBased_json(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -314,7 +356,7 @@ func Test_ImportColumnBased_json(t *testing.T) { assert.NotNil(t, err) } -func Test_ImportColumnBased_StringKey(t *testing.T) { +func Test_ImportWrapperColumnBased_StringKey(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -381,7 +423,7 @@ func Test_ImportColumnBased_StringKey(t *testing.T) { assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State) } -func Test_ImportColumnBased_numpy(t *testing.T) { +func Test_ImportWrapperColumnBased_numpy(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -519,7 +561,7 @@ func perfSchema(dim int) *schemapb.CollectionSchema { return schema } -func Test_ImportRowBased_perf(t *testing.T) { +func Test_ImportWrapperRowBased_perf(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -620,7 +662,7 @@ func Test_ImportRowBased_perf(t *testing.T) { tr.Record("parse large json file " + filePath) } -func Test_ImportColumnBased_perf(t *testing.T) { +func Test_ImportWrapperColumnBased_perf(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -733,7 +775,7 @@ func Test_ImportColumnBased_perf(t *testing.T) { tr.Record("parse large json files: " + filePath1 + "," + filePath2) } -func Test_FileValidation(t *testing.T) { +func Test_ImportWrapperFileValidation(t *testing.T) { ctx := context.Background() cm := &MockChunkManager{ @@ -748,64 +790,71 @@ func Test_FileValidation(t *testing.T) { wrapper := NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil, nil) // duplicate files - var files = [2]string{"1.npy", "1.npy"} - err := wrapper.fileValidation(files[:], false) + files := []string{"1.npy", "1.npy"} + err := wrapper.fileValidation(files, false) assert.NotNil(t, err) - err = wrapper.fileValidation(files[:], true) + err = wrapper.fileValidation(files, true) assert.NotNil(t, err) // unsupported file name files[0] = "a/1.npy" files[1] = "b/1.npy" - err = wrapper.fileValidation(files[:], true) + err = wrapper.fileValidation(files, true) assert.NotNil(t, err) - err = wrapper.fileValidation(files[:], false) + err = wrapper.fileValidation(files, false) assert.NotNil(t, err) // unsupported file type files[0] = "1" files[1] = "1" - err = wrapper.fileValidation(files[:], true) + err = wrapper.fileValidation(files, true) assert.NotNil(t, err) - err = wrapper.fileValidation(files[:], false) + err = wrapper.fileValidation(files, false) assert.NotNil(t, err) // valid cases files[0] = "1.json" files[1] = "2.json" - err = wrapper.fileValidation(files[:], true) + err = wrapper.fileValidation(files, true) assert.Nil(t, err) files[1] = "2.npy" - err = wrapper.fileValidation(files[:], false) + err = wrapper.fileValidation(files, false) assert.Nil(t, err) // empty file - cm = &MockChunkManager{ - size: 0, - } + cm.size = 0 wrapper = NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil, nil) - err = wrapper.fileValidation(files[:], true) + err = wrapper.fileValidation(files, true) assert.NotNil(t, err) - err = wrapper.fileValidation(files[:], false) + err = wrapper.fileValidation(files, false) assert.NotNil(t, err) - // file size exceed limit - cm = &MockChunkManager{ - size: MaxFileSize + 1, - } + // file size exceed MaxFileSize limit + cm.size = MaxFileSize + 1 wrapper = NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil, nil) - err = wrapper.fileValidation(files[:], true) + err = wrapper.fileValidation(files, true) assert.NotNil(t, err) - err = wrapper.fileValidation(files[:], false) + err = wrapper.fileValidation(files, false) + assert.NotNil(t, err) + + // total files size exceed MaxTotalSizeInMemory limit + cm.size = MaxFileSize - 1 + files = append(files, "3.npy") + err = wrapper.fileValidation(files, false) + assert.NotNil(t, err) + + // failed to get file size + cm.sizeErr = errors.New("error") + err = wrapper.fileValidation(files, false) assert.NotNil(t, err) } -func Test_ReportImportFailRowBased(t *testing.T) { +func Test_ImportWrapperReportFailRowBased(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -871,7 +920,7 @@ func Test_ReportImportFailRowBased(t *testing.T) { assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State) } -func Test_ReportImportFailColumnBased_json(t *testing.T) { +func Test_ImportWrapperReportFailColumnBased_json(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -952,7 +1001,7 @@ func Test_ReportImportFailColumnBased_json(t *testing.T) { assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State) } -func Test_ReportImportFailColumnBased_numpy(t *testing.T) { +func Test_ImportWrapperReportFailColumnBased_numpy(t *testing.T) { f := dependency.NewDefaultFactory(true) ctx := context.Background() cm, err := f.NewPersistentStorageChunkManager(ctx) @@ -1039,3 +1088,102 @@ func Test_ReportImportFailColumnBased_numpy(t *testing.T) { assert.Equal(t, 5, rowCount) assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State) } + +func Test_ImportWrapperIsBinlogImport(t *testing.T) { + ctx := context.Background() + + cm := &MockChunkManager{ + size: 1, + } + + idAllocator := newIDAllocator(ctx, t) + schema := perfSchema(128) + shardNum := 2 + segmentSize := 512 // unit: MB + + wrapper := NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil, nil) + + paths := []string{} + b := wrapper.isBinlogImport(paths) + assert.False(t, b) + + paths = []string{ + "path1", + "path2", + "path3", + } + b = wrapper.isBinlogImport(paths) + assert.False(t, b) + + paths = []string{ + "path1.txt", + "path2.jpg", + } + b = wrapper.isBinlogImport(paths) + assert.False(t, b) + + paths = []string{ + "/tmp", + "/tmp", + } + b = wrapper.isBinlogImport(paths) + assert.True(t, b) +} + +func Test_ImportWrapperDoBinlogImport(t *testing.T) { + ctx := context.Background() + + cm := &MockChunkManager{ + size: 1, + } + + idAllocator := newIDAllocator(ctx, t) + schema := perfSchema(128) + shardNum := 2 + segmentSize := 512 // unit: MB + + wrapper := NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil, nil) + paths := []string{ + "/tmp", + "/tmp", + } + wrapper.chunkManager = nil + + // failed to create new BinlogParser + err := wrapper.doBinlogImport(paths, 0) + assert.NotNil(t, err) + + cm.listErr = errors.New("error") + wrapper.chunkManager = cm + wrapper.callFlushFunc = func(fields map[storage.FieldID]storage.FieldData, shardID int) error { + return nil + } + + // failed to call parser.Parse() + err = wrapper.doBinlogImport(paths, 0) + assert.NotNil(t, err) + + // Import() failed + err = wrapper.Import(paths, false, false) + assert.NotNil(t, err) + + cm.listErr = nil + wrapper.reportFunc = func(res *rootcoordpb.ImportResult) error { + return nil + } + wrapper.importResult = &rootcoordpb.ImportResult{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + TaskId: 1, + DatanodeId: 1, + State: commonpb.ImportState_ImportStarted, + Segments: make([]int64, 0), + AutoIds: make([]int64, 0), + RowCount: 0, + } + + // succeed + err = wrapper.doBinlogImport(paths, 0) + assert.Nil(t, err) +} diff --git a/internal/util/importutil/json_handler.go b/internal/util/importutil/json_handler.go index 08f5a99be8..a9a5b354da 100644 --- a/internal/util/importutil/json_handler.go +++ b/internal/util/importutil/json_handler.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( @@ -575,6 +591,10 @@ func (v *JSONRowConsumer) flush(force bool) error { log.Info("JSON row consumer: flush fulled segment", zap.Int("bytes", memSize), zap.Int("rowNum", rowNum)) v.callFlushFunc(segmentData, i) v.segmentsData[i] = initSegmentData(v.collectionSchema) + if v.segmentsData[i] == nil { + log.Error("JSON row consumer: fail to initialize in-memory segment data") + return errors.New("fail to initialize in-memory segment data") + } } } diff --git a/internal/util/importutil/json_handler_test.go b/internal/util/importutil/json_handler_test.go index e360cc5c3c..4c7be16e00 100644 --- a/internal/util/importutil/json_handler_test.go +++ b/internal/util/importutil/json_handler_test.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( diff --git a/internal/util/importutil/json_parser.go b/internal/util/importutil/json_parser.go index f503fc73c9..2b09398528 100644 --- a/internal/util/importutil/json_parser.go +++ b/internal/util/importutil/json_parser.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( diff --git a/internal/util/importutil/json_parser_test.go b/internal/util/importutil/json_parser_test.go index 3b28d9903e..cc2c86b402 100644 --- a/internal/util/importutil/json_parser_test.go +++ b/internal/util/importutil/json_parser_test.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( @@ -72,7 +88,10 @@ func sampleSchema() *schemapb.CollectionSchema { Name: "field_string", IsPrimaryKey: false, Description: "string", - DataType: schemapb.DataType_String, + DataType: schemapb.DataType_VarChar, + TypeParams: []*commonpb.KeyValuePair{ + {Key: "max_length", Value: "128"}, + }, }, { FieldID: 110, @@ -168,7 +187,9 @@ func Test_AdjustBufSize(t *testing.T) { parser := NewJSONParser(ctx, schema) assert.NotNil(t, parser) - sizePerRecord, _ := typeutil.EstimateSizePerRecord(schema) + sizePerRecord, err := typeutil.EstimateSizePerRecord(schema) + assert.Nil(t, err) + assert.Greater(t, sizePerRecord, 0) assert.Equal(t, MaxBatchCount, MaxFileSize/(sizePerRecord*int(parser.bufSize))) // huge row @@ -194,7 +215,7 @@ func Test_AdjustBufSize(t *testing.T) { assert.Equal(t, int64(MinBufferSize), parser.bufSize) } -func Test_ParserRows(t *testing.T) { +func Test_JSONParserParserRows(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -299,7 +320,7 @@ func Test_ParserRows(t *testing.T) { assert.NotNil(t, err) } -func Test_ParserColumns(t *testing.T) { +func Test_JSONParserParserColumns(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -423,7 +444,7 @@ func Test_ParserColumns(t *testing.T) { assert.NotNil(t, err) } -func Test_ParserRowsStringKey(t *testing.T) { +func Test_JSONParserParserRowsStringKey(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -525,7 +546,7 @@ func Test_ParserRowsStringKey(t *testing.T) { assert.Equal(t, int64(10), validator.ValidateCount()) } -func Test_ParserColumnsStrKey(t *testing.T) { +func Test_JSONParserParserColumnsStrKey(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/internal/util/importutil/numpy_adapter.go b/internal/util/importutil/numpy_adapter.go index 399f885437..60b1ea386b 100644 --- a/internal/util/importutil/numpy_adapter.go +++ b/internal/util/importutil/numpy_adapter.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( diff --git a/internal/util/importutil/numpy_adapter_test.go b/internal/util/importutil/numpy_adapter_test.go index 4497260ab7..0a8284eaaa 100644 --- a/internal/util/importutil/numpy_adapter_test.go +++ b/internal/util/importutil/numpy_adapter_test.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( @@ -29,7 +45,21 @@ func Test_CreateNumpyFile(t *testing.T) { assert.NotNil(t, err) } -func Test_SetByteOrder(t *testing.T) { +func Test_CreateNumpyData(t *testing.T) { + // directory doesn't exist + data1 := []float32{1, 2, 3, 4, 5} + buf, err := CreateNumpyData(data1) + assert.NotNil(t, buf) + assert.Nil(t, err) + + // invalid data type + data2 := make(map[string]int) + buf, err = CreateNumpyData(data2) + assert.NotNil(t, err) + assert.Nil(t, buf) +} + +func Test_NumpyAdapterSetByteOrder(t *testing.T) { adapter := &NumpyAdapter{ reader: nil, npyReader: &npy.Reader{}, @@ -46,7 +76,7 @@ func Test_SetByteOrder(t *testing.T) { assert.Equal(t, binary.BigEndian, adapter.order) } -func Test_ReadError(t *testing.T) { +func Test_NumpyAdapterReadError(t *testing.T) { adapter := &NumpyAdapter{ reader: nil, npyReader: nil, @@ -174,7 +204,7 @@ func Test_ReadError(t *testing.T) { } } -func Test_Read(t *testing.T) { +func Test_NumpyAdapterRead(t *testing.T) { err := os.MkdirAll(TempFilesPath, os.ModePerm) assert.Nil(t, err) defer os.RemoveAll(TempFilesPath) diff --git a/internal/util/importutil/numpy_parser.go b/internal/util/importutil/numpy_parser.go index 1a8aa1fa5c..9936103453 100644 --- a/internal/util/importutil/numpy_parser.go +++ b/internal/util/importutil/numpy_parser.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( diff --git a/internal/util/importutil/numpy_parser_test.go b/internal/util/importutil/numpy_parser_test.go index 87fbfdda69..40d446e224 100644 --- a/internal/util/importutil/numpy_parser_test.go +++ b/internal/util/importutil/numpy_parser_test.go @@ -1,3 +1,19 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package importutil import ( @@ -42,7 +58,7 @@ func Test_ConvertNumpyType(t *testing.T) { assert.Equal(t, schemapb.DataType_None, dt) } -func Test_Validate(t *testing.T) { +func Test_NumpyParserValidate(t *testing.T) { ctx := context.Background() err := os.MkdirAll(TempFilesPath, os.ModePerm) assert.Nil(t, err) @@ -296,7 +312,7 @@ func Test_Validate(t *testing.T) { }() } -func Test_Parse(t *testing.T) { +func Test_NumpyParserParse(t *testing.T) { ctx := context.Background() err := os.MkdirAll(TempFilesPath, os.ModePerm) assert.Nil(t, err) @@ -488,7 +504,7 @@ func Test_Parse(t *testing.T) { checkFunc(data10, "field_float_vector", flushFunc) } -func Test_Parse_perf(t *testing.T) { +func Test_NumpyParserParse_perf(t *testing.T) { ctx := context.Background() err := os.MkdirAll(TempFilesPath, os.ModePerm) assert.Nil(t, err)