2022-09-30 02:32:54 +00:00
// 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 (
2022-10-27 08:21:34 +00:00
"context"
2022-09-30 02:32:54 +00:00
"encoding/json"
"fmt"
"strconv"
"strings"
2023-09-21 01:45:27 +00:00
"go.uber.org/zap"
2023-02-26 03:31:49 +00:00
2023-06-08 17:28:37 +00:00
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
2022-09-30 02:32:54 +00:00
"github.com/milvus-io/milvus/internal/storage"
2023-04-06 11:14:32 +00:00
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
2023-11-27 05:50:27 +00:00
"github.com/milvus-io/milvus/pkg/util/merr"
2023-04-06 11:14:32 +00:00
"github.com/milvus-io/milvus/pkg/util/typeutil"
2022-09-30 02:32:54 +00:00
)
2023-11-27 05:50:27 +00:00
// SegmentFilesHolder A struct to hold insert log paths and delta log paths of a segment
2022-09-30 02:32:54 +00:00
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
}
2023-11-27 05:50:27 +00:00
// BinlogAdapter Adapter class to process insertlog/deltalog of a backuped segment
2022-09-30 02:32:54 +00:00
// This class do the following works:
2023-07-11 07:18:28 +00:00
// 1. read insert log of each field, then constructs SegmentData in memory.
2022-09-30 02:32:54 +00:00
// 2. read delta log to remove deleted entities(TimeStampField is used to apply or skip the operation).
// 3. split data according to shard number
2022-10-27 08:21:34 +00:00
// 4. call the callFlushFunc function to flush data into new binlog file if data size reaches blockSize.
2022-09-30 02:32:54 +00:00
type BinlogAdapter struct {
2023-07-11 07:18:28 +00:00
ctx context . Context // for canceling parse process
collectionInfo * CollectionInfo // collection details including schema
chunkManager storage . ChunkManager // storage interfaces to read binlog files
callFlushFunc ImportFlushFunc // call back function to flush segment
blockSize int64 // maximum size of a read block(unit:byte)
maxTotalSize int64 // maximum size of in-memory segments(unit:byte)
2022-09-30 02:32:54 +00:00
2022-10-27 08:21:34 +00:00
// a timestamp to define the start time point of restore, data before this time point will be ignored
// set this value to 0, all the data will be imported
// set this value to math.MaxUint64, all the data will be ignored
// the tsStartPoint value must be less/equal than tsEndPoint
tsStartPoint uint64
// a timestamp to define the end time point of restore, data after this time point will be ignored
2022-09-30 02:32:54 +00:00
// set this value to 0, all the data will be ignored
// set this value to math.MaxUint64, all the data will be imported
2022-10-27 08:21:34 +00:00
// the tsEndPoint value must be larger/equal than tsStartPoint
2022-09-30 02:32:54 +00:00
tsEndPoint uint64
}
2022-10-27 08:21:34 +00:00
func NewBinlogAdapter ( ctx context . Context ,
2023-07-11 07:18:28 +00:00
collectionInfo * CollectionInfo ,
2022-10-27 08:21:34 +00:00
blockSize int64 ,
2022-09-30 02:32:54 +00:00
maxTotalSize int64 ,
chunkManager storage . ChunkManager ,
flushFunc ImportFlushFunc ,
2022-10-27 08:21:34 +00:00
tsStartPoint uint64 ,
2023-09-21 01:45:27 +00:00
tsEndPoint uint64 ,
) ( * BinlogAdapter , error ) {
2023-07-11 07:18:28 +00:00
if collectionInfo == nil {
log . Warn ( "Binlog adapter: collection schema is nil" )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( "collection schema is nil" )
2022-09-30 02:32:54 +00:00
}
if chunkManager == nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: chunk manager pointer is nil" )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( "chunk manager pointer is nil" )
2022-09-30 02:32:54 +00:00
}
if flushFunc == nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: flush function is nil" )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( "flush function is nil" )
2022-09-30 02:32:54 +00:00
}
adapter := & BinlogAdapter {
2023-07-11 07:18:28 +00:00
ctx : ctx ,
collectionInfo : collectionInfo ,
chunkManager : chunkManager ,
callFlushFunc : flushFunc ,
blockSize : blockSize ,
maxTotalSize : maxTotalSize ,
tsStartPoint : tsStartPoint ,
tsEndPoint : tsEndPoint ,
2022-09-30 02:32:54 +00:00
}
// amend the segment size to avoid portential OOM risk
2023-12-08 02:12:38 +00:00
if adapter . blockSize > Params . DataCoordCfg . SegmentMaxSize . GetAsInt64 ( ) {
adapter . blockSize = Params . DataCoordCfg . SegmentMaxSize . GetAsInt64 ( )
2022-09-30 02:32:54 +00:00
}
return adapter , nil
}
func ( p * BinlogAdapter ) Read ( segmentHolder * SegmentFilesHolder ) error {
if segmentHolder == nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: segment files holder is nil" )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( "segment files holder is nil" )
2022-09-30 02:32:54 +00:00
}
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
2022-10-27 08:21:34 +00:00
// if accumulate data exceed blockSize, call callFlushFunc to generate new binlog file
2022-09-30 02:32:54 +00:00
batchCount := 0
for _ , files := range segmentHolder . fieldFiles {
batchCount = len ( files )
break
}
2023-07-11 07:18:28 +00:00
// prepare shards in-memory data
shardsData := make ( [ ] ShardData , 0 , p . collectionInfo . ShardNum )
for i := 0 ; i < int ( p . collectionInfo . ShardNum ) ; i ++ {
shardData := initShardData ( p . collectionInfo . Schema , p . collectionInfo . PartitionIDs )
if shardData == nil {
log . Warn ( "Binlog adapter: fail to initialize in-memory segment data" , zap . Int ( "shardID" , i ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "fail to initialize in-memory segment data for shard id %d" , i ) )
2022-09-30 02:32:54 +00:00
}
2023-07-11 07:18:28 +00:00
shardsData = append ( shardsData , shardData )
2022-09-30 02:32:54 +00:00
}
// read binlog files batch by batch
2023-07-11 07:18:28 +00:00
primaryKey := p . collectionInfo . PrimaryKey
2022-09-30 02:32:54 +00:00
for i := 0 ; i < batchCount ; i ++ {
2023-11-27 05:50:27 +00:00
// batchFiles excludes the primary key field and the timestamp field.
2022-09-30 02:32:54 +00:00
// 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 {
2023-07-11 07:18:28 +00:00
if fieldID == primaryKey . GetFieldID ( ) || fieldID == common . TimeStampField {
2022-09-30 02:32:54 +00:00
continue
}
batchFiles [ fieldID ] = files [ i ]
}
log . Info ( "Binlog adapter: batch files to read" , zap . Any ( "batchFiles" , batchFiles ) )
// read primary keys firstly
2023-07-11 07:18:28 +00:00
primaryLog := segmentHolder . fieldFiles [ primaryKey . GetFieldID ( ) ] [ i ] // no need to check existence, already verified
log . Info ( "Binlog adapter: prepare to read primary key binglog" ,
zap . Int64 ( "pk" , primaryKey . GetFieldID ( ) ) , zap . String ( "logPath" , primaryLog ) )
2022-09-30 02:32:54 +00:00
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
2024-01-05 08:12:48 +00:00
log . Info ( "Binlog adapter: prepare to read timestamp binglog" , zap . String ( "logPath" , timestampLog ) )
2022-09-30 02:32:54 +00:00
timestampList , err := p . readTimestamp ( timestampLog )
if err != nil {
return err
}
var shardList [ ] int32
2023-07-11 07:18:28 +00:00
if primaryKey . GetDataType ( ) == schemapb . DataType_Int64 {
2022-09-30 02:32:54 +00:00
// calculate a shard num list by primary keys and deleted entities
2023-07-11 07:18:28 +00:00
shardList , err = p . getShardingListByPrimaryInt64 ( intList , timestampList , shardsData , intDeletedList )
2022-09-30 02:32:54 +00:00
if err != nil {
return err
}
2023-07-11 07:18:28 +00:00
} else if primaryKey . GetDataType ( ) == schemapb . DataType_VarChar {
2022-09-30 02:32:54 +00:00
// calculate a shard num list by primary keys and deleted entities
2023-07-11 07:18:28 +00:00
shardList , err = p . getShardingListByPrimaryVarchar ( strList , timestampList , shardsData , strDeletedList )
2022-09-30 02:32:54 +00:00
if err != nil {
return err
}
} else {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: unsupported primary key type" , zap . Int ( "type" , int ( primaryKey . GetDataType ( ) ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "unsupported primary key type %d, primary key should be int64 or varchar" , primaryKey . GetDataType ( ) ) )
2022-09-30 02:32:54 +00:00
}
// 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 {
2022-10-27 08:21:34 +00:00
// outside context might be canceled(service stop, or future enhancement for canceling import task)
if isCanceled ( p . ctx ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: import task was canceled" )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( "import task was canceled" )
2022-10-27 08:21:34 +00:00
}
2023-07-11 07:18:28 +00:00
err = p . readInsertlog ( fieldID , file , shardsData , shardList )
2022-09-30 02:32:54 +00:00
if err != nil {
return err
}
}
2022-10-27 08:21:34 +00:00
// flush segment whose size exceed blockSize
2023-07-11 07:18:28 +00:00
err = tryFlushBlocks ( p . ctx , shardsData , p . collectionInfo . Schema , p . callFlushFunc , p . blockSize , p . maxTotalSize , false )
2022-09-30 02:32:54 +00:00
if err != nil {
return err
}
}
// finally, force to flush
2023-07-11 07:18:28 +00:00
return tryFlushBlocks ( p . ctx , shardsData , p . collectionInfo . Schema , p . callFlushFunc , p . blockSize , p . maxTotalSize , true )
2022-09-30 02:32:54 +00:00
}
2022-10-27 08:21:34 +00:00
// verify method verify the schema and binlog files
2023-11-27 05:50:27 +00:00
// 1. each field must have binlog file
2022-09-30 02:32:54 +00:00
// 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,
2023-11-27 05:50:27 +00:00
// but the segmentHolder.fieldFiles need to contain the two fields.
2022-09-30 02:32:54 +00:00
func ( p * BinlogAdapter ) verify ( segmentHolder * SegmentFilesHolder ) error {
if segmentHolder == nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: segment files holder is nil" )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( "segment files holder is nil" )
2022-09-30 02:32:54 +00:00
}
firstFieldFileCount := 0
2023-11-27 05:50:27 +00:00
// each field must have binlog file
2023-07-11 07:18:28 +00:00
for i := 0 ; i < len ( p . collectionInfo . Schema . Fields ) ; i ++ {
schema := p . collectionInfo . Schema . Fields [ i ]
2022-09-30 02:32:54 +00:00
files , ok := segmentHolder . fieldFiles [ schema . FieldID ]
if ! ok {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: a field has no binlog file" , zap . Int64 ( "fieldID" , schema . FieldID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "the field %d has no binlog file" , schema . FieldID ) )
2022-09-30 02:32:54 +00:00
}
if i == 0 {
firstFieldFileCount = len ( files )
}
}
2023-11-27 05:50:27 +00:00
// the segmentHolder.fieldFiles need to contain RowIDField
2022-09-30 02:32:54 +00:00
_ , ok := segmentHolder . fieldFiles [ common . RowIDField ]
if ! ok {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: the binlog files of RowIDField is missed" )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( "the binlog files of RowIDField is missed" )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// the segmentHolder.fieldFiles need to contain TimeStampField
2022-09-30 02:32:54 +00:00
_ , ok = segmentHolder . fieldFiles [ common . TimeStampField ]
if ! ok {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: the binlog files of TimeStampField is missed" )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( "the binlog files of TimeStampField is missed" )
2022-09-30 02:32:54 +00:00
}
// binlog file count of each field must be equal
for _ , files := range segmentHolder . fieldFiles {
if firstFieldFileCount != len ( files ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: file count of each field must be equal" , zap . Int ( "firstFieldFileCount" , firstFieldFileCount ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog file count of each field must be equal, first field files count: %d, other field files count: %d" ,
firstFieldFileCount , len ( files ) ) )
2022-09-30 02:32:54 +00:00
}
}
return nil
}
2022-10-27 08:21:34 +00:00
// readDeltalogs method reads data from deltalog, and convert to a dict
2022-09-30 02:32:54 +00:00
// 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
}
2023-07-11 07:18:28 +00:00
primaryKey := p . collectionInfo . PrimaryKey
if primaryKey . GetDataType ( ) == schemapb . DataType_Int64 {
2022-09-30 02:32:54 +00:00
deletedIDDict := make ( map [ int64 ] uint64 )
for _ , deleteLog := range deleteLogs {
2023-12-14 06:36:40 +00:00
_ , exist := deletedIDDict [ deleteLog . Pk . GetValue ( ) . ( int64 ) ]
if ! exist || deleteLog . Ts > deletedIDDict [ deleteLog . Pk . GetValue ( ) . ( int64 ) ] {
deletedIDDict [ deleteLog . Pk . GetValue ( ) . ( int64 ) ] = deleteLog . Ts
}
2022-09-30 02:32:54 +00:00
}
log . Info ( "Binlog adapter: count of deleted entities" , zap . Int ( "deletedCount" , len ( deletedIDDict ) ) )
return deletedIDDict , nil , nil
2023-07-11 07:18:28 +00:00
} else if primaryKey . GetDataType ( ) == schemapb . DataType_VarChar {
2022-09-30 02:32:54 +00:00
deletedIDDict := make ( map [ string ] uint64 )
for _ , deleteLog := range deleteLogs {
2023-12-14 06:36:40 +00:00
_ , exist := deletedIDDict [ deleteLog . Pk . GetValue ( ) . ( string ) ]
if ! exist || deleteLog . Ts > deletedIDDict [ deleteLog . Pk . GetValue ( ) . ( string ) ] {
deletedIDDict [ deleteLog . Pk . GetValue ( ) . ( string ) ] = deleteLog . Ts
}
2022-09-30 02:32:54 +00:00
}
log . Info ( "Binlog adapter: count of deleted entities" , zap . Int ( "deletedCount" , len ( deletedIDDict ) ) )
return nil , deletedIDDict , nil
}
2023-11-21 02:04:21 +00:00
log . Warn ( "Binlog adapter: unsupported primary key type" , zap . Int ( "type" , int ( primaryKey . GetDataType ( ) ) ) )
2023-11-27 05:50:27 +00:00
return nil , nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "unsupported primary key type %d, primary key should be int64 or varchar" , primaryKey . GetDataType ( ) ) )
2022-09-30 02:32:54 +00:00
}
2022-10-27 08:21:34 +00:00
// decodeDeleteLogs decodes string array(read from delta log) to storage.DeleteLog array
2022-09-30 02:32:54 +00:00
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
}
2022-10-27 08:21:34 +00:00
// only the ts between tsStartPoint and tsEndPoint is effective
// ignore deletions whose timestamp is larger than the tsEndPoint or less than tsStartPoint
if deleteLog . Ts >= p . tsStartPoint && deleteLog . Ts <= p . tsEndPoint {
2022-09-30 02:32:54 +00:00
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
2023-07-11 07:18:28 +00:00
primaryKey := p . collectionInfo . PrimaryKey
2022-09-30 02:32:54 +00:00
for i := 0 ; i < len ( deleteLogs ) ; i ++ {
2023-07-11 07:18:28 +00:00
if deleteLogs [ i ] . PkType != int64 ( primaryKey . GetDataType ( ) ) {
log . Warn ( "Binlog adapter: delta log data type is not equal to collection's primary key data type" ,
2022-09-30 02:32:54 +00:00
zap . Int64 ( "deltaDataType" , deleteLogs [ i ] . PkType ) ,
2023-07-11 07:18:28 +00:00
zap . Int64 ( "pkDataType" , int64 ( primaryKey . GetDataType ( ) ) ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "delta log data type %d is not equal to collection's primary key data type %d" ,
deleteLogs [ i ] . PkType , primaryKey . GetDataType ( ) ) )
2022-09-30 02:32:54 +00:00
}
}
return deleteLogs , nil
}
2022-10-27 08:21:34 +00:00
// decodeDeleteLog decodes a string to storage.DeleteLog
2022-09-30 02:32:54 +00:00
// 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 {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: the format of deletion string is incorrect" , zap . String ( "deltaStr" , deltaStr ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "the format of deletion string is incorrect, '%s' can not be split" , deltaStr ) )
2022-09-30 02:32:54 +00:00
}
pk , err := strconv . ParseInt ( splits [ 0 ] , 10 , 64 )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to parse primary key of deletion string from old version" ,
2022-09-30 02:32:54 +00:00
zap . String ( "deltaStr" , deltaStr ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to parse primary key of deletion string '%s' from old version, error: %v" , deltaStr , err ) )
2022-09-30 02:32:54 +00:00
}
deleteLog . Pk = & storage . Int64PrimaryKey {
Value : pk ,
}
deleteLog . PkType = int64 ( schemapb . DataType_Int64 )
deleteLog . Ts , err = strconv . ParseUint ( splits [ 1 ] , 10 , 64 )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to parse timestamp of deletion string from old version" ,
2022-09-30 02:32:54 +00:00
zap . String ( "deltaStr" , deltaStr ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to parse timestamp of deletion string '%s' from old version, error: %v" , deltaStr , err ) )
2022-09-30 02:32:54 +00:00
}
}
return deleteLog , nil
}
2022-10-27 08:21:34 +00:00
// readDeltalog parses a delta log file. Each delta log data type is varchar, marshaled from an array of storage.DeleteLog objects.
2022-09-30 02:32:54 +00:00
func ( p * BinlogAdapter ) readDeltalog ( logPath string ) ( [ ] string , error ) {
// open the delta log file
binlogFile , err := NewBinlogFile ( p . chunkManager )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to initialize binlog file" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to initialize binlog file '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
err = binlogFile . Open ( logPath )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to open delta log" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to open delta log '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
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 {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to read delta log" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to read delta log '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
log . Info ( "Binlog adapter: successfully read deltalog" , zap . Int ( "deleteCount" , len ( data ) ) )
return data , nil
}
2022-10-27 08:21:34 +00:00
// readTimestamp method reads data from int64 field, currently we use it to read the timestamp field.
2022-09-30 02:32:54 +00:00
func ( p * BinlogAdapter ) readTimestamp ( logPath string ) ( [ ] int64 , error ) {
// open the log file
binlogFile , err := NewBinlogFile ( p . chunkManager )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to initialize binlog file" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to initialize binlog file '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
err = binlogFile . Open ( logPath )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to open timestamp log file" , zap . String ( "logPath" , logPath ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to open timestamp log file '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
defer binlogFile . Close ( )
// read int64 data
int64List , err := binlogFile . ReadInt64 ( )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to read timestamp data from log file" , zap . String ( "logPath" , logPath ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to read timestamp data from log file '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
log . Info ( "Binlog adapter: read timestamp from log file" , zap . Int ( "tsCount" , len ( int64List ) ) )
return int64List , nil
}
2022-10-27 08:21:34 +00:00
// readPrimaryKeys method reads primary keys from insert log.
2022-09-30 02:32:54 +00:00
func ( p * BinlogAdapter ) readPrimaryKeys ( logPath string ) ( [ ] int64 , [ ] string , error ) {
// open the delta log file
binlogFile , err := NewBinlogFile ( p . chunkManager )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to initialize binlog file" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to initialize binlog file '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
err = binlogFile . Open ( logPath )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to open primary key binlog" , zap . String ( "logPath" , logPath ) )
2023-11-27 05:50:27 +00:00
return nil , nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to open primary key binlog '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
defer binlogFile . Close ( )
// primary key can be int64 or varchar, we need to handle the two cases
2023-07-11 07:18:28 +00:00
primaryKey := p . collectionInfo . PrimaryKey
if primaryKey . GetDataType ( ) == schemapb . DataType_Int64 {
2022-09-30 02:32:54 +00:00
idList , err := binlogFile . ReadInt64 ( )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to read int64 primary key from binlog" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to read int64 primary key from binlog '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
log . Info ( "Binlog adapter: succeed to read int64 primary key binlog" , zap . Int ( "len" , len ( idList ) ) )
return idList , nil , nil
2023-07-11 07:18:28 +00:00
} else if primaryKey . GetDataType ( ) == schemapb . DataType_VarChar {
2022-09-30 02:32:54 +00:00
idList , err := binlogFile . ReadVarchar ( )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to read varchar primary key from binlog" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return nil , nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to read varchar primary key from binlog '%s', error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
log . Info ( "Binlog adapter: succeed to read varchar primary key binlog" , zap . Int ( "len" , len ( idList ) ) )
return nil , idList , nil
}
2023-11-21 02:04:21 +00:00
log . Warn ( "Binlog adapter: unsupported primary key type" , zap . Int ( "type" , int ( primaryKey . GetDataType ( ) ) ) )
2023-11-27 05:50:27 +00:00
return nil , nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "unsupported primary key type %d, primary key should be int64 or varchar" , primaryKey . GetDataType ( ) ) )
2022-09-30 02:32:54 +00:00
}
2022-10-27 08:21:34 +00:00
// getShardingListByPrimaryInt64 method generates a shard id list by primary key(int64) list and deleted list.
2022-09-30 02:32:54 +00:00
// 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 ,
2023-07-11 07:18:28 +00:00
memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
intDeletedList map [ int64 ] uint64 ,
) ( [ ] int32 , error ) {
2022-09-30 02:32:54 +00:00
if len ( timestampList ) != len ( primaryKeys ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: primary key length is not equal to timestamp list length" ,
2022-09-30 02:32:54 +00:00
zap . Int ( "primaryKeysLen" , len ( primaryKeys ) ) , zap . Int ( "timestampLen" , len ( timestampList ) ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "primary key length %d is not equal to timestamp list length %d" , len ( primaryKeys ) , len ( timestampList ) ) )
2022-09-30 02:32:54 +00:00
}
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 ) )
2023-07-11 07:18:28 +00:00
primaryKey := p . collectionInfo . PrimaryKey
2022-09-30 02:32:54 +00:00
for i , key := range primaryKeys {
2022-10-27 08:21:34 +00:00
// if this entity's timestamp is greater than the tsEndPoint, or less than tsStartPoint, set shardID = -1 to skip this entity
2022-09-30 09:42:57 +00:00
// timestamp is stored as int64 type in log file, actually it is uint64, compare with uint64
2022-09-30 02:32:54 +00:00
ts := timestampList [ i ]
2022-10-27 08:21:34 +00:00
if uint64 ( ts ) > p . tsEndPoint || uint64 ( ts ) < p . tsStartPoint {
2022-09-30 02:32:54 +00:00
shardList = append ( shardList , - 1 )
excluded ++
continue
}
2023-12-14 06:36:40 +00:00
deleteTs , deleted := intDeletedList [ key ]
2022-09-30 02:32:54 +00:00
// if the key exists in intDeletedList, that means this entity has been deleted
2023-12-14 06:36:40 +00:00
// only skip entity when delete happen after insert
if deleted && deleteTs > uint64 ( ts ) {
2022-09-30 02:32:54 +00:00
shardList = append ( shardList , - 1 ) // this entity has been deleted, set shardID = -1 and skip this entity
actualDeleted ++
} else {
hash , _ := typeutil . Hash32Int64 ( key )
2023-07-11 07:18:28 +00:00
shardID := hash % uint32 ( p . collectionInfo . ShardNum )
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ primaryKey . GetFieldID ( ) ] // initBlockData() can ensure the existence, no need to check here
2022-09-30 02:32:54 +00:00
// append the entity to primary key's FieldData
field . ( * storage . Int64FieldData ) . Data = append ( field . ( * storage . Int64FieldData ) . Data , key )
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
}
2022-10-27 08:21:34 +00:00
// getShardingListByPrimaryVarchar method generates a shard id list by primary key(varchar) list and deleted list.
2022-09-30 02:32:54 +00:00
// 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 ,
2023-07-11 07:18:28 +00:00
memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
strDeletedList map [ string ] uint64 ,
) ( [ ] int32 , error ) {
2022-09-30 02:32:54 +00:00
if len ( timestampList ) != len ( primaryKeys ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: primary key length is not equal to timestamp list length" ,
2022-09-30 02:32:54 +00:00
zap . Int ( "primaryKeysLen" , len ( primaryKeys ) ) , zap . Int ( "timestampLen" , len ( timestampList ) ) )
2023-11-27 05:50:27 +00:00
return nil , merr . WrapErrImportFailed ( fmt . Sprintf ( "primary key length %d is not equal to timestamp list length %d" , len ( primaryKeys ) , len ( timestampList ) ) )
2022-09-30 02:32:54 +00:00
}
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 ) )
2023-07-11 07:18:28 +00:00
primaryKey := p . collectionInfo . PrimaryKey
2022-09-30 02:32:54 +00:00
for i , key := range primaryKeys {
2022-10-27 08:21:34 +00:00
// if this entity's timestamp is greater than the tsEndPoint, or less than tsStartPoint, set shardID = -1 to skip this entity
2022-09-30 09:42:57 +00:00
// timestamp is stored as int64 type in log file, actually it is uint64, compare with uint64
2022-09-30 02:32:54 +00:00
ts := timestampList [ i ]
2022-10-27 08:21:34 +00:00
if uint64 ( ts ) > p . tsEndPoint || uint64 ( ts ) < p . tsStartPoint {
2022-09-30 02:32:54 +00:00
shardList = append ( shardList , - 1 )
excluded ++
continue
}
2023-12-14 06:36:40 +00:00
deleteTs , deleted := strDeletedList [ key ]
2022-09-30 02:32:54 +00:00
// if exists in strDeletedList, that means this entity has been deleted
2023-12-14 06:36:40 +00:00
// only skip entity when delete happen after insert
if deleted && deleteTs > uint64 ( ts ) {
2022-09-30 02:32:54 +00:00
shardList = append ( shardList , - 1 ) // this entity has been deleted, set shardID = -1 and skip this entity
actualDeleted ++
} else {
hash := typeutil . HashString2Uint32 ( key )
2023-07-11 07:18:28 +00:00
shardID := hash % uint32 ( p . collectionInfo . ShardNum )
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ primaryKey . GetFieldID ( ) ] // initBlockData() can ensure the existence, no need to check existence here
2022-09-30 02:32:54 +00:00
// append the entity to primary key's FieldData
field . ( * storage . StringFieldData ) . Data = append ( field . ( * storage . StringFieldData ) . Data , key )
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
}
2023-05-31 06:45:30 +00:00
// Sometimes the fieldID doesn't exist in the memoryData in the following case:
// Use an old backup tool(v0.2.2) to backup a collection of milvus v2.2.9, use a new backup tool to restore the collection
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) verifyField ( fieldID storage . FieldID , memoryData [ ] ShardData ) error {
for _ , partitions := range memoryData {
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
2023-05-31 06:45:30 +00:00
_ , ok := fields [ fieldID ]
if ! ok {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: the field ID doesn't exist in collection schema" , zap . Int64 ( "fieldID" , fieldID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "the field ID %d doesn't exist in collection schema" , fieldID ) )
2023-05-31 06:45:30 +00:00
}
}
return nil
}
2022-10-27 08:21:34 +00:00
// readInsertlog method reads an insert log, and split the data into different shards according to a shard list
2022-09-30 02:32:54 +00:00
// 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 ,
2023-09-21 01:45:27 +00:00
memoryData [ ] ShardData , shardList [ ] int32 ,
) error {
2023-05-31 06:45:30 +00:00
err := p . verifyField ( fieldID , memoryData )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: could not read binlog file" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "could not read binlog file %s, error: %v" , logPath , err ) )
2023-05-31 06:45:30 +00:00
}
2022-09-30 02:32:54 +00:00
// open the insert log file
binlogFile , err := NewBinlogFile ( p . chunkManager )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to initialize binlog file" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to initialize binlog file %s, error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
err = binlogFile . Open ( logPath )
if err != nil {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: failed to open insert log" , zap . String ( "logPath" , logPath ) , zap . Error ( err ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "failed to open insert log %s, error: %v" , logPath , err ) )
2022-09-30 02:32:54 +00:00
}
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
}
2023-05-18 08:33:23 +00:00
case schemapb . DataType_JSON :
data , err := binlogFile . ReadJSON ( )
if err != nil {
return err
}
err = p . dispatchBytesToShards ( data , memoryData , shardList , fieldID )
if err != nil {
return err
}
2022-09-30 02:32:54 +00:00
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
}
2023-11-27 05:50:27 +00:00
case schemapb . DataType_Array :
data , err := binlogFile . ReadArray ( )
if err != nil {
return err
}
err = p . dispatchArrayToShards ( data , memoryData , shardList , fieldID )
if err != nil {
return err
}
2022-09-30 02:32:54 +00:00
default :
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "unsupported data type %d" , binlogFile . DataType ( ) ) )
2022-09-30 02:32:54 +00:00
}
log . Info ( "Binlog adapter: read data into shard list" , zap . Int ( "dataType" , int ( binlogFile . DataType ( ) ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchBoolToShards ( data [ ] bool , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: bool field row count is not equal to shard list row count %d" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "bool field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: bool field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "bool field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . BoolFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is bool type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is bool type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchInt8ToShards ( data [ ] int8 , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: int8 field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int8 field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entity according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: int8 field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int8 field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . Int8FieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is int8 type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is int8 type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchInt16ToShards ( data [ ] int16 , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: int16 field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int16 field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: int16 field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int16 field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . Int16FieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is int16 type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is int16 type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchInt32ToShards ( data [ ] int32 , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: int32 field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int32 field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: int32 field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int32 field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . Int32FieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is int32 type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is int32 type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchInt64ToShards ( data [ ] int64 , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: int64 field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int64 field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: int64 field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "int64 field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . Int64FieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is int64 type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is int64 type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchFloatToShards ( data [ ] float32 , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: float field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "float field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: float field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "float field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . FloatFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is float type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is float type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchDoubleToShards ( data [ ] float64 , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: double field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "double field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: double field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "double field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . DoubleFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is double type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is double type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchVarcharToShards ( data [ ] string , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: varchar field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "varchar field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: varchar field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "varchar field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . StringFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is varchar type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is varchar type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchBytesToShards ( data [ ] [ ] byte , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2023-05-18 08:33:23 +00:00
// verify row count
if len ( data ) != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: JSON field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "varchar JSON row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2023-05-18 08:33:23 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2023-05-18 08:33:23 +00:00
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: JSON field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "JSON field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2023-05-18 08:33:23 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . JSONFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is JSON type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is JSON type, unequal to field %d" , fieldID ) )
2023-07-11 07:18:28 +00:00
}
fieldData . Data = append ( fieldData . Data , val )
2023-05-18 08:33:23 +00:00
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchBinaryVecToShards ( data [ ] byte , dim int , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
bytesPerVector := dim / 8
count := len ( data ) / bytesPerVector
if count != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: binary vector field row count is not equal to shard list row count" ,
2022-11-07 09:11:02 +00:00
zap . Int ( "dataLen" , count ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binary vector field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i := 0 ; i < count ; i ++ {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: binary vector field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binary vector field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . BinaryVectorFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is binary vector type, unequal to field" ,
2022-11-07 09:11:02 +00:00
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is binary vector type, unequal to field %d" , fieldID ) )
2022-09-30 02:32:54 +00:00
}
2023-07-11 07:18:28 +00:00
if fieldData . Dim != dim {
log . Warn ( "Binlog adapter: binary vector dimension mismatch" ,
zap . Int ( "sourceDim" , dim ) , zap . Int ( "schemaDim" , fieldData . Dim ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binary vector dimension %d is not equal to schema dimension %d" , dim , fieldData . Dim ) )
2022-09-30 02:32:54 +00:00
}
for j := 0 ; j < bytesPerVector ; j ++ {
val := data [ bytesPerVector * i + j ]
2023-07-11 07:18:28 +00:00
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
}
return nil
}
2023-07-11 07:18:28 +00:00
func ( p * BinlogAdapter ) dispatchFloatVecToShards ( data [ ] float32 , dim int , memoryData [ ] ShardData ,
2023-09-21 01:45:27 +00:00
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
2022-09-30 02:32:54 +00:00
// verify row count
count := len ( data ) / dim
if count != len ( shardList ) {
2023-07-11 07:18:28 +00:00
log . Warn ( "Binlog adapter: float vector field row count is not equal to shard list row count" ,
2022-11-07 09:11:02 +00:00
zap . Int ( "dataLen" , count ) , zap . Int ( "shardLen" , len ( shardList ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "float vector field row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
2022-09-30 02:32:54 +00:00
}
2023-11-27 05:50:27 +00:00
// dispatch entities according to shard list
2022-09-30 02:32:54 +00:00
for i := 0 ; i < count ; i ++ {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
2023-07-11 07:18:28 +00:00
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: float vector field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "float vector field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
2023-07-11 07:18:28 +00:00
}
2022-09-30 02:32:54 +00:00
2023-07-11 07:18:28 +00:00
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . FloatVectorFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is float vector type, unequal to field" ,
2022-11-07 09:11:02 +00:00
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is float vector type, unequal to field %d" , fieldID ) )
2022-09-30 02:32:54 +00:00
}
2023-07-11 07:18:28 +00:00
if fieldData . Dim != dim {
log . Warn ( "Binlog adapter: float vector dimension mismatch" ,
zap . Int ( "sourceDim" , dim ) , zap . Int ( "schemaDim" , fieldData . Dim ) )
2023-11-27 05:50:27 +00:00
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binary vector dimension %d is not equal to schema dimension %d" , dim , fieldData . Dim ) )
2022-09-30 02:32:54 +00:00
}
for j := 0 ; j < dim ; j ++ {
val := data [ dim * i + j ]
2023-07-11 07:18:28 +00:00
fieldData . Data = append ( fieldData . Data , val )
2022-09-30 02:32:54 +00:00
}
}
return nil
}
2023-11-27 05:50:27 +00:00
func ( p * BinlogAdapter ) dispatchArrayToShards ( data [ ] * schemapb . ScalarField , memoryData [ ] ShardData ,
shardList [ ] int32 , fieldID storage . FieldID ,
) error {
// verify row count
if len ( data ) != len ( shardList ) {
log . Warn ( "Binlog adapter: Array field row count is not equal to shard list row count" , zap . Int ( "dataLen" , len ( data ) ) , zap . Int ( "shardLen" , len ( shardList ) ) )
return merr . WrapErrImportFailed ( fmt . Sprintf ( "array row count %d is not equal to shard list row count %d" , len ( data ) , len ( shardList ) ) )
}
// dispatch entities according to shard list
for i , val := range data {
shardID := shardList [ i ]
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32 ( len ( memoryData ) ) {
log . Warn ( "Binlog adapter: Array field's shard ID is illegal" , zap . Int32 ( "shardID" , shardID ) , zap . Int ( "shardsCount" , len ( memoryData ) ) )
return merr . WrapErrImportFailed ( fmt . Sprintf ( "array field's shard ID %d is larger than shards number %d" , shardID , len ( memoryData ) ) )
}
partitions := memoryData [ shardID ] // initBlockData() can ensure the existence, no need to check bound here
fields := partitions [ p . collectionInfo . PartitionIDs [ 0 ] ] // NewBinlogAdapter() can ensure only one partition
field := fields [ fieldID ] // initBlockData() can ensure the existence, no need to check existence here
fieldData , ok := field . ( * storage . ArrayFieldData ) // avoid data type mismatch between binlog file and schema
if ! ok {
log . Warn ( "Binlog adapter: binlog is array type, unequal to field" ,
zap . Int64 ( "fieldID" , fieldID ) , zap . Int32 ( "shardID" , shardID ) )
return merr . WrapErrImportFailed ( fmt . Sprintf ( "binlog is array type, unequal to field %d" , fieldID ) )
}
fieldData . Data = append ( fieldData . Data , val )
// TODO @cai: set element type
}
return nil
}