Bulkinsert supports partition keys (#25284)

Signed-off-by: yhmo <yihua.mo@zilliz.com>
This commit is contained in:
groot 2023-07-11 15:18:28 +08:00 committed by GitHub
parent 8b06941da3
commit 96c987ed62
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
33 changed files with 4171 additions and 2469 deletions

View File

@ -124,6 +124,8 @@ type DataNode struct {
dispClient msgdispatcher.Client
factory dependency.Factory
reportImportRetryTimes uint // unitest set this value to 1 to save time, default is 10
}
// NewDataNode will return a DataNode with abnormal state.
@ -144,6 +146,8 @@ func NewDataNode(ctx context.Context, factory dependency.Factory) *DataNode {
eventManagerMap: typeutil.NewConcurrentMap[string, *channelEventManager](),
flowgraphManager: newFlowgraphManager(),
clearSignal: make(chan string, 100),
reportImportRetryTimes: 10,
}
node.UpdateStateCode(commonpb.StateCode_Abnormal)
return node

View File

@ -210,6 +210,11 @@ type RootCoordFactory struct {
ReportImportErr bool
ReportImportNotSuccess bool
ShowPartitionsErr bool
ShowPartitionsNotSuccess bool
ShowPartitionsNames []string
ShowPartitionsIDs []int64
}
type DataCoordFactory struct {
@ -1085,6 +1090,33 @@ func (m *RootCoordFactory) DescribeCollectionInternal(ctx context.Context, in *m
return resp, nil
}
func (m *RootCoordFactory) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
if m.ShowPartitionsErr {
return &milvuspb.ShowPartitionsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, fmt.Errorf("mock show partitions error")
}
if m.ShowPartitionsNotSuccess {
return &milvuspb.ShowPartitionsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "not success",
},
}, nil
}
return &milvuspb.ShowPartitionsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
PartitionNames: m.ShowPartitionsNames,
PartitionIDs: m.ShowPartitionsIDs,
}, nil
}
func (m *RootCoordFactory) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{},

View File

@ -408,18 +408,24 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
log.Info("DataNode receive import request",
logFields := []zap.Field{
zap.Int64("task ID", req.GetImportTask().GetTaskId()),
zap.Int64("collection ID", req.GetImportTask().GetCollectionId()),
zap.Int64("partition ID", req.GetImportTask().GetPartitionId()),
zap.String("database name", req.GetImportTask().GetDatabaseName()),
zap.Strings("channel names", req.GetImportTask().GetChannelNames()),
zap.Int64s("working dataNodes", req.WorkingNodes))
zap.Int64s("working dataNodes", req.WorkingNodes),
zap.Int64("node ID", paramtable.GetNodeID()),
}
log.Info("DataNode receive import request", logFields...)
defer func() {
log.Info("DataNode finish import request", zap.Int64("task ID", req.GetImportTask().GetTaskId()))
log.Info("DataNode finish import request", logFields...)
}()
importResult := &rootcoordpb.ImportResult{
Status: merr.Status(nil),
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
TaskId: req.GetImportTask().TaskId,
DatanodeId: paramtable.GetNodeID(),
State: commonpb.ImportState_ImportStarted,
@ -432,22 +438,28 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
// Spawn a new context to ignore cancellation from parental context.
newCtx, cancel := context.WithTimeout(context.TODO(), ImportCallTimeout)
defer cancel()
// func to report import state to RootCoord.
reportFunc := func(res *rootcoordpb.ImportResult) error {
status, err := node.rootCoord.ReportImport(ctx, res)
if err != nil {
log.Error("fail to report import state to RootCoord", zap.Error(err))
return err
}
if status != nil && status.ErrorCode != commonpb.ErrorCode_Success {
return errors.New(status.GetReason())
}
return nil
// function to report import state to RootCoord.
// retry 10 times, if the rootcoord is down, the report function will cost 20+ seconds
reportFunc := reportImportFunc(node)
returnFailFunc := func(msg string, inputErr error) (*commonpb.Status, error) {
logFields = append(logFields, zap.Error(inputErr))
log.Warn(msg, logFields...)
importResult.State = commonpb.ImportState_ImportFailed
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.FailedReason, Value: inputErr.Error()})
reportFunc(importResult)
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: inputErr.Error(),
}, nil
}
if !node.isHealthy() {
err := merr.WrapErrServiceNotReady(node.GetStateCode().String())
log.Warn("DataNode.SyncSegments failed", zap.Int64("nodeId", paramtable.GetNodeID()), zap.Error(err))
logFields = append(logFields, zap.Error(err))
log.Warn("DataNode import failed, node is not healthy", logFields...)
return merr.Status(err), nil
}
@ -457,22 +469,14 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestTSO),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(paramtable.GetNodeID()),
),
Count: 1,
})
if rep.Status.ErrorCode != commonpb.ErrorCode_Success || err != nil {
msg := "DataNode alloc ts failed"
log.Warn(msg)
importResult.State = commonpb.ImportState_ImportFailed
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.FailedReason, Value: msg})
if reportErr := reportFunc(importResult); reportErr != nil {
log.Warn("fail to report import state to RootCoord", zap.Error(reportErr))
}
if err != nil {
return merr.Status(err), nil
}
return returnFailFunc("DataNode alloc ts failed", err)
}
ts := rep.GetTimestamp()
@ -481,35 +485,28 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
metaService := newMetaService(node.rootCoord, req.GetImportTask().GetCollectionId())
colInfo, err := metaService.getCollectionInfo(newCtx, req.GetImportTask().GetCollectionId(), 0)
if err != nil {
log.Warn("failed to get collection info for collection ID",
zap.Int64("task ID", req.GetImportTask().GetTaskId()),
zap.Int64("collection ID", req.GetImportTask().GetCollectionId()),
zap.Error(err))
importResult.State = commonpb.ImportState_ImportFailed
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.FailedReason, Value: err.Error()})
reportErr := reportFunc(importResult)
if reportErr != nil {
log.Warn("fail to report import state to RootCoord", zap.Error(err))
}
return merr.Status(err), nil
return returnFailFunc("failed to get collection info for collection ID", err)
}
returnFailFunc := func(inputErr error) (*commonpb.Status, error) {
log.Warn("import wrapper failed to parse import request",
zap.Int64("task ID", req.GetImportTask().GetTaskId()),
zap.Error(inputErr))
importResult.State = commonpb.ImportState_ImportFailed
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.FailedReason, Value: inputErr.Error()})
reportErr := reportFunc(importResult)
if reportErr != nil {
log.Warn("fail to report import state to RootCoord", zap.Error(inputErr))
}
return merr.Status(err), nil
// the colInfo doesn't have a collect database name(it is empty). use the database name passed from rootcoord.
partitions, err := node.getPartitions(ctx, req.GetImportTask().GetDatabaseName(), colInfo.GetCollectionName())
if err != nil {
return returnFailFunc("failed to get partition id list", err)
}
partitionIDs, err := importutil.DeduceTargetPartitions(partitions, colInfo.GetSchema(), req.GetImportTask().GetPartitionId())
if err != nil {
return returnFailFunc("failed to decude target partitions", err)
}
collectionInfo, err := importutil.NewCollectionInfo(colInfo.GetSchema(), colInfo.GetShardsNum(), partitionIDs)
if err != nil {
return returnFailFunc("invalid collection info to import", err)
}
// parse files and generate segments
segmentSize := Params.DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024
importWrapper := importutil.NewImportWrapper(newCtx, colInfo.GetSchema(), colInfo.GetShardsNum(), segmentSize, node.allocator.GetIDAlloactor(),
importWrapper := importutil.NewImportWrapper(newCtx, collectionInfo, segmentSize, node.allocator.GetIDAlloactor(),
node.chunkManager, importResult, reportFunc)
importWrapper.SetCallbackFunctions(assignSegmentFunc(node, req),
createBinLogsFunc(node, req, colInfo.GetSchema(), ts),
@ -518,26 +515,73 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
tsStart, tsEnd, err := importutil.ParseTSFromOptions(req.GetImportTask().GetInfos())
isBackup := importutil.IsBackup(req.GetImportTask().GetInfos())
if err != nil {
return returnFailFunc(err)
return returnFailFunc("failed to parse timestamp from import options", err)
}
log.Info("import time range", zap.Uint64("start_ts", tsStart), zap.Uint64("end_ts", tsEnd))
logFields = append(logFields, zap.Uint64("start_ts", tsStart), zap.Uint64("end_ts", tsEnd))
log.Info("import time range", logFields...)
err = importWrapper.Import(req.GetImportTask().GetFiles(),
importutil.ImportOptions{OnlyValidate: false, TsStartPoint: tsStart, TsEndPoint: tsEnd, IsBackup: isBackup})
if err != nil {
return returnFailFunc(err)
return returnFailFunc("failed to import files", err)
}
return merr.Status(nil), nil
resp := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}
return resp, nil
}
func (node *DataNode) getPartitions(ctx context.Context, dbName string, collectionName string) (map[string]int64, error) {
req := &milvuspb.ShowPartitionsRequest{
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_ShowPartitions),
),
DbName: dbName,
CollectionName: collectionName,
}
logFields := []zap.Field{
zap.String("dbName", dbName),
zap.String("collection name", collectionName),
}
resp, err := node.rootCoord.ShowPartitions(ctx, req)
if err != nil {
logFields = append(logFields, zap.Error(err))
log.Warn("failed to get partitions of collection", logFields...)
return nil, err
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("failed to get partitions of collection", logFields...)
return nil, errors.New(resp.Status.Reason)
}
partitionNames := resp.GetPartitionNames()
partitionIDs := resp.GetPartitionIDs()
if len(partitionNames) != len(partitionIDs) {
logFields = append(logFields, zap.Int("number of names", len(partitionNames)), zap.Int("number of ids", len(partitionIDs)))
log.Warn("partition names and ids are unequal", logFields...)
return nil, fmt.Errorf("partition names and ids are unequal, number of names: %d, number of ids: %d",
len(partitionNames), len(partitionIDs))
}
partitions := make(map[string]int64)
for i := 0; i < len(partitionNames); i++ {
partitions[partitionNames[i]] = partitionIDs[i]
}
return partitions, nil
}
// AddImportSegment adds the import segment to the current DataNode.
func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
log.Info("adding segment to DataNode flow graph",
logFields := []zap.Field{
zap.Int64("segment ID", req.GetSegmentId()),
zap.Int64("collection ID", req.GetCollectionId()),
zap.Int64("partition ID", req.GetPartitionId()),
zap.String("channel name", req.GetChannelName()),
zap.Int64("# of rows", req.GetRowNum()))
zap.Int64("# of rows", req.GetRowNum()),
}
log.Info("adding segment to DataNode flow graph", logFields...)
// Fetch the flow graph on the given v-channel.
var ds *dataSyncService
// Retry in case the channel hasn't been watched yet.
@ -545,16 +589,19 @@ func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImpor
var ok bool
ds, ok = node.flowgraphManager.getFlowgraphService(req.GetChannelName())
if !ok {
return merr.WrapErrChannelNotFound(req.ChannelName)
return errors.New("channel not found")
}
return nil
}, retry.Attempts(getFlowGraphServiceAttempts))
if err != nil {
log.Error("channel not found in current DataNode",
zap.String("channel name", req.GetChannelName()),
zap.Int64("node ID", paramtable.GetNodeID()))
logFields = append(logFields, zap.Int64("node ID", paramtable.GetNodeID()))
log.Error("channel not found in current DataNode", logFields...)
return &datapb.AddImportSegmentResponse{
Status: merr.Status(err),
Status: &commonpb.Status{
// TODO: Add specific error code.
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "channel not found in current DataNode",
},
}, nil
}
// Get the current dml channel position ID, that will be used in segments start positions and end positions.
@ -564,16 +611,19 @@ func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImpor
posID = id
return innerError
}, retry.Attempts(30))
if err != nil {
return &datapb.AddImportSegmentResponse{
Status: merr.Status(merr.WrapErrChannelNotFound(
req.ChannelName, "failed to get channel position", err.Error())),
Status: &commonpb.Status{
// TODO: Add specific error code.
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "failed to get channel position",
},
}, nil
}
// Add the new segment to the channel.
if !ds.channel.hasSegment(req.GetSegmentId(), true) {
log.Info("adding a new segment to channel",
zap.Int64("segment ID", req.GetSegmentId()))
log.Info("adding a new segment to channel", logFields...)
// Add segment as a flushed segment, but set `importing` to true to add extra information of the segment.
// By 'extra information' we mean segment info while adding a `SegmentType_Flushed` typed segment.
if err := ds.channel.addSegment(
@ -597,45 +647,50 @@ func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImpor
recoverTs: req.GetBase().GetTimestamp(),
importing: true,
}); err != nil {
log.Error("failed to add segment to flow graph",
zap.Error(err))
logFields = append(logFields, zap.Error(err))
log.Error("failed to add segment to flow graph", logFields...)
return &datapb.AddImportSegmentResponse{
Status: merr.Status(err),
Status: &commonpb.Status{
// TODO: Add specific error code.
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err.Error(),
},
}, nil
}
}
ds.flushingSegCache.Remove(req.GetSegmentId())
return &datapb.AddImportSegmentResponse{
Status: merr.Status(nil),
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
ChannelPos: posID,
}, nil
}
func assignSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest) importutil.AssignSegmentFunc {
return func(shardID int) (int64, string, error) {
return func(shardID int, partID int64) (int64, string, error) {
chNames := req.GetImportTask().GetChannelNames()
importTaskID := req.GetImportTask().GetTaskId()
logFields := []zap.Field{
zap.Int64("task ID", importTaskID),
zap.Int("shard ID", shardID),
zap.Int64("partition ID", partID),
zap.Int("# of channels", len(chNames)),
zap.Strings("channel names", chNames),
}
if shardID >= len(chNames) {
log.Error("import task returns invalid shard ID",
zap.Int64("task ID", importTaskID),
zap.Int("shard ID", shardID),
zap.Int("# of channels", len(chNames)),
zap.Strings("channel names", chNames),
)
log.Error("import task returns invalid shard ID", logFields...)
return 0, "", fmt.Errorf("syncSegmentID Failed: invalid shard ID %d", shardID)
}
tr := timerecord.NewTimeRecorder("import callback function")
tr := timerecord.NewTimeRecorder("assign segment function")
defer tr.Elapse("finished")
colID := req.GetImportTask().GetCollectionId()
partID := req.GetImportTask().GetPartitionId()
segmentIDReq := composeAssignSegmentIDRequest(1, shardID, chNames, colID, partID)
targetChName := segmentIDReq.GetSegmentIDRequests()[0].GetChannelName()
log.Info("target channel for the import task",
zap.Int64("task ID", importTaskID),
zap.Int("shard ID", shardID),
zap.String("target channel name", targetChName))
logFields = append(logFields, zap.String("target channel name", targetChName))
log.Info("assign segment for the import task", logFields...)
resp, err := node.dataCoord.AssignSegmentID(context.Background(), segmentIDReq)
if err != nil {
return 0, "", fmt.Errorf("syncSegmentID Failed:%w", err)
@ -646,42 +701,34 @@ func assignSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest) importutil
if len(resp.SegIDAssignments) == 0 || resp.SegIDAssignments[0] == nil {
return 0, "", fmt.Errorf("syncSegmentID Failed: the collection was dropped")
}
segmentID := resp.SegIDAssignments[0].SegID
log.Info("new segment assigned",
zap.Int64("task ID", importTaskID),
zap.Int64("segmentID", segmentID),
zap.Int("shard ID", shardID),
zap.String("target channel name", targetChName))
logFields = append(logFields, zap.Int64("segment ID", segmentID))
log.Info("new segment assigned", logFields...)
// call report to notify the rootcoord update the segment id list for this task
// ignore the returned error, since even report failed the segments still can be cleaned
retry.Do(context.Background(), func() error {
importResult := &rootcoordpb.ImportResult{
Status: merr.Status(nil),
TaskId: req.GetImportTask().TaskId,
DatanodeId: paramtable.GetNodeID(),
State: commonpb.ImportState_ImportStarted,
Segments: []int64{segmentID},
AutoIds: make([]int64, 0),
RowCount: 0,
}
status, err := node.rootCoord.ReportImport(context.Background(), importResult)
if err != nil {
log.Error("fail to report import state to RootCoord", zap.Error(err))
return err
}
if status != nil && status.ErrorCode != commonpb.ErrorCode_Success {
return errors.New(status.GetReason())
}
return nil
})
// retry 10 times, if the rootcoord is down, the report function will cost 20+ seconds
importResult := &rootcoordpb.ImportResult{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
TaskId: req.GetImportTask().TaskId,
DatanodeId: paramtable.GetNodeID(),
State: commonpb.ImportState_ImportStarted,
Segments: []int64{segmentID},
AutoIds: make([]int64, 0),
RowCount: 0,
}
reportFunc := reportImportFunc(node)
reportFunc(importResult)
return segmentID, targetChName, nil
}
}
func createBinLogsFunc(node *DataNode, req *datapb.ImportTaskRequest, schema *schemapb.CollectionSchema, ts Timestamp) importutil.CreateBinlogsFunc {
return func(fields map[storage.FieldID]storage.FieldData, segmentID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
return func(fields importutil.BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
var rowNum int
for _, field := range fields {
rowNum = field.RowNum()
@ -690,34 +737,29 @@ func createBinLogsFunc(node *DataNode, req *datapb.ImportTaskRequest, schema *sc
chNames := req.GetImportTask().GetChannelNames()
importTaskID := req.GetImportTask().GetTaskId()
logFields := []zap.Field{
zap.Int64("task ID", importTaskID),
zap.Int64("partition ID", partID),
zap.Int64("segment ID", segmentID),
zap.Int("# of channels", len(chNames)),
zap.Strings("channel names", chNames),
}
if rowNum <= 0 {
log.Info("fields data is empty, no need to generate binlog",
zap.Int64("task ID", importTaskID),
zap.Int("# of channels", len(chNames)),
zap.Strings("channel names", chNames),
)
log.Info("fields data is empty, no need to generate binlog", logFields...)
return nil, nil, nil
}
colID := req.GetImportTask().GetCollectionId()
partID := req.GetImportTask().GetPartitionId()
fieldInsert, fieldStats, err := createBinLogs(rowNum, schema, ts, fields, node, segmentID, colID, partID)
if err != nil {
log.Error("failed to create binlogs",
zap.Int64("task ID", importTaskID),
zap.Int("# of channels", len(chNames)),
zap.Strings("channel names", chNames),
zap.Any("err", err),
)
logFields = append(logFields, zap.Any("err", err))
log.Error("failed to create binlogs", logFields...)
return nil, nil, err
}
log.Info("new binlog created",
zap.Int64("task ID", importTaskID),
zap.Int64("segmentID", segmentID),
zap.Int("insert log count", len(fieldInsert)),
zap.Int("stats log count", len(fieldStats)))
logFields = append(logFields, zap.Int("insert log count", len(fieldInsert)), zap.Int("stats log count", len(fieldStats)))
log.Info("new binlog created", logFields...)
return fieldInsert, fieldStats, err
}
@ -725,13 +767,17 @@ func createBinLogsFunc(node *DataNode, req *datapb.ImportTaskRequest, schema *sc
func saveSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest, res *rootcoordpb.ImportResult, ts Timestamp) importutil.SaveSegmentFunc {
importTaskID := req.GetImportTask().GetTaskId()
return func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64) error {
log.Info("adding segment to the correct DataNode flow graph and saving binlog paths",
return func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64,
targetChName string, rowCount int64, partID int64) error {
logFields := []zap.Field{
zap.Int64("task ID", importTaskID),
zap.Int64("segmentID", segmentID),
zap.String("targetChName", targetChName),
zap.Int64("rowCount", rowCount),
zap.Uint64("ts", ts))
zap.Int64("partition ID", partID),
zap.Int64("segment ID", segmentID),
zap.String("target channel name", targetChName),
zap.Int64("row count", rowCount),
zap.Uint64("ts", ts),
}
log.Info("adding segment to the correct DataNode flow graph and saving binlog paths", logFields...)
err := retry.Do(context.Background(), func() error {
// Ask DataCoord to save binlog path and add segment to the corresponding DataNode flow graph.
@ -743,14 +789,14 @@ func saveSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest, res *rootcoo
SegmentId: segmentID,
ChannelName: targetChName,
CollectionId: req.GetImportTask().GetCollectionId(),
PartitionId: req.GetImportTask().GetPartitionId(),
PartitionId: partID,
RowNum: rowCount,
SaveBinlogPathReq: &datapb.SaveBinlogPathsRequest{
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(node.session.ServerID),
commonpbutil.WithSourceID(paramtable.GetNodeID()),
),
SegmentID: segmentID,
CollectionID: req.GetImportTask().GetCollectionId(),
@ -773,9 +819,9 @@ func saveSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest, res *rootcoo
if err != nil {
return fmt.Errorf(err.Error())
}
if resp.ErrorCode != commonpb.ErrorCode_Success && resp.ErrorCode != commonpb.ErrorCode_DataCoordNA {
if resp.ErrorCode != commonpb.ErrorCode_Success && resp.ErrorCode != commonpb.ErrorCode_NotReadyServe {
return retry.Unrecoverable(fmt.Errorf("failed to save import segment, reason = %s", resp.Reason))
} else if resp.ErrorCode == commonpb.ErrorCode_DataCoordNA {
} else if resp.ErrorCode == commonpb.ErrorCode_NotReadyServe {
return fmt.Errorf("failed to save import segment: %s", resp.GetReason())
}
return nil
@ -784,9 +830,7 @@ func saveSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest, res *rootcoo
log.Warn("failed to save import segment", zap.Error(err))
return err
}
log.Info("segment imported and persisted",
zap.Int64("task ID", importTaskID),
zap.Int64("segmentID", segmentID))
log.Info("segment imported and persisted", logFields...)
res.Segments = append(res.Segments, segmentID)
res.RowCount += rowCount
return nil
@ -799,8 +843,11 @@ func composeAssignSegmentIDRequest(rowNum int, shardID int, chNames []string,
// all the fields row count are same, checked by ImportWrapper
// ask DataCoord to alloc a new segment
log.Info("import task flush segment",
zap.Int("rowCount", rowNum),
zap.Any("channel names", chNames),
zap.Int("shard ID", shardID))
zap.Int64("collectionID", collID),
zap.Int64("partitionID", partID),
zap.Int("shardID", shardID))
segReqs := []*datapb.SegmentIDRequest{
{
ChannelName: chNames[shardID],
@ -932,6 +979,24 @@ func createBinLogs(rowNum int, schema *schemapb.CollectionSchema, ts Timestamp,
return fieldInsert, fieldStats, nil
}
func reportImportFunc(node *DataNode) importutil.ReportFunc {
return func(importResult *rootcoordpb.ImportResult) error {
err := retry.Do(context.Background(), func() error {
status, err := node.rootCoord.ReportImport(context.Background(), importResult)
if err != nil {
log.Error("fail to report import state to RootCoord", zap.Error(err))
return err
}
if status != nil && status.ErrorCode != commonpb.ErrorCode_Success {
return errors.New(status.GetReason())
}
return nil
}, retry.Attempts(node.reportImportRetryTimes))
return err
}
}
func logDupFlush(cID, segID int64) {
log.Info("segment is already being flushed, ignoring flush request",
zap.Int64("collection ID", cID),

View File

@ -387,6 +387,7 @@ func (s *DataNodeServicesSuite) TestImport() {
collectionID: 100,
pkType: schemapb.DataType_Int64,
}
s.node.reportImportRetryTimes = 1 // save test time cost from 440s to 180s
s.Run("test normal", func() {
content := []byte(`{
"rows":[
@ -567,6 +568,42 @@ func (s *DataNodeServicesSuite) TestImport() {
s.Assert().NoError(err)
s.Assert().False(merr.Ok(stat))
})
s.Run("test get partitions", func() {
s.node.rootCoord = &RootCoordFactory{
ShowPartitionsErr: true,
}
_, err := s.node.getPartitions(context.Background(), "", "")
s.Assert().Error(err)
s.node.rootCoord = &RootCoordFactory{
ShowPartitionsNotSuccess: true,
}
_, err = s.node.getPartitions(context.Background(), "", "")
s.Assert().Error(err)
s.node.rootCoord = &RootCoordFactory{
ShowPartitionsNames: []string{"a", "b"},
ShowPartitionsIDs: []int64{1},
}
_, err = s.node.getPartitions(context.Background(), "", "")
s.Assert().Error(err)
s.node.rootCoord = &RootCoordFactory{
ShowPartitionsNames: []string{"a", "b"},
ShowPartitionsIDs: []int64{1, 2},
}
partitions, err := s.node.getPartitions(context.Background(), "", "")
s.Assert().NoError(err)
s.Assert().Contains(partitions, "a")
s.Assert().Equal(int64(1), partitions["a"])
s.Assert().Contains(partitions, "b")
s.Assert().Equal(int64(2), partitions["b"])
})
}
func (s *DataNodeServicesSuite) TestAddImportSegment() {

View File

@ -545,6 +545,7 @@ message ImportTask {
int64 task_id = 6; // id of the task
repeated string files = 7; // file paths to be imported
repeated common.KeyValuePair infos = 8; // extra information about the task, bucket, etc.
string database_name = 16; // Database name
}
message ImportTaskState {
@ -571,6 +572,7 @@ message ImportTaskInfo {
string partition_name = 13; // Partition name for the import task.
repeated common.KeyValuePair infos = 14; // extra information about the task, bucket, etc.
int64 start_ts = 15; // Timestamp when the import task is sent to datanode to execute.
string database_name = 16; // Database name
}
message ImportTaskResponse {

View File

@ -3759,6 +3759,7 @@ type ImportTask struct {
TaskId int64 `protobuf:"varint,6,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
Files []string `protobuf:"bytes,7,rep,name=files,proto3" json:"files,omitempty"`
Infos []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=infos,proto3" json:"infos,omitempty"`
DatabaseName string `protobuf:"bytes,16,opt,name=database_name,json=databaseName,proto3" json:"database_name,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -3845,6 +3846,13 @@ func (m *ImportTask) GetInfos() []*commonpb.KeyValuePair {
return nil
}
func (m *ImportTask) GetDatabaseName() string {
if m != nil {
return m.DatabaseName
}
return ""
}
type ImportTaskState struct {
StateCode commonpb.ImportState `protobuf:"varint,1,opt,name=stateCode,proto3,enum=milvus.proto.common.ImportState" json:"stateCode,omitempty"`
Segments []int64 `protobuf:"varint,2,rep,packed,name=segments,proto3" json:"segments,omitempty"`
@ -3932,6 +3940,7 @@ type ImportTaskInfo struct {
PartitionName string `protobuf:"bytes,13,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
Infos []*commonpb.KeyValuePair `protobuf:"bytes,14,rep,name=infos,proto3" json:"infos,omitempty"`
StartTs int64 `protobuf:"varint,15,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"`
DatabaseName string `protobuf:"bytes,16,opt,name=database_name,json=databaseName,proto3" json:"database_name,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -4068,6 +4077,13 @@ func (m *ImportTaskInfo) GetStartTs() int64 {
return 0
}
func (m *ImportTaskInfo) GetDatabaseName() string {
if m != nil {
return m.DatabaseName
}
return ""
}
type ImportTaskResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
DatanodeId int64 `protobuf:"varint,2,opt,name=datanode_id,json=datanodeId,proto3" json:"datanode_id,omitempty"`
@ -5035,306 +5051,307 @@ func init() {
func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) }
var fileDescriptor_82cd95f524594f49 = []byte{
// 4774 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0xdd, 0x6f, 0x1c, 0x49,
0x5e, 0xe9, 0xf9, 0xf2, 0xcc, 0x6f, 0xc6, 0xe3, 0x71, 0xc5, 0xeb, 0x4c, 0x26, 0x9b, 0x8f, 0xeb,
0x4d, 0x76, 0xbd, 0xd9, 0x5d, 0x27, 0xeb, 0x70, 0xba, 0xbd, 0xcd, 0xed, 0xde, 0xc5, 0xf6, 0x26,
0x3b, 0x10, 0xe7, 0x7c, 0x6d, 0x27, 0x41, 0x7b, 0x48, 0xa3, 0xf6, 0x74, 0x79, 0xdc, 0xe7, 0x99,
0xee, 0x49, 0x77, 0x8f, 0x1d, 0x2f, 0x12, 0x2c, 0x70, 0x20, 0xf1, 0x21, 0x40, 0x08, 0x04, 0xbc,
0x9d, 0x78, 0x40, 0x07, 0xe8, 0x9e, 0x0e, 0x1e, 0xe0, 0xe5, 0x5e, 0x17, 0xf1, 0x80, 0x10, 0x12,
0x12, 0x0f, 0xbc, 0x22, 0x78, 0xe6, 0x1f, 0x40, 0xf5, 0xd1, 0xd5, 0x5f, 0xd5, 0x3d, 0x6d, 0x4f,
0xb2, 0x91, 0xe0, 0x6d, 0xaa, 0xfa, 0x57, 0xbf, 0xaa, 0xfa, 0xd5, 0xef, 0xfb, 0x57, 0x35, 0xd0,
0x32, 0x74, 0x4f, 0xef, 0xf5, 0x6d, 0xdb, 0x31, 0x56, 0xc7, 0x8e, 0xed, 0xd9, 0x68, 0x71, 0x64,
0x0e, 0x8f, 0x26, 0x2e, 0x6b, 0xad, 0x92, 0xcf, 0x9d, 0x46, 0xdf, 0x1e, 0x8d, 0x6c, 0x8b, 0x75,
0x75, 0x9a, 0xa6, 0xe5, 0x61, 0xc7, 0xd2, 0x87, 0xbc, 0xdd, 0x08, 0x0f, 0xe8, 0x34, 0xdc, 0xfe,
0x01, 0x1e, 0xe9, 0xbc, 0x55, 0x1b, 0xb9, 0x03, 0xfe, 0x73, 0xd1, 0xb4, 0x0c, 0xfc, 0x3c, 0x3c,
0x95, 0x3a, 0x07, 0xe5, 0x4f, 0x46, 0x63, 0xef, 0x44, 0xfd, 0x5b, 0x05, 0x1a, 0xf7, 0x87, 0x13,
0xf7, 0x40, 0xc3, 0xcf, 0x26, 0xd8, 0xf5, 0xd0, 0x6d, 0x28, 0xed, 0xe9, 0x2e, 0x6e, 0x2b, 0xd7,
0x94, 0x95, 0xfa, 0xda, 0xeb, 0xab, 0x91, 0x35, 0xf1, 0xd5, 0x6c, 0xb9, 0x83, 0x75, 0xdd, 0xc5,
0x1a, 0x85, 0x44, 0x08, 0x4a, 0xc6, 0x5e, 0x77, 0xb3, 0x5d, 0xb8, 0xa6, 0xac, 0x14, 0x35, 0xfa,
0x1b, 0x5d, 0x01, 0x70, 0xf1, 0x60, 0x84, 0x2d, 0xaf, 0xbb, 0xe9, 0xb6, 0x8b, 0xd7, 0x8a, 0x2b,
0x45, 0x2d, 0xd4, 0x83, 0x54, 0x68, 0xf4, 0xed, 0xe1, 0x10, 0xf7, 0x3d, 0xd3, 0xb6, 0xba, 0x9b,
0xed, 0x12, 0x1d, 0x1b, 0xe9, 0x43, 0x1d, 0xa8, 0x9a, 0x6e, 0x77, 0x34, 0xb6, 0x1d, 0xaf, 0x5d,
0xbe, 0xa6, 0xac, 0x54, 0x35, 0xd1, 0x56, 0xff, 0x53, 0x81, 0x79, 0xbe, 0x6c, 0x77, 0x6c, 0x5b,
0x2e, 0x46, 0x77, 0xa0, 0xe2, 0x7a, 0xba, 0x37, 0x71, 0xf9, 0xca, 0x2f, 0x49, 0x57, 0xbe, 0x43,
0x41, 0x34, 0x0e, 0x2a, 0x5d, 0x7a, 0x7c, 0x69, 0x45, 0xc9, 0xd2, 0xa2, 0xdb, 0x2b, 0x25, 0xb6,
0xb7, 0x02, 0x0b, 0xfb, 0x64, 0x75, 0x3b, 0x01, 0x50, 0x99, 0x02, 0xc5, 0xbb, 0x09, 0x26, 0xcf,
0x1c, 0xe1, 0xef, 0xee, 0xef, 0x60, 0x7d, 0xd8, 0xae, 0xd0, 0xb9, 0x42, 0x3d, 0xea, 0xbf, 0x28,
0xd0, 0x12, 0xe0, 0xfe, 0x19, 0x2d, 0x41, 0xb9, 0x6f, 0x4f, 0x2c, 0x8f, 0x6e, 0x75, 0x5e, 0x63,
0x0d, 0xf4, 0x35, 0x68, 0xf4, 0x0f, 0x74, 0xcb, 0xc2, 0xc3, 0x9e, 0xa5, 0x8f, 0x30, 0xdd, 0x54,
0x4d, 0xab, 0xf3, 0xbe, 0x47, 0xfa, 0x08, 0xe7, 0xda, 0xdb, 0x35, 0xa8, 0x8f, 0x75, 0xc7, 0x33,
0x23, 0x27, 0x13, 0xee, 0xca, 0x3a, 0x18, 0x32, 0x83, 0x49, 0x7f, 0xed, 0xea, 0xee, 0x61, 0x77,
0x93, 0xef, 0x28, 0xd2, 0xa7, 0xfe, 0x48, 0x81, 0xe5, 0x7b, 0xae, 0x6b, 0x0e, 0xac, 0xc4, 0xce,
0x96, 0xa1, 0x62, 0xd9, 0x06, 0xee, 0x6e, 0xd2, 0xad, 0x15, 0x35, 0xde, 0x42, 0x97, 0xa0, 0x36,
0xc6, 0xd8, 0xe9, 0x39, 0xf6, 0xd0, 0xdf, 0x58, 0x95, 0x74, 0x68, 0xf6, 0x10, 0xa3, 0xef, 0xc1,
0xa2, 0x1b, 0x43, 0xc4, 0x78, 0xae, 0xbe, 0xf6, 0xc6, 0x6a, 0x42, 0xa6, 0x56, 0xe3, 0x93, 0x6a,
0xc9, 0xd1, 0xea, 0x17, 0x05, 0x38, 0x2f, 0xe0, 0xd8, 0x5a, 0xc9, 0x6f, 0x42, 0x79, 0x17, 0x0f,
0xc4, 0xf2, 0x58, 0x23, 0x0f, 0xe5, 0xc5, 0x91, 0x15, 0xc3, 0x47, 0x96, 0x47, 0x0c, 0x62, 0xe7,
0x51, 0x4e, 0x9e, 0xc7, 0x55, 0xa8, 0xe3, 0xe7, 0x63, 0xd3, 0xc1, 0x3d, 0xc2, 0x38, 0x94, 0xe4,
0x25, 0x0d, 0x58, 0xd7, 0xae, 0x39, 0x0a, 0xcb, 0xc6, 0x5c, 0x6e, 0xd9, 0x50, 0xff, 0x42, 0x81,
0x0b, 0x89, 0x53, 0xe2, 0xc2, 0xa6, 0x41, 0x8b, 0xee, 0x3c, 0xa0, 0x0c, 0x11, 0x3b, 0x42, 0xf0,
0x37, 0xb3, 0x08, 0x1e, 0x80, 0x6b, 0x89, 0xf1, 0xa1, 0x45, 0x16, 0xf2, 0x2f, 0xf2, 0x10, 0x2e,
0x3c, 0xc0, 0x1e, 0x9f, 0x80, 0x7c, 0xc3, 0xee, 0xd9, 0x15, 0x59, 0x54, 0xaa, 0x0b, 0x71, 0xa9,
0x56, 0xff, 0xb2, 0x20, 0x64, 0x91, 0x4e, 0xd5, 0xb5, 0xf6, 0x6d, 0xf4, 0x3a, 0xd4, 0x04, 0x08,
0xe7, 0x8a, 0xa0, 0x03, 0x7d, 0x03, 0xca, 0x64, 0xa5, 0x8c, 0x25, 0x9a, 0x6b, 0x5f, 0x93, 0xef,
0x29, 0x84, 0x53, 0x63, 0xf0, 0x68, 0x13, 0x9a, 0xae, 0xa7, 0x3b, 0x5e, 0x6f, 0x6c, 0xbb, 0xf4,
0x9c, 0x29, 0xe3, 0xd4, 0xd7, 0x2e, 0x47, 0x31, 0x10, 0x25, 0xbf, 0xe5, 0x0e, 0xb6, 0x39, 0x90,
0x36, 0x4f, 0x07, 0xf9, 0x4d, 0xf4, 0x1d, 0x68, 0x60, 0xcb, 0x08, 0x70, 0x94, 0xf2, 0xe0, 0xa8,
0x63, 0xcb, 0x10, 0x18, 0x82, 0x53, 0x29, 0xe7, 0x3f, 0x95, 0xdf, 0x53, 0xa0, 0x9d, 0x3c, 0x96,
0x59, 0x14, 0xf5, 0x5d, 0x36, 0x08, 0xb3, 0x63, 0xc9, 0x94, 0x6b, 0x71, 0x34, 0x1a, 0x1f, 0xa2,
0xfe, 0x89, 0x02, 0xaf, 0x05, 0xcb, 0xa1, 0x9f, 0x5e, 0x16, 0x8f, 0xa0, 0x9b, 0xd0, 0x32, 0xad,
0xfe, 0x70, 0x62, 0xe0, 0xc7, 0xd6, 0xa7, 0x58, 0x1f, 0x7a, 0x07, 0x27, 0xf4, 0xe4, 0xaa, 0x5a,
0xa2, 0x5f, 0xfd, 0xf7, 0x02, 0x2c, 0xc7, 0xd7, 0x35, 0x0b, 0x91, 0x7e, 0x0e, 0xca, 0xa6, 0xb5,
0x6f, 0xfb, 0x34, 0xba, 0x92, 0x21, 0x8a, 0x64, 0x2e, 0x06, 0x8c, 0x6c, 0x40, 0xbe, 0xf2, 0xea,
0x1f, 0xe0, 0xfe, 0xe1, 0xd8, 0x36, 0xa9, 0x9a, 0x22, 0x28, 0xbe, 0x23, 0x41, 0x21, 0x5f, 0xf1,
0xea, 0x06, 0xc3, 0xb1, 0x21, 0x50, 0x7c, 0x62, 0x79, 0xce, 0x89, 0xb6, 0xd8, 0x8f, 0xf7, 0x77,
0xfa, 0xb0, 0x2c, 0x07, 0x46, 0x2d, 0x28, 0x1e, 0xe2, 0x13, 0xba, 0xe5, 0x9a, 0x46, 0x7e, 0xa2,
0x3b, 0x50, 0x3e, 0xd2, 0x87, 0x13, 0xcc, 0x75, 0xc2, 0x14, 0xce, 0x65, 0xb0, 0x1f, 0x16, 0x3e,
0x50, 0xd4, 0x11, 0x5c, 0x7a, 0x80, 0xbd, 0xae, 0xe5, 0x62, 0xc7, 0x5b, 0x37, 0xad, 0xa1, 0x3d,
0xd8, 0xd6, 0xbd, 0x83, 0x19, 0x94, 0x43, 0x44, 0xce, 0x0b, 0x31, 0x39, 0x57, 0x7f, 0xac, 0xc0,
0xeb, 0xf2, 0xf9, 0xf8, 0x81, 0x76, 0xa0, 0xba, 0x6f, 0xe2, 0xa1, 0x41, 0xb8, 0x46, 0xa1, 0x5c,
0x23, 0xda, 0x44, 0x49, 0x8c, 0x09, 0x30, 0x3f, 0xb7, 0x98, 0x92, 0x10, 0x3e, 0xdf, 0x8e, 0xe7,
0x98, 0xd6, 0xe0, 0xa1, 0xe9, 0x7a, 0x1a, 0x83, 0x0f, 0x71, 0x49, 0x31, 0xbf, 0x70, 0xfe, 0x8e,
0x02, 0x57, 0x1e, 0x60, 0x6f, 0x43, 0xd8, 0x18, 0xf2, 0xdd, 0x74, 0x3d, 0xb3, 0xef, 0xbe, 0x58,
0x1f, 0x30, 0x87, 0xb3, 0xa1, 0xfe, 0x81, 0x02, 0x57, 0x53, 0x17, 0xc3, 0x49, 0xc7, 0x75, 0xa8,
0x6f, 0x61, 0xe4, 0x3a, 0xf4, 0x17, 0xf0, 0xc9, 0x13, 0x72, 0xf8, 0xdb, 0xba, 0xe9, 0x30, 0x1d,
0x7a, 0x46, 0x8b, 0xf2, 0x13, 0x05, 0x2e, 0x3f, 0xc0, 0xde, 0xb6, 0x6f, 0x5f, 0x5f, 0x21, 0x75,
0x08, 0x4c, 0xc8, 0xce, 0xfb, 0x8e, 0x66, 0xa4, 0x4f, 0xfd, 0x7d, 0x76, 0x9c, 0xd2, 0xf5, 0xbe,
0x12, 0x02, 0x5e, 0xa1, 0x92, 0x10, 0x52, 0x11, 0x5c, 0xd8, 0x39, 0xf9, 0xd4, 0x1f, 0x96, 0xa1,
0xf1, 0x84, 0x6b, 0x05, 0x6a, 0x41, 0xe3, 0x94, 0x50, 0xe4, 0x4e, 0x50, 0xc8, 0x9b, 0x92, 0x39,
0x58, 0xeb, 0x30, 0xef, 0x62, 0x7c, 0x78, 0x4a, 0x7b, 0xd9, 0x20, 0x63, 0x84, 0xb1, 0x7b, 0x08,
0x8b, 0x13, 0x8b, 0x7a, 0xe8, 0xd8, 0xe0, 0x1b, 0x60, 0x44, 0x9f, 0xae, 0x4c, 0x93, 0x03, 0xd1,
0xa7, 0x3c, 0x08, 0x08, 0xe1, 0x2a, 0xe7, 0xc2, 0x15, 0x1f, 0x86, 0xba, 0xd0, 0x32, 0x1c, 0x7b,
0x3c, 0xc6, 0x46, 0xcf, 0xf5, 0x51, 0x55, 0xf2, 0xa1, 0xe2, 0xe3, 0x04, 0xaa, 0xdb, 0x70, 0x3e,
0xbe, 0xd2, 0xae, 0x41, 0xfc, 0x42, 0xc2, 0x59, 0xb2, 0x4f, 0xe8, 0x5d, 0x58, 0x4c, 0xc2, 0x57,
0x29, 0x7c, 0xf2, 0x03, 0x7a, 0x0f, 0x50, 0x6c, 0xa9, 0x04, 0xbc, 0xc6, 0xc0, 0xa3, 0x8b, 0xe1,
0xe0, 0x34, 0x38, 0x8d, 0x82, 0x03, 0x03, 0xe7, 0x5f, 0x42, 0xe0, 0x5d, 0x62, 0x5d, 0x23, 0xe0,
0x6e, 0xbb, 0x9e, 0x8f, 0x10, 0x51, 0x64, 0xae, 0xfa, 0xdb, 0x0a, 0x2c, 0x3f, 0xd5, 0xbd, 0xfe,
0xc1, 0xe6, 0x88, 0x33, 0xe8, 0x0c, 0x02, 0xfe, 0x11, 0xd4, 0x8e, 0x38, 0x33, 0xfa, 0x5a, 0xfc,
0xaa, 0x64, 0x41, 0x61, 0xb6, 0xd7, 0x82, 0x11, 0x24, 0x20, 0x5a, 0xba, 0x1f, 0x0a, 0x0c, 0x5f,
0x81, 0xaa, 0x99, 0x12, 0xd1, 0xaa, 0xcf, 0x01, 0xf8, 0xe2, 0xb6, 0xdc, 0xc1, 0x19, 0xd6, 0xf5,
0x01, 0xcc, 0x71, 0x6c, 0x5c, 0x97, 0x4c, 0x3b, 0x30, 0x1f, 0x5c, 0xfd, 0x51, 0x05, 0xea, 0xa1,
0x0f, 0xa8, 0x09, 0x05, 0xa1, 0x24, 0x0a, 0x92, 0xdd, 0x15, 0xa6, 0xc7, 0x50, 0xc5, 0x64, 0x0c,
0x75, 0x03, 0x9a, 0x26, 0x35, 0xde, 0x3d, 0x7e, 0x2a, 0xd4, 0x57, 0xae, 0x69, 0xf3, 0xac, 0x97,
0xb3, 0x08, 0xba, 0x02, 0x75, 0x6b, 0x32, 0xea, 0xd9, 0xfb, 0x3d, 0xc7, 0x3e, 0x76, 0x79, 0x30,
0x56, 0xb3, 0x26, 0xa3, 0xef, 0xee, 0x6b, 0xf6, 0xb1, 0x1b, 0xf8, 0xfb, 0x95, 0x53, 0xfa, 0xfb,
0x57, 0xa0, 0x3e, 0xd2, 0x9f, 0x13, 0xac, 0x3d, 0x6b, 0x32, 0xa2, 0x71, 0x5a, 0x51, 0xab, 0x8d,
0xf4, 0xe7, 0x9a, 0x7d, 0xfc, 0x68, 0x32, 0x42, 0x2b, 0xd0, 0x1a, 0xea, 0xae, 0xd7, 0x0b, 0x07,
0x7a, 0x55, 0x1a, 0xe8, 0x35, 0x49, 0xff, 0x27, 0x41, 0xb0, 0x97, 0x8c, 0x1c, 0x6a, 0x67, 0x8b,
0x1c, 0x8c, 0xd1, 0x30, 0xc0, 0x01, 0xb9, 0x22, 0x07, 0x63, 0x34, 0x14, 0x18, 0x3e, 0x80, 0xb9,
0x3d, 0xea, 0x08, 0x65, 0x89, 0xe8, 0x7d, 0xe2, 0x03, 0x31, 0x7f, 0x49, 0xf3, 0xc1, 0xd1, 0xb7,
0xa0, 0x46, 0xed, 0x0f, 0x1d, 0xdb, 0xc8, 0x35, 0x36, 0x18, 0x40, 0x46, 0x1b, 0x78, 0xe8, 0xe9,
0x74, 0xf4, 0x7c, 0xbe, 0xd1, 0x62, 0x00, 0xd1, 0x8f, 0x7d, 0x07, 0xeb, 0x1e, 0x36, 0xd6, 0x4f,
0x36, 0xec, 0xd1, 0x58, 0xa7, 0x2c, 0xd4, 0x6e, 0x52, 0x17, 0x5e, 0xf6, 0x09, 0xbd, 0x09, 0xcd,
0xbe, 0x68, 0xdd, 0x77, 0xec, 0x51, 0x7b, 0x81, 0x4a, 0x4f, 0xac, 0x17, 0x5d, 0x06, 0xf0, 0x35,
0xa3, 0xee, 0xb5, 0x5b, 0xf4, 0xec, 0x6a, 0xbc, 0xe7, 0x1e, 0xcd, 0xde, 0x98, 0x6e, 0x8f, 0xe5,
0x49, 0x4c, 0x6b, 0xd0, 0x5e, 0xa4, 0x33, 0xd6, 0xfd, 0xc4, 0x8a, 0x69, 0x0d, 0xd0, 0x05, 0x98,
0x33, 0xdd, 0xde, 0xbe, 0x7e, 0x88, 0xdb, 0x88, 0x7e, 0xad, 0x98, 0xee, 0x7d, 0xfd, 0x10, 0xab,
0x9f, 0xc3, 0x52, 0xc0, 0x53, 0xa1, 0x43, 0x4c, 0xb2, 0x82, 0x72, 0x06, 0x56, 0xc8, 0xf6, 0x7c,
0xff, 0xbb, 0x04, 0xcb, 0x3b, 0xfa, 0x11, 0x7e, 0xf9, 0x4e, 0x76, 0x2e, 0x3d, 0xf6, 0x10, 0x16,
0xa9, 0x5f, 0xbd, 0x16, 0x5a, 0x4f, 0x86, 0x09, 0x0f, 0x73, 0x41, 0x72, 0x20, 0xfa, 0x36, 0x71,
0x3b, 0x70, 0xff, 0x70, 0x9b, 0xc4, 0x28, 0xbe, 0xf9, 0xbe, 0x2c, 0xc1, 0xb3, 0x21, 0xa0, 0xb4,
0xf0, 0x08, 0xb4, 0x0d, 0x0b, 0xd1, 0x13, 0xf0, 0x0d, 0xf7, 0x5b, 0x99, 0x01, 0x6c, 0x40, 0x7d,
0xad, 0x19, 0x39, 0x0c, 0x17, 0xb5, 0x61, 0x8e, 0x5b, 0x5d, 0xaa, 0x24, 0xaa, 0x9a, 0xdf, 0x44,
0xdb, 0x70, 0x9e, 0xed, 0x60, 0x87, 0xcb, 0x02, 0xdb, 0x7c, 0x35, 0xd7, 0xe6, 0x65, 0x43, 0xa3,
0xa2, 0x54, 0x3b, 0xad, 0x28, 0xb5, 0x61, 0x8e, 0xb3, 0x37, 0xd5, 0x1e, 0x55, 0xcd, 0x6f, 0x92,
0x63, 0x0e, 0x18, 0xbd, 0x4e, 0xbf, 0x05, 0x1d, 0x64, 0x9c, 0xaf, 0x83, 0x1b, 0x54, 0x07, 0xfb,
0x4d, 0xf5, 0x37, 0x15, 0x80, 0x80, 0xd2, 0x53, 0x52, 0x2f, 0xdf, 0x84, 0xaa, 0x60, 0xfb, 0x5c,
0xd1, 0xa3, 0x00, 0x8f, 0x6b, 0xf9, 0x62, 0x4c, 0xcb, 0xab, 0xff, 0xa4, 0x40, 0x63, 0x93, 0xec,
0xf3, 0xa1, 0x3d, 0xa0, 0x36, 0xe9, 0x06, 0x34, 0x1d, 0xdc, 0xb7, 0x1d, 0xa3, 0x87, 0x2d, 0xcf,
0x31, 0x31, 0x0b, 0xdb, 0x4b, 0xda, 0x3c, 0xeb, 0xfd, 0x84, 0x75, 0x12, 0x30, 0xa2, 0xb8, 0x5d,
0x4f, 0x1f, 0x8d, 0x7b, 0xfb, 0x44, 0x55, 0x14, 0x18, 0x98, 0xe8, 0xa5, 0x9a, 0xe2, 0x6b, 0xd0,
0x08, 0xc0, 0x3c, 0x9b, 0xce, 0x5f, 0xd2, 0xea, 0xa2, 0x6f, 0xd7, 0x46, 0xd7, 0xa1, 0x49, 0x09,
0xdd, 0x1b, 0xda, 0x83, 0x1e, 0x09, 0x06, 0xb9, 0xb9, 0x6a, 0x18, 0x7c, 0x59, 0xe4, 0x00, 0xa3,
0x50, 0xae, 0xf9, 0x39, 0xe6, 0x06, 0x4b, 0x40, 0xed, 0x98, 0x9f, 0x63, 0xf5, 0x37, 0x14, 0x98,
0xe7, 0xf6, 0x6d, 0x47, 0xa4, 0xc5, 0x69, 0x1e, 0x93, 0x05, 0xe2, 0xf4, 0x37, 0xfa, 0x30, 0x9a,
0xc9, 0xba, 0x2e, 0x15, 0x02, 0x8a, 0x84, 0x7a, 0x55, 0x11, 0xe3, 0x96, 0x27, 0x12, 0xfc, 0x82,
0xd0, 0x54, 0xf7, 0xf4, 0x47, 0xb6, 0xc1, 0x12, 0x6b, 0x6d, 0x98, 0xd3, 0x0d, 0xc3, 0xc1, 0xae,
0xcb, 0xd7, 0xe1, 0x37, 0xc9, 0x97, 0x23, 0xec, 0xb8, 0xfe, 0xc1, 0x16, 0x35, 0xbf, 0x89, 0xbe,
0x05, 0x55, 0xe1, 0x86, 0xb1, 0x0c, 0xc6, 0xb5, 0xf4, 0x75, 0xf2, 0xb8, 0x45, 0x8c, 0x50, 0xff,
0xae, 0x00, 0x4d, 0x2e, 0x83, 0xeb, 0xdc, 0x14, 0x65, 0xb3, 0xd8, 0x3a, 0x34, 0xf6, 0x03, 0xde,
0xcf, 0xca, 0xbb, 0x84, 0x45, 0x24, 0x32, 0x66, 0x1a, 0xaf, 0x45, 0x8d, 0x61, 0x69, 0x26, 0x63,
0x58, 0x3e, 0xad, 0x04, 0x27, 0x9d, 0xa2, 0x8a, 0xc4, 0x29, 0x52, 0x7f, 0x09, 0xea, 0x21, 0x04,
0x54, 0x43, 0xb1, 0xd4, 0x06, 0xa7, 0x98, 0xdf, 0x44, 0x77, 0x02, 0x97, 0x80, 0x91, 0xea, 0xa2,
0x64, 0x2d, 0x31, 0x6f, 0x40, 0xfd, 0x99, 0x02, 0x15, 0x8e, 0xf9, 0x2a, 0xd4, 0xb9, 0x7c, 0x51,
0x27, 0x89, 0x61, 0x07, 0xde, 0x45, 0xbc, 0xa4, 0x17, 0x27, 0x60, 0x17, 0xa1, 0x1a, 0x13, 0xad,
0x39, 0xae, 0x16, 0xfd, 0x4f, 0x21, 0x79, 0x22, 0x9f, 0x88, 0x28, 0xa1, 0x25, 0x28, 0x0f, 0xed,
0x81, 0x28, 0x7b, 0xb0, 0x86, 0xfa, 0xa5, 0x42, 0xb3, 0xd4, 0x1a, 0xee, 0xdb, 0x47, 0xd8, 0x39,
0x99, 0x3d, 0xd1, 0x77, 0x37, 0xc4, 0xe6, 0x39, 0xa3, 0x0d, 0x31, 0x00, 0xdd, 0x0d, 0x0e, 0xa1,
0x28, 0xcb, 0x07, 0x84, 0x4d, 0x11, 0x67, 0xd2, 0xe0, 0x30, 0xfe, 0x50, 0xa1, 0x29, 0xcb, 0xe8,
0x56, 0xce, 0x6a, 0xed, 0x5f, 0x88, 0xe7, 0xae, 0xfe, 0xa3, 0x02, 0x17, 0x53, 0xa8, 0xfb, 0x64,
0xed, 0x15, 0xd0, 0xf7, 0x43, 0xa8, 0x8a, 0xd8, 0xb4, 0x98, 0x2b, 0x36, 0x15, 0xf0, 0xea, 0x1f,
0xb3, 0xc4, 0xb9, 0x84, 0xbc, 0x4f, 0xd6, 0x5e, 0x12, 0x81, 0xe3, 0x39, 0xa6, 0xa2, 0x24, 0xc7,
0xf4, 0xcf, 0x0a, 0x74, 0x82, 0x9c, 0x8e, 0xbb, 0x7e, 0x32, 0x6b, 0xa5, 0xe5, 0xc5, 0xc4, 0x6c,
0xdf, 0x14, 0x45, 0x01, 0xa2, 0x17, 0x73, 0x45, 0x5b, 0x7e, 0x49, 0xc0, 0xa2, 0xe9, 0xe1, 0xe4,
0x86, 0x66, 0x91, 0xca, 0x4e, 0xe8, 0xe0, 0x59, 0x61, 0x20, 0x38, 0xd8, 0x9f, 0x31, 0x26, 0xbd,
0x1f, 0x4d, 0xec, 0xbc, 0x6a, 0x02, 0x86, 0x8b, 0x15, 0x07, 0xbc, 0x58, 0x51, 0x8a, 0x15, 0x2b,
0x78, 0xbf, 0x3a, 0xa2, 0x2c, 0x90, 0xd8, 0xc0, 0xcb, 0x22, 0xd8, 0x6f, 0x29, 0xd0, 0xe6, 0xb3,
0xd0, 0x39, 0x49, 0xc0, 0x35, 0xc4, 0x1e, 0x36, 0xbe, 0xea, 0xf4, 0xc3, 0x9f, 0x15, 0xa0, 0x15,
0x76, 0x6c, 0xa8, 0x6f, 0xf2, 0x75, 0x28, 0xd3, 0xec, 0x0d, 0x5f, 0xc1, 0x54, 0xed, 0xc0, 0xa0,
0x89, 0x65, 0xa4, 0xde, 0xfc, 0xae, 0xeb, 0x3b, 0x2e, 0xbc, 0x19, 0x78, 0x57, 0xc5, 0xd3, 0x7b,
0x57, 0xaf, 0x43, 0x8d, 0x58, 0x2e, 0x7b, 0x42, 0xf0, 0xb2, 0x0a, 0x72, 0xd0, 0x81, 0x3e, 0x82,
0x0a, 0xbb, 0x17, 0xc2, 0x0b, 0x78, 0x37, 0xa2, 0xa8, 0xf9, 0x9d, 0x91, 0x50, 0x02, 0x9e, 0x76,
0x68, 0x7c, 0x10, 0x39, 0xa3, 0xb1, 0x63, 0x0f, 0xa8, 0x1b, 0x46, 0x8c, 0x5a, 0x59, 0x13, 0x6d,
0xf5, 0xe7, 0x61, 0x39, 0x88, 0x83, 0xd9, 0x92, 0xce, 0xca, 0xd0, 0xea, 0xbf, 0x29, 0x70, 0x7e,
0xe7, 0xc4, 0xea, 0xc7, 0x45, 0x63, 0x19, 0x2a, 0xe3, 0xa1, 0x1e, 0xa4, 0x85, 0x79, 0x8b, 0x96,
0xdc, 0xd9, 0xdc, 0xd8, 0x20, 0x26, 0x9c, 0xd1, 0xb3, 0x2e, 0xfa, 0x76, 0xed, 0xa9, 0x9e, 0xd5,
0x0d, 0x11, 0xb8, 0x63, 0x83, 0x39, 0x0b, 0x2c, 0xed, 0x35, 0x2f, 0x7a, 0xa9, 0xb3, 0xf0, 0x11,
0x00, 0xf5, 0xa7, 0x7a, 0xa7, 0xf1, 0xa1, 0xe8, 0x88, 0x87, 0xc4, 0x62, 0xfe, 0xb4, 0x00, 0xed,
0x10, 0x95, 0xbe, 0x6a, 0xf7, 0x32, 0x25, 0x28, 0x2c, 0xbe, 0xa0, 0xa0, 0xb0, 0x34, 0xbb, 0x4b,
0x59, 0x96, 0xb9, 0x94, 0xbf, 0x56, 0x84, 0x66, 0x40, 0xb5, 0xed, 0xa1, 0x6e, 0xa5, 0x72, 0xc2,
0x0e, 0x34, 0xdd, 0x08, 0x55, 0x39, 0x9d, 0xde, 0x91, 0xc9, 0x50, 0xca, 0x41, 0x68, 0x31, 0x14,
0xe8, 0x32, 0x3d, 0x74, 0xc7, 0x63, 0x89, 0x36, 0xe6, 0x1f, 0xd6, 0x98, 0xb0, 0x9a, 0x23, 0x8c,
0xde, 0x05, 0xc4, 0x25, 0xac, 0x67, 0x5a, 0x3d, 0x17, 0xf7, 0x6d, 0xcb, 0x60, 0xb2, 0x57, 0xd6,
0x5a, 0xfc, 0x4b, 0xd7, 0xda, 0x61, 0xfd, 0xe8, 0xeb, 0x50, 0xf2, 0x4e, 0xc6, 0xcc, 0x59, 0x6c,
0x4a, 0xdd, 0xad, 0x60, 0x5d, 0xbb, 0x27, 0x63, 0xac, 0x51, 0x70, 0xff, 0x6a, 0x90, 0xe7, 0xe8,
0x47, 0xdc, 0xf3, 0x2e, 0x69, 0xa1, 0x9e, 0x70, 0x9c, 0x3c, 0x17, 0x89, 0x93, 0x19, 0x67, 0xfb,
0x02, 0xdd, 0xf3, 0xbc, 0x21, 0x4d, 0x15, 0x52, 0xce, 0xf6, 0x7b, 0x77, 0xbd, 0x21, 0xd9, 0xa4,
0x67, 0x7b, 0xfa, 0x90, 0xc9, 0x47, 0x8d, 0x6b, 0x0e, 0xd2, 0x43, 0xa3, 0xdc, 0x7f, 0x25, 0x9a,
0x4f, 0x2c, 0x4c, 0xc3, 0xee, 0x64, 0x98, 0x2e, 0x8f, 0xd9, 0x99, 0x9b, 0x69, 0xa2, 0xf8, 0x6d,
0xa8, 0x73, 0xae, 0x38, 0x05, 0x57, 0x01, 0x1b, 0xf2, 0x30, 0x83, 0xcd, 0xcb, 0x2f, 0x88, 0xcd,
0x2b, 0x67, 0xc8, 0x7d, 0xc8, 0xcf, 0x46, 0xfd, 0xb1, 0x02, 0xaf, 0x25, 0xb4, 0x66, 0x26, 0x69,
0xb3, 0x23, 0x6f, 0xae, 0x4d, 0xe3, 0x28, 0xb9, 0x6d, 0xb8, 0x0b, 0x15, 0x87, 0x62, 0xe7, 0xe5,
0xb0, 0x37, 0x32, 0x99, 0x8f, 0x2d, 0x44, 0xe3, 0x43, 0xd4, 0x3f, 0x52, 0xe0, 0x42, 0x72, 0xa9,
0x33, 0x18, 0xfc, 0x75, 0x98, 0x63, 0xa8, 0x7d, 0x19, 0x5d, 0xc9, 0x96, 0xd1, 0x80, 0x38, 0x9a,
0x3f, 0x50, 0xdd, 0x81, 0x65, 0xdf, 0x2f, 0x08, 0x48, 0xbf, 0x85, 0x3d, 0x3d, 0x23, 0xee, 0xbc,
0x0a, 0x75, 0x16, 0xc0, 0xb0, 0x78, 0x8e, 0x55, 0x0f, 0x61, 0x4f, 0x24, 0xfa, 0xd4, 0xff, 0x52,
0x60, 0x89, 0x1a, 0xd6, 0x78, 0x29, 0x28, 0x4f, 0x6d, 0x52, 0x15, 0xb7, 0xbf, 0x1e, 0xe9, 0x23,
0x7e, 0x43, 0xa5, 0xa6, 0x45, 0xfa, 0x50, 0x37, 0x99, 0x07, 0x94, 0xe6, 0x27, 0x82, 0x62, 0xec,
0xa6, 0xee, 0xe9, 0xb4, 0x16, 0x1b, 0x4f, 0x00, 0x06, 0x06, 0xbd, 0x74, 0x06, 0x83, 0xae, 0x3e,
0x84, 0xd7, 0x62, 0x3b, 0x9d, 0xe1, 0x44, 0xd5, 0xbf, 0x52, 0xc8, 0x71, 0x44, 0x6e, 0xfa, 0x9c,
0xdd, 0xa9, 0xbd, 0x2c, 0x6a, 0x50, 0x3d, 0xd3, 0x88, 0x2b, 0x11, 0x03, 0x7d, 0x0c, 0x35, 0x0b,
0x1f, 0xf7, 0xc2, 0x7e, 0x52, 0x0e, 0x8f, 0xbf, 0x6a, 0xe1, 0x63, 0xfa, 0x4b, 0x7d, 0x04, 0x17,
0x12, 0x4b, 0x9d, 0x65, 0xef, 0xff, 0xa0, 0xc0, 0xc5, 0x4d, 0xc7, 0x1e, 0x3f, 0x31, 0x1d, 0x6f,
0xa2, 0x0f, 0xa3, 0x65, 0xee, 0x33, 0x6c, 0x3f, 0xc7, 0x2d, 0xc2, 0x4f, 0x13, 0xb1, 0xe5, 0xbb,
0x12, 0x09, 0x4a, 0x2e, 0x8a, 0x6f, 0x3a, 0xe4, 0x5f, 0xff, 0x47, 0x51, 0xb6, 0x78, 0x0e, 0x37,
0xc5, 0x2f, 0xc9, 0x13, 0x7c, 0x48, 0xf3, 0xf0, 0xc5, 0xb3, 0xe6, 0xe1, 0x53, 0xd4, 0x7b, 0xe9,
0x05, 0xa9, 0xf7, 0x53, 0x27, 0xc6, 0x36, 0x20, 0x5a, 0x23, 0xa1, 0xd6, 0xf9, 0xb4, 0x75, 0x95,
0x8f, 0x00, 0x82, 0x52, 0x01, 0xbf, 0x99, 0x39, 0x05, 0x43, 0x68, 0x00, 0x39, 0x23, 0x61, 0x40,
0xb9, 0x7d, 0x0f, 0xa5, 0xa8, 0xbf, 0x07, 0x1d, 0x19, 0x6f, 0xce, 0xc2, 0xef, 0x3f, 0x2d, 0x00,
0x74, 0xc5, 0x3d, 0xde, 0xb3, 0x59, 0x80, 0x37, 0x20, 0xe4, 0x83, 0x04, 0x52, 0x1e, 0xe6, 0x1d,
0x83, 0x08, 0x82, 0x88, 0x52, 0x09, 0x4c, 0x22, 0x72, 0x35, 0x28, 0x9e, 0x90, 0xac, 0x30, 0x56,
0x88, 0x2b, 0xdd, 0x4b, 0x50, 0x73, 0xec, 0xe3, 0x1e, 0x11, 0x2e, 0xc3, 0xbf, 0xa8, 0xec, 0xd8,
0xc7, 0x44, 0xe4, 0x0c, 0x74, 0x01, 0xe6, 0x3c, 0xdd, 0x3d, 0x24, 0xf8, 0x59, 0xb2, 0xae, 0x42,
0x9a, 0x5d, 0x03, 0x2d, 0x41, 0x79, 0xdf, 0x1c, 0x62, 0x76, 0x27, 0xa2, 0xa6, 0xb1, 0x06, 0xfa,
0x86, 0x7f, 0xb7, 0xae, 0x9a, 0xfb, 0x0e, 0x0d, 0x85, 0x57, 0xbf, 0x54, 0x60, 0x21, 0xa0, 0x1a,
0x55, 0x3b, 0x44, 0x93, 0x51, 0x2d, 0xb6, 0x61, 0x1b, 0x4c, 0x41, 0x34, 0x53, 0xec, 0x00, 0x1b,
0xc8, 0x74, 0x55, 0x30, 0x24, 0x2b, 0x70, 0x26, 0xfb, 0x22, 0x9b, 0x36, 0x0d, 0x3f, 0x95, 0x53,
0x71, 0xec, 0xe3, 0xae, 0x21, 0xa8, 0xc1, 0x6e, 0x21, 0xb3, 0x30, 0x91, 0x50, 0x63, 0x83, 0x5e,
0x44, 0x7e, 0x03, 0xe6, 0xb1, 0xe3, 0xd8, 0x4e, 0x6f, 0x84, 0x5d, 0x57, 0x1f, 0x60, 0xee, 0x95,
0x37, 0x68, 0xe7, 0x16, 0xeb, 0x53, 0xff, 0xb4, 0x04, 0xcd, 0x60, 0x2b, 0x7e, 0x31, 0xde, 0x34,
0xfc, 0x62, 0xbc, 0x49, 0x8e, 0x0e, 0x1c, 0xa6, 0x00, 0xc5, 0xe1, 0xae, 0x17, 0xda, 0x8a, 0x56,
0xe3, 0xbd, 0x5d, 0x83, 0x18, 0x63, 0x22, 0x5a, 0x96, 0x6d, 0xe0, 0xe0, 0x70, 0xc1, 0xef, 0xe2,
0x67, 0x1b, 0xe1, 0x91, 0x52, 0x0e, 0x1e, 0x29, 0xe7, 0xe0, 0x91, 0x8a, 0x84, 0x47, 0x96, 0xa1,
0xb2, 0x37, 0xe9, 0x1f, 0x62, 0x8f, 0xfb, 0x69, 0xbc, 0x15, 0xe5, 0x9d, 0x6a, 0x8c, 0x77, 0x04,
0x8b, 0xd4, 0xc2, 0x2c, 0x72, 0x09, 0x6a, 0xac, 0x3e, 0xdc, 0xf3, 0x5c, 0x5a, 0xf1, 0x2a, 0x6a,
0x55, 0xd6, 0xb1, 0xeb, 0xa2, 0x0f, 0x7c, 0x27, 0xae, 0x4e, 0x85, 0x45, 0x95, 0xe8, 0x9a, 0x18,
0x97, 0xf8, 0x2e, 0xdc, 0x5b, 0xb0, 0x10, 0x22, 0x07, 0xb5, 0x0c, 0xac, 0x2c, 0x16, 0xf2, 0xf1,
0xa9, 0x71, 0xb8, 0x01, 0xcd, 0x80, 0x24, 0x14, 0x6e, 0x9e, 0x85, 0x56, 0xa2, 0x97, 0x82, 0x09,
0x4e, 0x6e, 0x9e, 0x8e, 0x93, 0xd1, 0x45, 0xa8, 0xf2, 0x98, 0xc8, 0x6d, 0x2f, 0x44, 0xd2, 0x17,
0xea, 0x0f, 0x00, 0x05, 0xab, 0x9f, 0xcd, 0x47, 0x8c, 0xb1, 0x47, 0x21, 0xce, 0x1e, 0xea, 0x5f,
0x2b, 0xb0, 0x18, 0x9e, 0xec, 0xac, 0xe6, 0xf6, 0x63, 0xa8, 0xb3, 0x9a, 0x63, 0x8f, 0x08, 0xbe,
0xbc, 0x44, 0x18, 0x3b, 0x17, 0x0d, 0x82, 0x77, 0x0c, 0x84, 0xbd, 0x8e, 0x6d, 0xe7, 0xd0, 0xb4,
0x06, 0x3d, 0xb2, 0x32, 0x91, 0x39, 0xe5, 0x9d, 0x8f, 0x48, 0x9f, 0xfa, 0xbb, 0x0a, 0x5c, 0x79,
0x3c, 0x36, 0x74, 0x0f, 0x87, 0xfc, 0x8e, 0x59, 0xaf, 0x13, 0x8a, 0xfb, 0x7c, 0x85, 0x8c, 0x13,
0x0c, 0xcd, 0xe7, 0xf2, 0xfb, 0x7c, 0xc4, 0x5b, 0xe3, 0xab, 0x49, 0x5c, 0xc0, 0x3d, 0xfb, 0x6a,
0x3a, 0x50, 0x3d, 0xe2, 0xe8, 0xfc, 0x97, 0x19, 0x7e, 0x3b, 0x52, 0x83, 0x2d, 0x9e, 0xaa, 0x06,
0xab, 0x6e, 0xc1, 0x45, 0x0d, 0xbb, 0xd8, 0x32, 0x22, 0x1b, 0x39, 0x73, 0x7e, 0x69, 0x0c, 0x1d,
0x19, 0xba, 0x59, 0x38, 0x95, 0xb9, 0xab, 0x3d, 0x87, 0xa0, 0xf5, 0xb8, 0x1e, 0x26, 0x5e, 0x12,
0x9d, 0xc7, 0x53, 0xff, 0xa6, 0x00, 0x17, 0xee, 0x19, 0x06, 0x57, 0xe1, 0xdc, 0x01, 0x7b, 0x59,
0xbe, 0x71, 0xdc, 0x77, 0x2c, 0x26, 0x7d, 0xc7, 0x17, 0xa5, 0x56, 0xb9, 0x81, 0xb1, 0x26, 0x23,
0xdf, 0x70, 0x3a, 0xec, 0x8a, 0xd2, 0x5d, 0x5e, 0xa9, 0x24, 0x31, 0x3c, 0x35, 0x9e, 0xd3, 0x5d,
0xaa, 0xaa, 0x9f, 0x27, 0x53, 0xc7, 0xd0, 0x4e, 0x12, 0x6b, 0x46, 0x3d, 0xe2, 0x53, 0x64, 0x6c,
0xb3, 0x7c, 0x6b, 0x83, 0xf8, 0x4f, 0xb4, 0x6b, 0xdb, 0x76, 0xd5, 0xff, 0x29, 0x40, 0x7b, 0x47,
0x3f, 0xc2, 0xff, 0x7f, 0x0e, 0xe8, 0x33, 0x58, 0x72, 0xf5, 0x23, 0xdc, 0x0b, 0xc5, 0xc2, 0x3d,
0x07, 0x3f, 0xe3, 0xae, 0xe7, 0xdb, 0xb2, 0x8c, 0xb8, 0xf4, 0x62, 0x8f, 0xb6, 0xe8, 0x46, 0xfa,
0x35, 0xfc, 0x0c, 0xbd, 0x09, 0x0b, 0xe1, 0xfb, 0x62, 0x64, 0x69, 0x55, 0x4a, 0xf2, 0xf9, 0xd0,
0x9d, 0xb0, 0xae, 0xa1, 0x3e, 0x83, 0xd7, 0x1f, 0x5b, 0x2e, 0xf6, 0xba, 0xc1, 0xbd, 0xa6, 0x19,
0xa3, 0xc6, 0xab, 0x50, 0x0f, 0x08, 0x9f, 0x78, 0x92, 0x61, 0xb8, 0xaa, 0x0d, 0x9d, 0x2d, 0xdd,
0x39, 0xf4, 0x33, 0xcb, 0x9b, 0xec, 0x12, 0xca, 0x4b, 0x9c, 0x70, 0x5f, 0x5c, 0xc7, 0xd2, 0xf0,
0x3e, 0x76, 0xb0, 0xd5, 0xc7, 0x0f, 0xed, 0xfe, 0x21, 0xf1, 0x35, 0x3c, 0xf6, 0x2a, 0x4e, 0x09,
0x79, 0x9c, 0x9b, 0xa1, 0x47, 0x6f, 0x85, 0xc8, 0xa3, 0xb7, 0x29, 0x8f, 0x28, 0xd5, 0x9f, 0x14,
0x60, 0xf9, 0xde, 0xd0, 0xc3, 0x4e, 0x10, 0xec, 0x9f, 0x26, 0x6f, 0x11, 0x24, 0x12, 0x0a, 0x67,
0xa9, 0x0c, 0xe4, 0x28, 0x1c, 0xca, 0xd2, 0x1e, 0xa5, 0x33, 0xa6, 0x3d, 0xee, 0x01, 0x8c, 0x1d,
0x7b, 0x8c, 0x1d, 0xcf, 0xc4, 0x7e, 0xc4, 0x96, 0xc3, 0x77, 0x09, 0x0d, 0x52, 0x3f, 0x83, 0xd6,
0x83, 0xfe, 0x86, 0x6d, 0xed, 0x9b, 0xce, 0xc8, 0x27, 0x54, 0x42, 0xe8, 0x94, 0x1c, 0x42, 0x57,
0x48, 0x08, 0x9d, 0x6a, 0xc2, 0x62, 0x08, 0xf7, 0x8c, 0x8a, 0x6b, 0xd0, 0xef, 0xed, 0x9b, 0x96,
0x49, 0x2f, 0x79, 0x15, 0xa8, 0xef, 0x09, 0x83, 0xfe, 0x7d, 0xde, 0xa3, 0xfe, 0x50, 0x81, 0x4b,
0x1a, 0x26, 0xc2, 0xe3, 0xdf, 0x97, 0xd9, 0xf5, 0xb6, 0xdc, 0xc1, 0x0c, 0x0e, 0xc5, 0x1d, 0x28,
0x8d, 0xdc, 0x41, 0x4a, 0xad, 0x9b, 0x98, 0xe8, 0xc8, 0x44, 0x1a, 0x05, 0xbe, 0xf9, 0xb1, 0xb8,
0x96, 0xbb, 0x7b, 0x32, 0xc6, 0x68, 0x0e, 0x8a, 0x8f, 0xf0, 0x71, 0xeb, 0x1c, 0x02, 0xa8, 0x3c,
0xb2, 0x9d, 0x91, 0x3e, 0x6c, 0x29, 0xa8, 0x0e, 0x73, 0xbc, 0x62, 0xd8, 0x2a, 0xa0, 0x79, 0xa8,
0x6d, 0xf8, 0x95, 0x95, 0x56, 0xf1, 0xe6, 0x9f, 0x2b, 0xb0, 0x98, 0xa8, 0x69, 0xa1, 0x26, 0xc0,
0x63, 0xab, 0xcf, 0x8b, 0x7d, 0xad, 0x73, 0xa8, 0x01, 0x55, 0xbf, 0xf4, 0xc7, 0xf0, 0xed, 0xda,
0x14, 0xba, 0x55, 0x40, 0x2d, 0x68, 0xb0, 0x81, 0x93, 0x7e, 0x1f, 0xbb, 0x6e, 0xab, 0x28, 0x7a,
0xee, 0xeb, 0xe6, 0x70, 0xe2, 0xe0, 0x56, 0x89, 0xcc, 0xb9, 0x6b, 0x6b, 0x78, 0x88, 0x75, 0x17,
0xb7, 0xca, 0x08, 0x41, 0x93, 0x37, 0xfc, 0x41, 0x95, 0x50, 0x9f, 0x3f, 0x6c, 0xee, 0xe6, 0xd3,
0x70, 0xf5, 0x81, 0x6e, 0xef, 0x02, 0x9c, 0x7f, 0x6c, 0x19, 0x78, 0xdf, 0xb4, 0xb0, 0x11, 0x7c,
0x6a, 0x9d, 0x43, 0xe7, 0x61, 0x61, 0x0b, 0x3b, 0x03, 0x1c, 0xea, 0x2c, 0xa0, 0x45, 0x98, 0xdf,
0x32, 0x9f, 0x87, 0xba, 0x8a, 0x6a, 0xa9, 0xaa, 0xb4, 0x94, 0xb5, 0xbf, 0xbf, 0x01, 0x35, 0x42,
0xcc, 0x0d, 0xdb, 0x76, 0x0c, 0x34, 0x04, 0x44, 0x1f, 0xbf, 0x8c, 0xc6, 0xb6, 0x25, 0x1e, 0xca,
0xa1, 0xd5, 0x18, 0xfd, 0x59, 0x23, 0x09, 0xc8, 0xcf, 0xbb, 0x73, 0x5d, 0x0a, 0x1f, 0x03, 0x56,
0xcf, 0xa1, 0x11, 0x9d, 0x6d, 0xd7, 0x1c, 0xe1, 0x5d, 0xb3, 0x7f, 0xe8, 0xbb, 0x68, 0xb7, 0x53,
0x5e, 0x1b, 0x25, 0x41, 0xfd, 0xf9, 0xde, 0x90, 0xce, 0xc7, 0x5e, 0x27, 0xf9, 0xac, 0xaf, 0x9e,
0x43, 0xcf, 0x60, 0xe9, 0x01, 0x0e, 0xf9, 0xbb, 0xfe, 0x84, 0x6b, 0xe9, 0x13, 0x26, 0x80, 0x4f,
0x39, 0xe5, 0x43, 0x28, 0x53, 0x76, 0x43, 0xb2, 0x82, 0x6c, 0xf8, 0x95, 0x7b, 0xe7, 0x5a, 0x3a,
0x80, 0xc0, 0xf6, 0x03, 0x58, 0x88, 0xbd, 0x7f, 0x45, 0x32, 0x1b, 0x29, 0x7f, 0xc9, 0xdc, 0xb9,
0x99, 0x07, 0x54, 0xcc, 0x35, 0x80, 0x66, 0xf4, 0xd1, 0x0c, 0x5a, 0xc9, 0xf1, 0xf4, 0x8e, 0xcd,
0xf4, 0x76, 0xee, 0x47, 0x7a, 0x94, 0x09, 0x5a, 0xf1, 0x97, 0x99, 0xe8, 0x66, 0x26, 0x82, 0x28,
0xb3, 0xbd, 0x93, 0x0b, 0x56, 0x4c, 0x77, 0x42, 0x99, 0x20, 0xf1, 0x2c, 0x2e, 0xce, 0xe3, 0x3e,
0x9a, 0xb4, 0xf7, 0x7a, 0x9d, 0x5b, 0xb9, 0xe1, 0xc5, 0xd4, 0xbf, 0xce, 0x6e, 0x5d, 0xc9, 0x9e,
0x96, 0xa1, 0xf7, 0xe5, 0xe8, 0x32, 0xde, 0xc4, 0x75, 0xd6, 0x4e, 0x33, 0x44, 0x2c, 0xe2, 0x57,
0xe9, 0x75, 0x29, 0xc9, 0xe3, 0xac, 0xb8, 0xdc, 0xf9, 0xf8, 0xd2, 0xdf, 0x9d, 0x75, 0xde, 0x3f,
0xc5, 0x08, 0xb1, 0x00, 0x3b, 0xfe, 0xf4, 0xd5, 0x17, 0xc3, 0x5b, 0x53, 0xb9, 0xe6, 0x6c, 0x32,
0xf8, 0x7d, 0x58, 0x88, 0x79, 0x8d, 0x28, 0xbf, 0x67, 0xd9, 0xc9, 0xb2, 0x90, 0x4c, 0x24, 0x63,
0xd7, 0xa3, 0x50, 0x0a, 0xf7, 0x4b, 0xae, 0x50, 0x75, 0x6e, 0xe6, 0x01, 0x15, 0x1b, 0x19, 0xc3,
0x62, 0xec, 0xe3, 0x93, 0x35, 0xf4, 0x4e, 0xee, 0xd9, 0x9e, 0xac, 0x75, 0xde, 0xcd, 0x3f, 0xdf,
0x93, 0x35, 0xf5, 0x1c, 0x72, 0xa9, 0x82, 0x8e, 0x5d, 0xb1, 0x41, 0x29, 0x58, 0xe4, 0x57, 0x89,
0x3a, 0xef, 0xe5, 0x84, 0x16, 0xdb, 0x3c, 0x82, 0xf3, 0x92, 0x9b, 0x50, 0xe8, 0xbd, 0x4c, 0xf6,
0x88, 0x5f, 0x01, 0xeb, 0xac, 0xe6, 0x05, 0x0f, 0x99, 0x87, 0x96, 0xbf, 0xae, 0x7b, 0xc3, 0x21,
0x33, 0xfe, 0xef, 0xa6, 0x59, 0xbe, 0x08, 0x58, 0xca, 0x56, 0x53, 0xa1, 0xc5, 0x94, 0xbf, 0x0c,
0x68, 0xe7, 0xc0, 0x3e, 0xa6, 0x5e, 0xda, 0x60, 0xe2, 0xe8, 0xcc, 0xb1, 0x4c, 0x33, 0x80, 0x49,
0xd0, 0x14, 0x41, 0xcc, 0x1c, 0x21, 0x26, 0xef, 0x01, 0x3c, 0xc0, 0xde, 0x16, 0xf6, 0x1c, 0x22,
0xfd, 0x6f, 0xa6, 0xad, 0x9d, 0x03, 0xf8, 0x53, 0xbd, 0x35, 0x15, 0x2e, 0x4c, 0xd0, 0x2d, 0xdd,
0x9a, 0xe8, 0xc3, 0xd0, 0xdb, 0x14, 0x39, 0x41, 0xe3, 0x60, 0xd9, 0x04, 0x4d, 0x42, 0x8b, 0x29,
0x8f, 0x85, 0xff, 0x12, 0x2a, 0xd8, 0x66, 0xfb, 0x2f, 0xc9, 0xcb, 0x42, 0x71, 0xdd, 0x9e, 0x01,
0x2f, 0x26, 0xfe, 0x42, 0xa1, 0xf7, 0xf7, 0x62, 0x00, 0x4f, 0x4d, 0xef, 0x60, 0x7b, 0xa8, 0x5b,
0x6e, 0x9e, 0x25, 0x50, 0xc0, 0x53, 0x2c, 0x81, 0xc3, 0x8b, 0x25, 0x18, 0x30, 0x1f, 0xa9, 0xa3,
0x22, 0xd9, 0x8b, 0x0e, 0x59, 0x4d, 0xb9, 0xb3, 0x32, 0x1d, 0x50, 0xcc, 0x72, 0x00, 0xf3, 0x3e,
0x43, 0x33, 0xe2, 0xbe, 0x9d, 0xc9, 0xf4, 0x11, 0xba, 0xde, 0xcc, 0x03, 0x2a, 0x66, 0x72, 0x01,
0x25, 0x0b, 0x46, 0x28, 0x5f, 0x79, 0x31, 0x4b, 0xf9, 0xa4, 0x57, 0xa1, 0x98, 0x3e, 0x8f, 0x95,
0x64, 0xe5, 0xc6, 0x42, 0x5a, 0x61, 0x96, 0xea, 0xf3, 0x94, 0x0a, 0xaf, 0x7a, 0x0e, 0x3d, 0x85,
0x0a, 0xff, 0x8f, 0x9a, 0xeb, 0xd9, 0x49, 0x5e, 0x8e, 0xfd, 0xc6, 0x14, 0x28, 0x81, 0xf8, 0x10,
0x2e, 0xa4, 0xa4, 0x78, 0xa5, 0x7e, 0x46, 0x76, 0x3a, 0x78, 0x9a, 0x05, 0x14, 0x93, 0x25, 0x32,
0xb8, 0x19, 0x93, 0xa5, 0x65, 0x7b, 0xa7, 0x4d, 0xd6, 0x83, 0xc5, 0x44, 0x86, 0x4c, 0x6a, 0x02,
0xd3, 0xf2, 0x68, 0xd3, 0x26, 0x18, 0xc0, 0x6b, 0xd2, 0x6c, 0x90, 0xd4, 0x3b, 0xc9, 0xca, 0x1b,
0x4d, 0x9b, 0xa8, 0x0f, 0xe7, 0x25, 0x39, 0x20, 0xa9, 0x95, 0x4b, 0xcf, 0x15, 0x4d, 0x9b, 0x64,
0x1f, 0x3a, 0xeb, 0x8e, 0xad, 0x1b, 0x7d, 0xdd, 0xf5, 0x68, 0x5e, 0x86, 0x84, 0x8a, 0xbe, 0x7b,
0x28, 0x8f, 0x1d, 0xa4, 0xd9, 0x9b, 0x69, 0xf3, 0xec, 0x41, 0x9d, 0x1e, 0x25, 0xfb, 0x1f, 0x11,
0x24, 0xb7, 0x11, 0x21, 0x88, 0x14, 0xc5, 0x23, 0x03, 0x14, 0x4c, 0xbd, 0x0b, 0xf5, 0x0d, 0x5a,
0xbb, 0xea, 0x5a, 0x06, 0x7e, 0x1e, 0xb7, 0x57, 0xf4, 0x31, 0xf5, 0x6a, 0x08, 0x20, 0x37, 0x85,
0xe6, 0xa9, 0xd7, 0x6e, 0xe0, 0xe7, 0xec, 0x9c, 0x57, 0x64, 0x78, 0x23, 0x20, 0x29, 0x51, 0x8e,
0x14, 0x32, 0x64, 0xe9, 0x97, 0xc2, 0xbe, 0xac, 0x98, 0xee, 0x56, 0x0a, 0x92, 0x04, 0xa4, 0x3f,
0xeb, 0xed, 0xfc, 0x03, 0xc2, 0x96, 0xc1, 0x5f, 0x57, 0x97, 0x16, 0xce, 0xde, 0xca, 0x5a, 0x7a,
0xd8, 0x41, 0x5d, 0x99, 0x0e, 0x28, 0x66, 0xd9, 0x86, 0x1a, 0xe1, 0x4e, 0x76, 0x3c, 0xd7, 0x65,
0x03, 0xc5, 0xe7, 0xfc, 0x87, 0xb3, 0x89, 0xdd, 0xbe, 0x63, 0xee, 0xf1, 0x43, 0x97, 0x2e, 0x27,
0x02, 0x92, 0x79, 0x38, 0x31, 0x48, 0xb1, 0xf2, 0x09, 0xf5, 0x1a, 0x04, 0xe9, 0xb8, 0xaa, 0x7c,
0x6f, 0xda, 0xf9, 0x46, 0xd5, 0xe4, 0x6a, 0x5e, 0x70, 0x31, 0xed, 0xaf, 0xd0, 0x48, 0x88, 0x7e,
0x5f, 0x9f, 0x98, 0x43, 0x63, 0x9b, 0x5f, 0x63, 0x46, 0xb7, 0xb3, 0x50, 0x45, 0x40, 0x53, 0x1d,
0xc0, 0x8c, 0x11, 0x62, 0xfe, 0x5f, 0x84, 0x9a, 0xc8, 0x10, 0x22, 0xd9, 0x3d, 0xbc, 0x78, 0x6e,
0xb2, 0x73, 0x3d, 0x1b, 0x48, 0x60, 0xc6, 0xb0, 0x24, 0xcb, 0x07, 0x4a, 0x83, 0xec, 0x8c, 0xc4,
0xe1, 0x14, 0xfe, 0x58, 0xfb, 0xb2, 0x06, 0x55, 0x7f, 0xe0, 0x57, 0x9c, 0xba, 0x7a, 0x05, 0xb9,
0xa4, 0xef, 0xc3, 0x42, 0xec, 0xef, 0x21, 0xa4, 0x1a, 0x5c, 0xfe, 0x17, 0x12, 0xd3, 0x44, 0xed,
0x29, 0xff, 0xf7, 0x42, 0x11, 0xe4, 0xbd, 0x95, 0x96, 0x8f, 0x8a, 0xc7, 0x77, 0x53, 0x10, 0xff,
0xdf, 0x0e, 0x71, 0x1e, 0x01, 0x84, 0x82, 0x9b, 0xec, 0x8b, 0xd4, 0xc4, 0x5f, 0x9f, 0x46, 0xad,
0x91, 0x34, 0x7e, 0x79, 0x3b, 0xcf, 0xa5, 0xd4, 0x74, 0x0f, 0x34, 0x3d, 0x6a, 0x79, 0x0c, 0x8d,
0xf0, 0x13, 0x07, 0x24, 0xfd, 0xaf, 0xbc, 0xe4, 0x1b, 0x88, 0x69, 0xbb, 0xd8, 0x3a, 0xa5, 0x63,
0x3b, 0x05, 0x9d, 0x0b, 0x28, 0x59, 0x29, 0x97, 0x06, 0x02, 0xa9, 0xf5, 0x79, 0x69, 0x20, 0x90,
0x5e, 0x7e, 0x67, 0x69, 0xc9, 0x78, 0xf9, 0x57, 0x9a, 0x96, 0x4c, 0x29, 0xa8, 0x4b, 0xd3, 0x92,
0x69, 0xf5, 0x64, 0xf5, 0xdc, 0xfa, 0x9d, 0xcf, 0xde, 0x1f, 0x98, 0xde, 0xc1, 0x64, 0x8f, 0xec,
0xfe, 0x16, 0x1b, 0xfa, 0x9e, 0x69, 0xf3, 0x5f, 0xb7, 0x7c, 0x76, 0xbf, 0x45, 0xb1, 0xdd, 0x22,
0xd8, 0xc6, 0x7b, 0x7b, 0x15, 0xda, 0xba, 0xf3, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xef, 0x87,
0x8e, 0x2d, 0xb9, 0x55, 0x00, 0x00,
// 4793 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3c, 0xdd, 0x8f, 0x1c, 0x47,
0x5e, 0xee, 0xf9, 0xda, 0x99, 0xdf, 0xcc, 0xce, 0xce, 0x96, 0x9d, 0xf5, 0x78, 0x1c, 0x7f, 0x5c,
0xdb, 0x4e, 0x36, 0x4e, 0x62, 0x27, 0x6b, 0x4e, 0x97, 0x4b, 0x2e, 0xb9, 0xf3, 0xee, 0xc6, 0xce,
0x80, 0xd7, 0xb7, 0xd7, 0xbb, 0x76, 0x50, 0x0e, 0x69, 0xd4, 0x9e, 0xae, 0x9d, 0xed, 0xdb, 0x9e,
0xee, 0x71, 0x77, 0xcf, 0xae, 0x37, 0x48, 0x10, 0xe0, 0x40, 0xe2, 0x43, 0x80, 0x10, 0x48, 0xf0,
0x76, 0xe2, 0x01, 0x1d, 0xa0, 0x7b, 0x02, 0x1e, 0x40, 0x48, 0xf7, 0x1a, 0xc4, 0x03, 0x42, 0x48,
0x08, 0x1e, 0x78, 0x45, 0xf0, 0xcc, 0x3f, 0x80, 0xea, 0xa3, 0xab, 0xbf, 0xaa, 0x7b, 0x7a, 0x77,
0xec, 0x58, 0x82, 0xb7, 0xa9, 0xea, 0x5f, 0x55, 0xfd, 0xea, 0x57, 0xbf, 0xef, 0x5f, 0xd5, 0x40,
0xc7, 0xd0, 0x7d, 0x7d, 0x30, 0x74, 0x1c, 0xd7, 0xb8, 0x35, 0x71, 0x1d, 0xdf, 0x41, 0xcb, 0x63,
0xd3, 0x3a, 0x9c, 0x7a, 0xac, 0x75, 0x8b, 0x7c, 0xee, 0xb5, 0x86, 0xce, 0x78, 0xec, 0xd8, 0xac,
0xab, 0xd7, 0x36, 0x6d, 0x1f, 0xbb, 0xb6, 0x6e, 0xf1, 0x76, 0x2b, 0x3a, 0xa0, 0xd7, 0xf2, 0x86,
0xfb, 0x78, 0xac, 0xf3, 0x56, 0x63, 0xec, 0x8d, 0xf8, 0xcf, 0x65, 0xd3, 0x36, 0xf0, 0xb3, 0xe8,
0x52, 0xea, 0x02, 0x54, 0x3f, 0x1e, 0x4f, 0xfc, 0x63, 0xf5, 0xaf, 0x15, 0x68, 0xdd, 0xb3, 0xa6,
0xde, 0xbe, 0x86, 0x9f, 0x4e, 0xb1, 0xe7, 0xa3, 0x77, 0xa0, 0xf2, 0x44, 0xf7, 0x70, 0x57, 0xb9,
0xaa, 0xac, 0x36, 0xd7, 0x5e, 0xbd, 0x15, 0xc3, 0x89, 0x63, 0xb3, 0xe5, 0x8d, 0xd6, 0x75, 0x0f,
0x6b, 0x14, 0x12, 0x21, 0xa8, 0x18, 0x4f, 0xfa, 0x9b, 0xdd, 0xd2, 0x55, 0x65, 0xb5, 0xac, 0xd1,
0xdf, 0xe8, 0x32, 0x80, 0x87, 0x47, 0x63, 0x6c, 0xfb, 0xfd, 0x4d, 0xaf, 0x5b, 0xbe, 0x5a, 0x5e,
0x2d, 0x6b, 0x91, 0x1e, 0xa4, 0x42, 0x6b, 0xe8, 0x58, 0x16, 0x1e, 0xfa, 0xa6, 0x63, 0xf7, 0x37,
0xbb, 0x15, 0x3a, 0x36, 0xd6, 0x87, 0x7a, 0x50, 0x37, 0xbd, 0xfe, 0x78, 0xe2, 0xb8, 0x7e, 0xb7,
0x7a, 0x55, 0x59, 0xad, 0x6b, 0xa2, 0xad, 0xfe, 0xa7, 0x02, 0x8b, 0x1c, 0x6d, 0x6f, 0xe2, 0xd8,
0x1e, 0x46, 0x77, 0xa0, 0xe6, 0xf9, 0xba, 0x3f, 0xf5, 0x38, 0xe6, 0x17, 0xa5, 0x98, 0xef, 0x50,
0x10, 0x8d, 0x83, 0x4a, 0x51, 0x4f, 0xa2, 0x56, 0x96, 0xa0, 0x16, 0xdf, 0x5e, 0x25, 0xb5, 0xbd,
0x55, 0x58, 0xda, 0x23, 0xd8, 0xed, 0x84, 0x40, 0x55, 0x0a, 0x94, 0xec, 0x26, 0x33, 0xf9, 0xe6,
0x18, 0x7f, 0x77, 0x6f, 0x07, 0xeb, 0x56, 0xb7, 0x46, 0xd7, 0x8a, 0xf4, 0xa8, 0xff, 0xac, 0x40,
0x47, 0x80, 0x07, 0x67, 0x74, 0x0e, 0xaa, 0x43, 0x67, 0x6a, 0xfb, 0x74, 0xab, 0x8b, 0x1a, 0x6b,
0xa0, 0xaf, 0x41, 0x6b, 0xb8, 0xaf, 0xdb, 0x36, 0xb6, 0x06, 0xb6, 0x3e, 0xc6, 0x74, 0x53, 0x0d,
0xad, 0xc9, 0xfb, 0x1e, 0xea, 0x63, 0x5c, 0x68, 0x6f, 0x57, 0xa1, 0x39, 0xd1, 0x5d, 0xdf, 0x8c,
0x9d, 0x4c, 0xb4, 0x2b, 0xef, 0x60, 0xc8, 0x0a, 0x26, 0xfd, 0xb5, 0xab, 0x7b, 0x07, 0xfd, 0x4d,
0xbe, 0xa3, 0x58, 0x9f, 0xfa, 0x23, 0x05, 0x56, 0xee, 0x7a, 0x9e, 0x39, 0xb2, 0x53, 0x3b, 0x5b,
0x81, 0x9a, 0xed, 0x18, 0xb8, 0xbf, 0x49, 0xb7, 0x56, 0xd6, 0x78, 0x0b, 0x5d, 0x84, 0xc6, 0x04,
0x63, 0x77, 0xe0, 0x3a, 0x56, 0xb0, 0xb1, 0x3a, 0xe9, 0xd0, 0x1c, 0x0b, 0xa3, 0xef, 0xc1, 0xb2,
0x97, 0x98, 0x88, 0xf1, 0x5c, 0x73, 0xed, 0xda, 0xad, 0x94, 0x4c, 0xdd, 0x4a, 0x2e, 0xaa, 0xa5,
0x47, 0xab, 0x5f, 0x94, 0xe0, 0xac, 0x80, 0x63, 0xb8, 0x92, 0xdf, 0x84, 0xf2, 0x1e, 0x1e, 0x09,
0xf4, 0x58, 0xa3, 0x08, 0xe5, 0xc5, 0x91, 0x95, 0xa3, 0x47, 0x56, 0x44, 0x0c, 0x12, 0xe7, 0x51,
0x4d, 0x9f, 0xc7, 0x15, 0x68, 0xe2, 0x67, 0x13, 0xd3, 0xc5, 0x03, 0xc2, 0x38, 0x94, 0xe4, 0x15,
0x0d, 0x58, 0xd7, 0xae, 0x39, 0x8e, 0xca, 0xc6, 0x42, 0x61, 0xd9, 0x50, 0xff, 0x54, 0x81, 0xf3,
0xa9, 0x53, 0xe2, 0xc2, 0xa6, 0x41, 0x87, 0xee, 0x3c, 0xa4, 0x0c, 0x11, 0x3b, 0x42, 0xf0, 0xd7,
0xf2, 0x08, 0x1e, 0x82, 0x6b, 0xa9, 0xf1, 0x11, 0x24, 0x4b, 0xc5, 0x91, 0x3c, 0x80, 0xf3, 0xf7,
0xb1, 0xcf, 0x17, 0x20, 0xdf, 0xb0, 0x77, 0x7a, 0x45, 0x16, 0x97, 0xea, 0x52, 0x52, 0xaa, 0xd5,
0x3f, 0x2b, 0x09, 0x59, 0xa4, 0x4b, 0xf5, 0xed, 0x3d, 0x07, 0xbd, 0x0a, 0x0d, 0x01, 0xc2, 0xb9,
0x22, 0xec, 0x40, 0xdf, 0x80, 0x2a, 0xc1, 0x94, 0xb1, 0x44, 0x7b, 0xed, 0x6b, 0xf2, 0x3d, 0x45,
0xe6, 0xd4, 0x18, 0x3c, 0xda, 0x84, 0xb6, 0xe7, 0xeb, 0xae, 0x3f, 0x98, 0x38, 0x1e, 0x3d, 0x67,
0xca, 0x38, 0xcd, 0xb5, 0x4b, 0xf1, 0x19, 0x88, 0x92, 0xdf, 0xf2, 0x46, 0xdb, 0x1c, 0x48, 0x5b,
0xa4, 0x83, 0x82, 0x26, 0xfa, 0x0e, 0xb4, 0xb0, 0x6d, 0x84, 0x73, 0x54, 0x8a, 0xcc, 0xd1, 0xc4,
0xb6, 0x21, 0x66, 0x08, 0x4f, 0xa5, 0x5a, 0xfc, 0x54, 0x7e, 0x47, 0x81, 0x6e, 0xfa, 0x58, 0xe6,
0x51, 0xd4, 0x1f, 0xb0, 0x41, 0x98, 0x1d, 0x4b, 0xae, 0x5c, 0x8b, 0xa3, 0xd1, 0xf8, 0x10, 0xf5,
0x8f, 0x14, 0x78, 0x25, 0x44, 0x87, 0x7e, 0x7a, 0x51, 0x3c, 0x82, 0x6e, 0x42, 0xc7, 0xb4, 0x87,
0xd6, 0xd4, 0xc0, 0x8f, 0xec, 0x4f, 0xb0, 0x6e, 0xf9, 0xfb, 0xc7, 0xf4, 0xe4, 0xea, 0x5a, 0xaa,
0x5f, 0xfd, 0xf7, 0x12, 0xac, 0x24, 0xf1, 0x9a, 0x87, 0x48, 0x3f, 0x03, 0x55, 0xd3, 0xde, 0x73,
0x02, 0x1a, 0x5d, 0xce, 0x11, 0x45, 0xb2, 0x16, 0x03, 0x46, 0x0e, 0xa0, 0x40, 0x79, 0x0d, 0xf7,
0xf1, 0xf0, 0x60, 0xe2, 0x98, 0x54, 0x4d, 0x91, 0x29, 0xbe, 0x23, 0x99, 0x42, 0x8e, 0xf1, 0xad,
0x0d, 0x36, 0xc7, 0x86, 0x98, 0xe2, 0x63, 0xdb, 0x77, 0x8f, 0xb5, 0xe5, 0x61, 0xb2, 0xbf, 0x37,
0x84, 0x15, 0x39, 0x30, 0xea, 0x40, 0xf9, 0x00, 0x1f, 0xd3, 0x2d, 0x37, 0x34, 0xf2, 0x13, 0xdd,
0x81, 0xea, 0xa1, 0x6e, 0x4d, 0x31, 0xd7, 0x09, 0x33, 0x38, 0x97, 0xc1, 0xbe, 0x5f, 0x7a, 0x4f,
0x51, 0xc7, 0x70, 0xf1, 0x3e, 0xf6, 0xfb, 0xb6, 0x87, 0x5d, 0x7f, 0xdd, 0xb4, 0x2d, 0x67, 0xb4,
0xad, 0xfb, 0xfb, 0x73, 0x28, 0x87, 0x98, 0x9c, 0x97, 0x12, 0x72, 0xae, 0xfe, 0x58, 0x81, 0x57,
0xe5, 0xeb, 0xf1, 0x03, 0xed, 0x41, 0x7d, 0xcf, 0xc4, 0x96, 0x41, 0xb8, 0x46, 0xa1, 0x5c, 0x23,
0xda, 0x44, 0x49, 0x4c, 0x08, 0x30, 0x3f, 0xb7, 0x84, 0x92, 0x10, 0x3e, 0xdf, 0x8e, 0xef, 0x9a,
0xf6, 0xe8, 0x81, 0xe9, 0xf9, 0x1a, 0x83, 0x8f, 0x70, 0x49, 0xb9, 0xb8, 0x70, 0xfe, 0x96, 0x02,
0x97, 0xef, 0x63, 0x7f, 0x43, 0xd8, 0x18, 0xf2, 0xdd, 0xf4, 0x7c, 0x73, 0xe8, 0x3d, 0x5f, 0x1f,
0xb0, 0x80, 0xb3, 0xa1, 0xfe, 0x9e, 0x02, 0x57, 0x32, 0x91, 0xe1, 0xa4, 0xe3, 0x3a, 0x34, 0xb0,
0x30, 0x72, 0x1d, 0xfa, 0x73, 0xf8, 0xf8, 0x31, 0x39, 0xfc, 0x6d, 0xdd, 0x74, 0x99, 0x0e, 0x3d,
0xa5, 0x45, 0xf9, 0x89, 0x02, 0x97, 0xee, 0x63, 0x7f, 0x3b, 0xb0, 0xaf, 0x2f, 0x91, 0x3a, 0x04,
0x26, 0x62, 0xe7, 0x03, 0x47, 0x33, 0xd6, 0xa7, 0xfe, 0x2e, 0x3b, 0x4e, 0x29, 0xbe, 0x2f, 0x85,
0x80, 0x97, 0xa9, 0x24, 0x44, 0x54, 0x04, 0x17, 0x76, 0x4e, 0x3e, 0xf5, 0x87, 0x55, 0x68, 0x3d,
0xe6, 0x5a, 0x81, 0x5a, 0xd0, 0x24, 0x25, 0x14, 0xb9, 0x13, 0x14, 0xf1, 0xa6, 0x64, 0x0e, 0xd6,
0x3a, 0x2c, 0x7a, 0x18, 0x1f, 0x9c, 0xd0, 0x5e, 0xb6, 0xc8, 0x18, 0x61, 0xec, 0x1e, 0xc0, 0xf2,
0xd4, 0xa6, 0x1e, 0x3a, 0x36, 0xf8, 0x06, 0x18, 0xd1, 0x67, 0x2b, 0xd3, 0xf4, 0x40, 0xf4, 0x09,
0x0f, 0x02, 0x22, 0x73, 0x55, 0x0b, 0xcd, 0x95, 0x1c, 0x86, 0xfa, 0xd0, 0x31, 0x5c, 0x67, 0x32,
0xc1, 0xc6, 0xc0, 0x0b, 0xa6, 0xaa, 0x15, 0x9b, 0x8a, 0x8f, 0x13, 0x53, 0xbd, 0x03, 0x67, 0x93,
0x98, 0xf6, 0x0d, 0xe2, 0x17, 0x12, 0xce, 0x92, 0x7d, 0x42, 0x6f, 0xc1, 0x72, 0x1a, 0xbe, 0x4e,
0xe1, 0xd3, 0x1f, 0xd0, 0xdb, 0x80, 0x12, 0xa8, 0x12, 0xf0, 0x06, 0x03, 0x8f, 0x23, 0xc3, 0xc1,
0x69, 0x70, 0x1a, 0x07, 0x07, 0x06, 0xce, 0xbf, 0x44, 0xc0, 0xfb, 0xc4, 0xba, 0xc6, 0xc0, 0xbd,
0x6e, 0xb3, 0x18, 0x21, 0xe2, 0x93, 0x79, 0xea, 0x6f, 0x2a, 0xb0, 0xf2, 0xa9, 0xee, 0x0f, 0xf7,
0x37, 0xc7, 0x9c, 0x41, 0xe7, 0x10, 0xf0, 0x0f, 0xa1, 0x71, 0xc8, 0x99, 0x31, 0xd0, 0xe2, 0x57,
0x24, 0x08, 0x45, 0xd9, 0x5e, 0x0b, 0x47, 0x90, 0x80, 0xe8, 0xdc, 0xbd, 0x48, 0x60, 0xf8, 0x12,
0x54, 0xcd, 0x8c, 0x88, 0x56, 0x7d, 0x06, 0xc0, 0x91, 0xdb, 0xf2, 0x46, 0xa7, 0xc0, 0xeb, 0x3d,
0x58, 0xe0, 0xb3, 0x71, 0x5d, 0x32, 0xeb, 0xc0, 0x02, 0x70, 0xf5, 0x47, 0x35, 0x68, 0x46, 0x3e,
0xa0, 0x36, 0x94, 0x84, 0x92, 0x28, 0x49, 0x76, 0x57, 0x9a, 0x1d, 0x43, 0x95, 0xd3, 0x31, 0xd4,
0x0d, 0x68, 0x9b, 0xd4, 0x78, 0x0f, 0xf8, 0xa9, 0x50, 0x5f, 0xb9, 0xa1, 0x2d, 0xb2, 0x5e, 0xce,
0x22, 0xe8, 0x32, 0x34, 0xed, 0xe9, 0x78, 0xe0, 0xec, 0x0d, 0x5c, 0xe7, 0xc8, 0xe3, 0xc1, 0x58,
0xc3, 0x9e, 0x8e, 0xbf, 0xbb, 0xa7, 0x39, 0x47, 0x5e, 0xe8, 0xef, 0xd7, 0x4e, 0xe8, 0xef, 0x5f,
0x86, 0xe6, 0x58, 0x7f, 0x46, 0x66, 0x1d, 0xd8, 0xd3, 0x31, 0x8d, 0xd3, 0xca, 0x5a, 0x63, 0xac,
0x3f, 0xd3, 0x9c, 0xa3, 0x87, 0xd3, 0x31, 0x5a, 0x85, 0x8e, 0xa5, 0x7b, 0xfe, 0x20, 0x1a, 0xe8,
0xd5, 0x69, 0xa0, 0xd7, 0x26, 0xfd, 0x1f, 0x87, 0xc1, 0x5e, 0x3a, 0x72, 0x68, 0x9c, 0x2e, 0x72,
0x30, 0xc6, 0x56, 0x38, 0x07, 0x14, 0x8a, 0x1c, 0x8c, 0xb1, 0x25, 0x66, 0x78, 0x0f, 0x16, 0x9e,
0x50, 0x47, 0x28, 0x4f, 0x44, 0xef, 0x11, 0x1f, 0x88, 0xf9, 0x4b, 0x5a, 0x00, 0x8e, 0xbe, 0x05,
0x0d, 0x6a, 0x7f, 0xe8, 0xd8, 0x56, 0xa1, 0xb1, 0xe1, 0x00, 0x32, 0xda, 0xc0, 0x96, 0xaf, 0xd3,
0xd1, 0x8b, 0xc5, 0x46, 0x8b, 0x01, 0x44, 0x3f, 0x0e, 0x5d, 0xac, 0xfb, 0xd8, 0x58, 0x3f, 0xde,
0x70, 0xc6, 0x13, 0x9d, 0xb2, 0x50, 0xb7, 0x4d, 0x5d, 0x78, 0xd9, 0x27, 0xf4, 0x1a, 0xb4, 0x87,
0xa2, 0x75, 0xcf, 0x75, 0xc6, 0xdd, 0x25, 0x2a, 0x3d, 0x89, 0x5e, 0x74, 0x09, 0x20, 0xd0, 0x8c,
0xba, 0xdf, 0xed, 0xd0, 0xb3, 0x6b, 0xf0, 0x9e, 0xbb, 0x34, 0x7b, 0x63, 0x7a, 0x03, 0x96, 0x27,
0x31, 0xed, 0x51, 0x77, 0x99, 0xae, 0xd8, 0x0c, 0x12, 0x2b, 0xa6, 0x3d, 0x42, 0xe7, 0x61, 0xc1,
0xf4, 0x06, 0x7b, 0xfa, 0x01, 0xee, 0x22, 0xfa, 0xb5, 0x66, 0x7a, 0xf7, 0xf4, 0x03, 0xac, 0x7e,
0x0e, 0xe7, 0x42, 0x9e, 0x8a, 0x1c, 0x62, 0x9a, 0x15, 0x94, 0x53, 0xb0, 0x42, 0xbe, 0xe7, 0xfb,
0xdf, 0x15, 0x58, 0xd9, 0xd1, 0x0f, 0xf1, 0x8b, 0x77, 0xb2, 0x0b, 0xe9, 0xb1, 0x07, 0xb0, 0x4c,
0xfd, 0xea, 0xb5, 0x08, 0x3e, 0x39, 0x26, 0x3c, 0xca, 0x05, 0xe9, 0x81, 0xe8, 0xdb, 0xc4, 0xed,
0xc0, 0xc3, 0x83, 0x6d, 0x12, 0xa3, 0x04, 0xe6, 0xfb, 0x92, 0x64, 0x9e, 0x0d, 0x01, 0xa5, 0x45,
0x47, 0xa0, 0x6d, 0x58, 0x8a, 0x9f, 0x40, 0x60, 0xb8, 0x5f, 0xcf, 0x0d, 0x60, 0x43, 0xea, 0x6b,
0xed, 0xd8, 0x61, 0x78, 0xa8, 0x0b, 0x0b, 0xdc, 0xea, 0x52, 0x25, 0x51, 0xd7, 0x82, 0x26, 0xda,
0x86, 0xb3, 0x6c, 0x07, 0x3b, 0x5c, 0x16, 0xd8, 0xe6, 0xeb, 0x85, 0x36, 0x2f, 0x1b, 0x1a, 0x17,
0xa5, 0xc6, 0x49, 0x45, 0xa9, 0x0b, 0x0b, 0x9c, 0xbd, 0xa9, 0xf6, 0xa8, 0x6b, 0x41, 0x93, 0x1c,
0x73, 0xc8, 0xe8, 0x4d, 0xfa, 0x2d, 0xec, 0x20, 0xe3, 0x02, 0x1d, 0xdc, 0xa2, 0x3a, 0x38, 0x68,
0xaa, 0xbf, 0xae, 0x00, 0x84, 0x94, 0x9e, 0x91, 0x7a, 0xf9, 0x26, 0xd4, 0x05, 0xdb, 0x17, 0x8a,
0x1e, 0x05, 0x78, 0x52, 0xcb, 0x97, 0x13, 0x5a, 0x5e, 0xfd, 0x47, 0x05, 0x5a, 0x9b, 0x64, 0x9f,
0x0f, 0x9c, 0x11, 0xb5, 0x49, 0x37, 0xa0, 0xed, 0xe2, 0xa1, 0xe3, 0x1a, 0x03, 0x6c, 0xfb, 0xae,
0x89, 0x59, 0xd8, 0x5e, 0xd1, 0x16, 0x59, 0xef, 0xc7, 0xac, 0x93, 0x80, 0x11, 0xc5, 0xed, 0xf9,
0xfa, 0x78, 0x32, 0xd8, 0x23, 0xaa, 0xa2, 0xc4, 0xc0, 0x44, 0x2f, 0xd5, 0x14, 0x5f, 0x83, 0x56,
0x08, 0xe6, 0x3b, 0x74, 0xfd, 0x8a, 0xd6, 0x14, 0x7d, 0xbb, 0x0e, 0xba, 0x0e, 0x6d, 0x4a, 0xe8,
0x81, 0xe5, 0x8c, 0x06, 0x24, 0x18, 0xe4, 0xe6, 0xaa, 0x65, 0x70, 0xb4, 0xc8, 0x01, 0xc6, 0xa1,
0x3c, 0xf3, 0x73, 0xcc, 0x0d, 0x96, 0x80, 0xda, 0x31, 0x3f, 0xc7, 0xea, 0xaf, 0x29, 0xb0, 0xc8,
0xed, 0xdb, 0x8e, 0x48, 0x8b, 0xd3, 0x3c, 0x26, 0x0b, 0xc4, 0xe9, 0x6f, 0xf4, 0x7e, 0x3c, 0x93,
0x75, 0x5d, 0x2a, 0x04, 0x74, 0x12, 0xea, 0x55, 0xc5, 0x8c, 0x5b, 0x91, 0x48, 0xf0, 0x0b, 0x42,
0x53, 0xdd, 0xd7, 0x1f, 0x3a, 0x06, 0x4b, 0xac, 0x75, 0x61, 0x41, 0x37, 0x0c, 0x17, 0x7b, 0x1e,
0xc7, 0x23, 0x68, 0x92, 0x2f, 0x87, 0xd8, 0xf5, 0x82, 0x83, 0x2d, 0x6b, 0x41, 0x13, 0x7d, 0x0b,
0xea, 0xc2, 0x0d, 0x63, 0x19, 0x8c, 0xab, 0xd9, 0x78, 0xf2, 0xb8, 0x45, 0x8c, 0x50, 0xff, 0xa6,
0x04, 0x6d, 0x2e, 0x83, 0xeb, 0xdc, 0x14, 0xe5, 0xb3, 0xd8, 0x3a, 0xb4, 0xf6, 0x42, 0xde, 0xcf,
0xcb, 0xbb, 0x44, 0x45, 0x24, 0x36, 0x66, 0x16, 0xaf, 0xc5, 0x8d, 0x61, 0x65, 0x2e, 0x63, 0x58,
0x3d, 0xa9, 0x04, 0xa7, 0x9d, 0xa2, 0x9a, 0xc4, 0x29, 0x52, 0x7f, 0x01, 0x9a, 0x91, 0x09, 0xa8,
0x86, 0x62, 0xa9, 0x0d, 0x4e, 0xb1, 0xa0, 0x89, 0xee, 0x84, 0x2e, 0x01, 0x23, 0xd5, 0x05, 0x09,
0x2e, 0x09, 0x6f, 0x40, 0xfd, 0xa9, 0x02, 0x35, 0x3e, 0xf3, 0x15, 0x68, 0x72, 0xf9, 0xa2, 0x4e,
0x12, 0x9b, 0x1d, 0x78, 0x17, 0xf1, 0x92, 0x9e, 0x9f, 0x80, 0x5d, 0x80, 0x7a, 0x42, 0xb4, 0x16,
0xb8, 0x5a, 0x0c, 0x3e, 0x45, 0xe4, 0x89, 0x7c, 0x22, 0xa2, 0x84, 0xce, 0x41, 0xd5, 0x72, 0x46,
0xa2, 0xec, 0xc1, 0x1a, 0xea, 0x97, 0x0a, 0xcd, 0x52, 0x6b, 0x78, 0xe8, 0x1c, 0x62, 0xf7, 0x78,
0xfe, 0x44, 0xdf, 0x07, 0x11, 0x36, 0x2f, 0x18, 0x6d, 0x88, 0x01, 0xe8, 0x83, 0xf0, 0x10, 0xca,
0xb2, 0x7c, 0x40, 0xd4, 0x14, 0x71, 0x26, 0x0d, 0x0f, 0xe3, 0xf7, 0x15, 0x9a, 0xb2, 0x8c, 0x6f,
0xe5, 0xb4, 0xd6, 0xfe, 0xb9, 0x78, 0xee, 0xea, 0x3f, 0x28, 0x70, 0x21, 0x83, 0xba, 0x8f, 0xd7,
0x5e, 0x02, 0x7d, 0xdf, 0x87, 0xba, 0x88, 0x4d, 0xcb, 0x85, 0x62, 0x53, 0x01, 0xaf, 0xfe, 0x21,
0x4b, 0x9c, 0x4b, 0xc8, 0xfb, 0x78, 0xed, 0x05, 0x11, 0x38, 0x99, 0x63, 0x2a, 0x4b, 0x72, 0x4c,
0xff, 0xa4, 0x40, 0x2f, 0xcc, 0xe9, 0x78, 0xeb, 0xc7, 0xf3, 0x56, 0x5a, 0x9e, 0x4f, 0xcc, 0xf6,
0x4d, 0x51, 0x14, 0x20, 0x7a, 0xb1, 0x50, 0xb4, 0x15, 0x94, 0x04, 0x6c, 0x9a, 0x1e, 0x4e, 0x6f,
0x68, 0x1e, 0xa9, 0xec, 0x45, 0x0e, 0x9e, 0x15, 0x06, 0xc2, 0x83, 0xfd, 0x29, 0x63, 0xd2, 0x7b,
0xf1, 0xc4, 0xce, 0xcb, 0x26, 0x60, 0xb4, 0x58, 0xb1, 0xcf, 0x8b, 0x15, 0x95, 0x44, 0xb1, 0x82,
0xf7, 0xab, 0x63, 0xca, 0x02, 0xa9, 0x0d, 0xbc, 0x28, 0x82, 0xfd, 0x86, 0x02, 0x5d, 0xbe, 0x0a,
0x5d, 0x93, 0x04, 0x5c, 0x16, 0xf6, 0xb1, 0xf1, 0x55, 0xa7, 0x1f, 0xfe, 0xb8, 0x04, 0x9d, 0xa8,
0x63, 0x43, 0x7d, 0x93, 0xaf, 0x43, 0x95, 0x66, 0x6f, 0x38, 0x06, 0x33, 0xb5, 0x03, 0x83, 0x26,
0x96, 0x91, 0x7a, 0xf3, 0xbb, 0x5e, 0xe0, 0xb8, 0xf0, 0x66, 0xe8, 0x5d, 0x95, 0x4f, 0xee, 0x5d,
0xbd, 0x0a, 0x0d, 0x62, 0xb9, 0x9c, 0x29, 0x99, 0x97, 0x55, 0x90, 0xc3, 0x0e, 0xf4, 0x21, 0xd4,
0xd8, 0xbd, 0x10, 0x5e, 0xc0, 0xbb, 0x11, 0x9f, 0x9a, 0xdf, 0x19, 0x89, 0x24, 0xe0, 0x69, 0x87,
0xc6, 0x07, 0x91, 0x33, 0x9a, 0xb8, 0xce, 0x88, 0xba, 0x61, 0xc4, 0xa8, 0x55, 0x35, 0xd1, 0x56,
0x7f, 0x16, 0x56, 0xc2, 0x38, 0x98, 0xa1, 0x74, 0x5a, 0x86, 0x56, 0xff, 0x55, 0x81, 0xb3, 0x3b,
0xc7, 0xf6, 0x30, 0x29, 0x1a, 0x2b, 0x50, 0x9b, 0x58, 0x7a, 0x98, 0x16, 0xe6, 0x2d, 0x5a, 0x72,
0x67, 0x6b, 0x63, 0x83, 0x98, 0x70, 0x46, 0xcf, 0xa6, 0xe8, 0xdb, 0x75, 0x66, 0x7a, 0x56, 0x37,
0x44, 0xe0, 0x8e, 0x0d, 0xe6, 0x2c, 0xb0, 0xb4, 0xd7, 0xa2, 0xe8, 0xa5, 0xce, 0xc2, 0x87, 0x00,
0xd4, 0x9f, 0x1a, 0x9c, 0xc4, 0x87, 0xa2, 0x23, 0x1e, 0x10, 0x8b, 0xf9, 0x57, 0x25, 0xe8, 0x46,
0xa8, 0xf4, 0x55, 0xbb, 0x97, 0x19, 0x41, 0x61, 0xf9, 0x39, 0x05, 0x85, 0x95, 0xf9, 0x5d, 0xca,
0xaa, 0xcc, 0xa5, 0xfc, 0x95, 0x32, 0xb4, 0x43, 0xaa, 0x6d, 0x5b, 0xba, 0x9d, 0xc9, 0x09, 0x3b,
0xd0, 0xf6, 0x62, 0x54, 0xe5, 0x74, 0x7a, 0x53, 0x26, 0x43, 0x19, 0x07, 0xa1, 0x25, 0xa6, 0x40,
0x97, 0xe8, 0xa1, 0xbb, 0x3e, 0x4b, 0xb4, 0x31, 0xff, 0xb0, 0xc1, 0x84, 0xd5, 0x1c, 0x63, 0xf4,
0x16, 0x20, 0x2e, 0x61, 0x03, 0xd3, 0x1e, 0x78, 0x78, 0xe8, 0xd8, 0x06, 0x93, 0xbd, 0xaa, 0xd6,
0xe1, 0x5f, 0xfa, 0xf6, 0x0e, 0xeb, 0x47, 0x5f, 0x87, 0x8a, 0x7f, 0x3c, 0x61, 0xce, 0x62, 0x5b,
0xea, 0x6e, 0x85, 0x78, 0xed, 0x1e, 0x4f, 0xb0, 0x46, 0xc1, 0x83, 0xab, 0x41, 0xbe, 0xab, 0x1f,
0x72, 0xcf, 0xbb, 0xa2, 0x45, 0x7a, 0xa2, 0x71, 0xf2, 0x42, 0x2c, 0x4e, 0x66, 0x9c, 0x1d, 0x08,
0xf4, 0xc0, 0xf7, 0x2d, 0x9a, 0x2a, 0xa4, 0x9c, 0x1d, 0xf4, 0xee, 0xfa, 0x16, 0xd9, 0xa4, 0xef,
0xf8, 0xba, 0xc5, 0xe4, 0xa3, 0xc1, 0x35, 0x07, 0xe9, 0xa1, 0x51, 0xee, 0xbf, 0x10, 0xcd, 0x27,
0x10, 0xd3, 0xb0, 0x37, 0xb5, 0xb2, 0xe5, 0x31, 0x3f, 0x73, 0x33, 0x4b, 0x14, 0xbf, 0x0d, 0x4d,
0xce, 0x15, 0x27, 0xe0, 0x2a, 0x60, 0x43, 0x1e, 0xe4, 0xb0, 0x79, 0xf5, 0x39, 0xb1, 0x79, 0xed,
0x14, 0xb9, 0x0f, 0xf9, 0xd9, 0xa8, 0x3f, 0x56, 0xe0, 0x95, 0x94, 0xd6, 0xcc, 0x25, 0x6d, 0x7e,
0xe4, 0xcd, 0xb5, 0x69, 0x72, 0x4a, 0x6e, 0x1b, 0x3e, 0x80, 0x9a, 0x4b, 0x67, 0xe7, 0xe5, 0xb0,
0x6b, 0xb9, 0xcc, 0xc7, 0x10, 0xd1, 0xf8, 0x10, 0xf5, 0x0f, 0x14, 0x38, 0x9f, 0x46, 0x75, 0x0e,
0x83, 0xbf, 0x0e, 0x0b, 0x6c, 0xea, 0x40, 0x46, 0x57, 0xf3, 0x65, 0x34, 0x24, 0x8e, 0x16, 0x0c,
0x54, 0x77, 0x60, 0x25, 0xf0, 0x0b, 0x42, 0xd2, 0x6f, 0x61, 0x5f, 0xcf, 0x89, 0x3b, 0xaf, 0x40,
0x93, 0x05, 0x30, 0x2c, 0x9e, 0x63, 0xd5, 0x43, 0x78, 0x22, 0x12, 0x7d, 0xea, 0x7f, 0x29, 0x70,
0x8e, 0x1a, 0xd6, 0x64, 0x29, 0xa8, 0x48, 0x6d, 0x52, 0x15, 0xb7, 0xbf, 0x1e, 0xea, 0x63, 0x7e,
0x43, 0xa5, 0xa1, 0xc5, 0xfa, 0x50, 0x3f, 0x9d, 0x07, 0x94, 0xe6, 0x27, 0xc2, 0x62, 0xec, 0xa6,
0xee, 0xeb, 0xb4, 0x16, 0x9b, 0x4c, 0x00, 0x86, 0x06, 0xbd, 0x72, 0x0a, 0x83, 0xae, 0x3e, 0x80,
0x57, 0x12, 0x3b, 0x9d, 0xe3, 0x44, 0xd5, 0x3f, 0x57, 0xc8, 0x71, 0xc4, 0x6e, 0xfa, 0x9c, 0xde,
0xa9, 0xbd, 0x24, 0x6a, 0x50, 0x03, 0xd3, 0x48, 0x2a, 0x11, 0x03, 0x7d, 0x04, 0x0d, 0x1b, 0x1f,
0x0d, 0xa2, 0x7e, 0x52, 0x01, 0x8f, 0xbf, 0x6e, 0xe3, 0x23, 0xfa, 0x4b, 0x7d, 0x08, 0xe7, 0x53,
0xa8, 0xce, 0xb3, 0xf7, 0xbf, 0x53, 0xe0, 0xc2, 0xa6, 0xeb, 0x4c, 0x1e, 0x9b, 0xae, 0x3f, 0xd5,
0xad, 0x78, 0x99, 0xfb, 0x14, 0xdb, 0x2f, 0x70, 0x8b, 0xf0, 0x93, 0x54, 0x6c, 0xf9, 0x96, 0x44,
0x82, 0xd2, 0x48, 0xf1, 0x4d, 0x47, 0xfc, 0xeb, 0xff, 0x28, 0xcb, 0x90, 0xe7, 0x70, 0x33, 0xfc,
0x92, 0x22, 0xc1, 0x87, 0x34, 0x0f, 0x5f, 0x3e, 0x6d, 0x1e, 0x3e, 0x43, 0xbd, 0x57, 0x9e, 0x93,
0x7a, 0x3f, 0x71, 0x62, 0x6c, 0x03, 0xe2, 0x35, 0x12, 0x6a, 0x9d, 0x4f, 0x5a, 0x57, 0xf9, 0x10,
0x20, 0x2c, 0x15, 0xf0, 0x9b, 0x99, 0x33, 0x66, 0x88, 0x0c, 0x20, 0x67, 0x24, 0x0c, 0x28, 0xb7,
0xef, 0x91, 0x14, 0xf5, 0xf7, 0xa0, 0x27, 0xe3, 0xcd, 0x79, 0xf8, 0xfd, 0xdf, 0x4a, 0x00, 0x7d,
0x71, 0x8f, 0xf7, 0x74, 0x16, 0xe0, 0x1a, 0x44, 0x7c, 0x90, 0x50, 0xca, 0xa3, 0xbc, 0x63, 0x10,
0x41, 0x10, 0x51, 0x2a, 0x81, 0x49, 0x45, 0xae, 0x06, 0x9d, 0x27, 0x22, 0x2b, 0x8c, 0x15, 0x92,
0x4a, 0xf7, 0x22, 0x34, 0x5c, 0xe7, 0x68, 0x40, 0x84, 0xcb, 0x08, 0x2e, 0x2a, 0xbb, 0xce, 0x11,
0x11, 0x39, 0x03, 0x9d, 0x87, 0x05, 0x5f, 0xf7, 0x0e, 0xc8, 0xfc, 0x2c, 0x59, 0x57, 0x23, 0xcd,
0xbe, 0x81, 0xce, 0x41, 0x75, 0xcf, 0xb4, 0x30, 0xbb, 0x13, 0xd1, 0xd0, 0x58, 0x03, 0x7d, 0x23,
0xb8, 0x5b, 0x57, 0x2f, 0x7c, 0x87, 0x86, 0x5d, 0xaf, 0xbb, 0x06, 0x8b, 0x84, 0x93, 0x08, 0x12,
0x4c, 0xac, 0x3b, 0x3c, 0x51, 0xcf, 0x3b, 0x09, 0xaa, 0xea, 0x97, 0x0a, 0x2c, 0x85, 0xa4, 0xa5,
0xba, 0x89, 0xa8, 0x3b, 0xaa, 0xea, 0x36, 0x1c, 0x83, 0x69, 0x91, 0x76, 0x86, 0xb1, 0x60, 0x03,
0x99, 0x42, 0x0b, 0x87, 0xe4, 0x45, 0xd7, 0x64, 0xf3, 0x84, 0x32, 0xa6, 0x11, 0xe4, 0x7b, 0x6a,
0xae, 0x73, 0xd4, 0x37, 0x04, 0xc9, 0xd8, 0x55, 0x65, 0x16, 0x4b, 0x12, 0x92, 0x6d, 0xd0, 0xdb,
0xca, 0xd7, 0x60, 0x11, 0xbb, 0xae, 0xe3, 0x0e, 0xc6, 0xd8, 0xf3, 0xf4, 0x11, 0xe6, 0xae, 0x7b,
0x8b, 0x76, 0x6e, 0xb1, 0x3e, 0xf5, 0xef, 0x2b, 0xd0, 0x0e, 0xb7, 0x12, 0x54, 0xec, 0x4d, 0x23,
0xa8, 0xd8, 0x9b, 0xe4, 0x7c, 0xc1, 0x65, 0x5a, 0x52, 0x70, 0xc0, 0x7a, 0xa9, 0xab, 0x68, 0x0d,
0xde, 0xdb, 0x37, 0x88, 0xc5, 0x26, 0x04, 0xb2, 0x1d, 0x03, 0x87, 0x1c, 0x00, 0x41, 0x17, 0x67,
0x80, 0x18, 0x23, 0x55, 0x0a, 0x30, 0x52, 0xb5, 0x00, 0x23, 0xd5, 0x24, 0x8c, 0xb4, 0x02, 0xb5,
0x27, 0xd3, 0xe1, 0x01, 0xf6, 0xb9, 0x33, 0xc7, 0x5b, 0x71, 0x06, 0xab, 0x27, 0x18, 0x4c, 0xf0,
0x51, 0x23, 0xca, 0x47, 0x17, 0xa1, 0xc1, 0x8a, 0xc8, 0x03, 0xdf, 0xa3, 0x65, 0xb1, 0xb2, 0x56,
0x67, 0x1d, 0xbb, 0x1e, 0x7a, 0x2f, 0xf0, 0xf4, 0x9a, 0x54, 0xa2, 0x54, 0x89, 0x42, 0x4a, 0x70,
0x49, 0xe0, 0xe7, 0xbd, 0x0e, 0x4b, 0x11, 0x72, 0x50, 0x3e, 0x63, 0xb5, 0xb3, 0x48, 0x20, 0x40,
0x2d, 0xc8, 0x0d, 0x68, 0x87, 0x24, 0xa1, 0x70, 0x8b, 0x2c, 0xfe, 0x12, 0xbd, 0x14, 0x4c, 0xb0,
0x7b, 0xfb, 0x84, 0xec, 0x7e, 0x01, 0xea, 0x3c, 0x70, 0xf2, 0xba, 0x4b, 0xf1, 0x1c, 0x47, 0x21,
0x49, 0xf8, 0x01, 0xa0, 0x70, 0x8b, 0xf3, 0x79, 0x9b, 0x09, 0x1e, 0x2a, 0x25, 0x79, 0x48, 0xfd,
0x0b, 0x05, 0x96, 0xa3, 0x8b, 0x9d, 0xd6, 0x70, 0x7f, 0x04, 0x4d, 0x56, 0xbd, 0x1c, 0x10, 0x15,
0x22, 0x2f, 0x36, 0x26, 0x0e, 0x4f, 0x83, 0xf0, 0x45, 0x04, 0x21, 0xcc, 0x91, 0xe3, 0x1e, 0x98,
0xf6, 0x68, 0x40, 0x30, 0x13, 0x39, 0x58, 0xde, 0xf9, 0x90, 0xf4, 0xa9, 0xbf, 0xad, 0xc0, 0xe5,
0x47, 0x13, 0x43, 0xf7, 0x71, 0xc4, 0x83, 0x99, 0xf7, 0x62, 0xa2, 0xb8, 0x19, 0x58, 0xca, 0x39,
0xe6, 0xc8, 0x7a, 0x1e, 0xbf, 0x19, 0x48, 0xfc, 0x3e, 0x8e, 0x4d, 0xea, 0x2a, 0xef, 0xe9, 0xb1,
0xe9, 0x41, 0xfd, 0x90, 0x4f, 0x17, 0xbc, 0xf1, 0x08, 0xda, 0xb1, 0x6a, 0x6e, 0xf9, 0x44, 0xd5,
0x5c, 0x75, 0x0b, 0x2e, 0x68, 0xd8, 0xc3, 0xb6, 0x11, 0xdb, 0xc8, 0xa9, 0x33, 0x55, 0x13, 0xe8,
0xc9, 0xa6, 0x9b, 0x87, 0x53, 0x99, 0xe3, 0x3b, 0x70, 0xc9, 0xb4, 0x3e, 0x57, 0xd6, 0xc4, 0xdf,
0xa2, 0xeb, 0xf8, 0xea, 0x5f, 0x96, 0xe0, 0xfc, 0x5d, 0xc3, 0xe0, 0x7a, 0x9e, 0xbb, 0x72, 0x2f,
0xca, 0xcb, 0x4e, 0x7a, 0xa1, 0xe5, 0xb4, 0x17, 0xfa, 0xbc, 0x74, 0x2f, 0xb7, 0x42, 0xf6, 0x74,
0x1c, 0x98, 0x60, 0x97, 0x5d, 0x76, 0xfa, 0x80, 0xd7, 0x3c, 0x07, 0x96, 0x33, 0xa2, 0x66, 0x78,
0xb6, 0x73, 0x56, 0x0f, 0x32, 0x6e, 0xea, 0x04, 0xba, 0x69, 0x62, 0xcd, 0xa9, 0x47, 0x02, 0x8a,
0x4c, 0x1c, 0x96, 0xb9, 0x6d, 0x11, 0x4f, 0x8c, 0x76, 0x6d, 0x3b, 0x9e, 0xfa, 0x3f, 0x25, 0xe8,
0xee, 0xe8, 0x87, 0xf8, 0xff, 0xcf, 0x01, 0x7d, 0x06, 0xe7, 0x3c, 0xfd, 0x10, 0x0f, 0x22, 0x51,
0xf5, 0xc0, 0xc5, 0x4f, 0xb9, 0x13, 0xfb, 0x86, 0x2c, 0xb7, 0x2e, 0xbd, 0x22, 0xa4, 0x2d, 0x7b,
0xb1, 0x7e, 0x0d, 0x3f, 0x45, 0xaf, 0xc1, 0x52, 0xf4, 0xe6, 0x19, 0x41, 0xad, 0x4e, 0x49, 0xbe,
0x18, 0xb9, 0x5d, 0xd6, 0x37, 0xd4, 0xa7, 0xf0, 0xea, 0x23, 0xdb, 0xc3, 0x7e, 0x3f, 0xbc, 0x21,
0x35, 0x67, 0xfc, 0x79, 0x05, 0x9a, 0x21, 0xe1, 0x53, 0x8f, 0x3b, 0x0c, 0x4f, 0x75, 0xa0, 0xb7,
0xa5, 0xbb, 0x07, 0x41, 0x8e, 0x7a, 0x93, 0x5d, 0x67, 0x79, 0x81, 0x0b, 0xee, 0x89, 0x8b, 0x5d,
0x1a, 0xde, 0xc3, 0x2e, 0xb6, 0x87, 0xf8, 0x81, 0x33, 0x3c, 0x20, 0x0e, 0x89, 0xcf, 0xde, 0xd7,
0x29, 0x11, 0xdf, 0x75, 0x33, 0xf2, 0x7c, 0xae, 0x14, 0x7b, 0x3e, 0x37, 0xe3, 0x39, 0xa6, 0xfa,
0x93, 0x12, 0xac, 0xdc, 0xb5, 0x7c, 0xec, 0x86, 0x69, 0x83, 0x93, 0x64, 0x40, 0xc2, 0x94, 0x44,
0xe9, 0x34, 0x35, 0x86, 0x02, 0x25, 0x48, 0x59, 0x02, 0xa5, 0x72, 0xca, 0x04, 0xca, 0x5d, 0x80,
0x89, 0xeb, 0x4c, 0xb0, 0xeb, 0x9b, 0x38, 0x88, 0xfd, 0x0a, 0x38, 0x38, 0x91, 0x41, 0xea, 0x67,
0xd0, 0xb9, 0x3f, 0xdc, 0x70, 0xec, 0x3d, 0xd3, 0x1d, 0x07, 0x84, 0x4a, 0x09, 0x9d, 0x52, 0x40,
0xe8, 0x4a, 0x29, 0xa1, 0x53, 0x4d, 0x58, 0x8e, 0xcc, 0x3d, 0xa7, 0xe2, 0x1a, 0x0d, 0x07, 0x7b,
0xa6, 0x6d, 0xd2, 0xeb, 0x62, 0x25, 0xea, 0xa0, 0xc2, 0x68, 0x78, 0x8f, 0xf7, 0xa8, 0x3f, 0x54,
0xe0, 0xa2, 0x86, 0x89, 0xf0, 0x04, 0x37, 0x6f, 0x76, 0xfd, 0x2d, 0x6f, 0x34, 0x87, 0x43, 0x71,
0x07, 0x2a, 0x63, 0x6f, 0x94, 0x51, 0x35, 0x27, 0x26, 0x3a, 0xb6, 0x90, 0x46, 0x81, 0x6f, 0x7e,
0x24, 0x2e, 0xf8, 0xee, 0x1e, 0x4f, 0x30, 0x5a, 0x80, 0xf2, 0x43, 0x7c, 0xd4, 0x39, 0x83, 0x00,
0x6a, 0x0f, 0x1d, 0x77, 0xac, 0x5b, 0x1d, 0x05, 0x35, 0x61, 0x81, 0xd7, 0x1e, 0x3b, 0x25, 0xb4,
0x08, 0x8d, 0x8d, 0xa0, 0x46, 0xd3, 0x29, 0xdf, 0xfc, 0x13, 0x05, 0x96, 0x53, 0xd5, 0x31, 0xd4,
0x06, 0x78, 0x64, 0x0f, 0x79, 0xd9, 0xb0, 0x73, 0x06, 0xb5, 0xa0, 0x1e, 0x14, 0x11, 0xd9, 0x7c,
0xbb, 0x0e, 0x85, 0xee, 0x94, 0x50, 0x07, 0x5a, 0x6c, 0xe0, 0x74, 0x38, 0xc4, 0x9e, 0xd7, 0x29,
0x8b, 0x9e, 0x7b, 0xba, 0x69, 0x4d, 0x5d, 0xdc, 0xa9, 0x90, 0x35, 0x77, 0x1d, 0x0d, 0x5b, 0x58,
0xf7, 0x70, 0xa7, 0x8a, 0x10, 0xb4, 0x79, 0x23, 0x18, 0x54, 0x8b, 0xf4, 0x05, 0xc3, 0x16, 0x6e,
0x7e, 0x1a, 0xad, 0x63, 0xd0, 0xed, 0x9d, 0x87, 0xb3, 0x8f, 0x6c, 0x03, 0xef, 0x99, 0x36, 0x36,
0xc2, 0x4f, 0x9d, 0x33, 0xe8, 0x2c, 0x2c, 0x6d, 0x61, 0x77, 0x84, 0x23, 0x9d, 0x25, 0xb4, 0x0c,
0x8b, 0x5b, 0xe6, 0xb3, 0x48, 0x57, 0x59, 0xad, 0xd4, 0x95, 0x8e, 0xb2, 0xf6, 0xb7, 0x37, 0xa0,
0x41, 0x88, 0xb9, 0xe1, 0x38, 0xae, 0x81, 0x2c, 0x40, 0xf4, 0x19, 0xcd, 0x78, 0xe2, 0xd8, 0xe2,
0xc9, 0x1d, 0xba, 0x95, 0xa0, 0x3f, 0x6b, 0xa4, 0x01, 0xf9, 0x79, 0xf7, 0xae, 0x4b, 0xe1, 0x13,
0xc0, 0xea, 0x19, 0x34, 0xa6, 0xab, 0xed, 0x9a, 0x63, 0xbc, 0x6b, 0x0e, 0x0f, 0x02, 0x17, 0xed,
0x9d, 0x8c, 0x77, 0x4b, 0x69, 0xd0, 0x60, 0xbd, 0x6b, 0xd2, 0xf5, 0xd8, 0x3b, 0xa7, 0x80, 0xf5,
0xd5, 0x33, 0xe8, 0x29, 0x9c, 0xbb, 0x8f, 0x23, 0xfe, 0x6e, 0xb0, 0xe0, 0x5a, 0xf6, 0x82, 0x29,
0xe0, 0x13, 0x2e, 0xf9, 0x00, 0xaa, 0x94, 0xdd, 0x90, 0xac, 0xb4, 0x1b, 0x7d, 0x2f, 0xdf, 0xbb,
0x9a, 0x0d, 0x20, 0x66, 0xfb, 0x01, 0x2c, 0x25, 0x5e, 0xd2, 0x22, 0x99, 0x8d, 0x94, 0xbf, 0x89,
0xee, 0xdd, 0x2c, 0x02, 0x2a, 0xd6, 0x1a, 0x41, 0x3b, 0xfe, 0xfc, 0x06, 0xad, 0x16, 0x78, 0xc4,
0xc7, 0x56, 0x7a, 0xa3, 0xf0, 0x73, 0x3f, 0xca, 0x04, 0x9d, 0xe4, 0x1b, 0x4f, 0x74, 0x33, 0x77,
0x82, 0x38, 0xb3, 0xbd, 0x59, 0x08, 0x56, 0x2c, 0x77, 0x4c, 0x99, 0x20, 0xf5, 0xc0, 0x2e, 0xc9,
0xe3, 0xc1, 0x34, 0x59, 0x2f, 0xff, 0x7a, 0xb7, 0x0b, 0xc3, 0x8b, 0xa5, 0x7f, 0x95, 0xdd, 0xdf,
0x92, 0x3d, 0x52, 0x43, 0xef, 0xca, 0xa7, 0xcb, 0x79, 0x5d, 0xd7, 0x5b, 0x3b, 0xc9, 0x10, 0x81,
0xc4, 0x2f, 0xd3, 0x8b, 0x57, 0x92, 0x67, 0x5e, 0x49, 0xb9, 0x0b, 0xe6, 0xcb, 0x7e, 0xc1, 0xd6,
0x7b, 0xf7, 0x04, 0x23, 0x04, 0x02, 0x4e, 0xf2, 0x11, 0x6d, 0x20, 0x86, 0xb7, 0x67, 0x72, 0xcd,
0xe9, 0x64, 0xf0, 0xfb, 0xb0, 0x94, 0xf0, 0x1a, 0x51, 0x71, 0xcf, 0xb2, 0x97, 0x67, 0x21, 0x99,
0x48, 0x26, 0x2e, 0x5a, 0xa1, 0x0c, 0xee, 0x97, 0x5c, 0xc6, 0xea, 0xdd, 0x2c, 0x02, 0x2a, 0x36,
0x32, 0x81, 0xe5, 0xc4, 0xc7, 0xc7, 0x6b, 0xe8, 0xcd, 0xc2, 0xab, 0x3d, 0x5e, 0xeb, 0xbd, 0x55,
0x7c, 0xbd, 0xc7, 0x6b, 0xea, 0x19, 0xe4, 0x51, 0x05, 0x9d, 0xb8, 0xac, 0x83, 0x32, 0x66, 0x91,
0x5f, 0x4a, 0xea, 0xbd, 0x5d, 0x10, 0x5a, 0x6c, 0xf3, 0x10, 0xce, 0x4a, 0xee, 0x54, 0xa1, 0xb7,
0x73, 0xd9, 0x23, 0x79, 0x99, 0xac, 0x77, 0xab, 0x28, 0x78, 0xc4, 0x3c, 0x74, 0x02, 0xbc, 0xee,
0x5a, 0x16, 0x33, 0xfe, 0x6f, 0x65, 0x59, 0xbe, 0x18, 0x58, 0xc6, 0x56, 0x33, 0xa1, 0xc5, 0x92,
0xbf, 0x08, 0x68, 0x67, 0xdf, 0x39, 0xa2, 0x5e, 0xda, 0x68, 0xea, 0xea, 0xcc, 0xb1, 0xcc, 0x32,
0x80, 0x69, 0xd0, 0x0c, 0x41, 0xcc, 0x1d, 0x21, 0x16, 0x1f, 0x00, 0xdc, 0xc7, 0xfe, 0x16, 0xf6,
0x5d, 0x22, 0xfd, 0xaf, 0x65, 0xe1, 0xce, 0x01, 0x82, 0xa5, 0x5e, 0x9f, 0x09, 0x17, 0x25, 0xe8,
0x96, 0x6e, 0x4f, 0x75, 0x2b, 0xf2, 0xca, 0x45, 0x4e, 0xd0, 0x24, 0x58, 0x3e, 0x41, 0xd3, 0xd0,
0x62, 0xc9, 0x23, 0xe1, 0xbf, 0x44, 0x4a, 0xbf, 0xf9, 0xfe, 0x4b, 0xfa, 0xda, 0x51, 0x52, 0xb7,
0xe7, 0xc0, 0x8b, 0x85, 0xbf, 0x50, 0xe8, 0x4d, 0xc0, 0x04, 0xc0, 0xa7, 0xa6, 0xbf, 0xbf, 0x6d,
0xe9, 0xb6, 0x57, 0x04, 0x05, 0x0a, 0x78, 0x02, 0x14, 0x38, 0xbc, 0x40, 0xc1, 0x80, 0xc5, 0x58,
0x45, 0x16, 0xc9, 0xde, 0x86, 0xc8, 0xaa, 0xd3, 0xbd, 0xd5, 0xd9, 0x80, 0x62, 0x95, 0x7d, 0x58,
0x0c, 0x18, 0x9a, 0x11, 0xf7, 0x8d, 0x5c, 0xa6, 0x8f, 0xd1, 0xf5, 0x66, 0x11, 0x50, 0xb1, 0x92,
0x07, 0x28, 0x5d, 0x7a, 0x42, 0xc5, 0x0a, 0x95, 0x79, 0xca, 0x27, 0xbb, 0x9e, 0xc5, 0xf4, 0x79,
0xa2, 0xb8, 0x2b, 0x37, 0x16, 0xd2, 0x5a, 0xb5, 0x54, 0x9f, 0x67, 0xd4, 0x8a, 0xd5, 0x33, 0xe8,
0x53, 0xa8, 0xf1, 0x7f, 0xbb, 0xb9, 0x9e, 0x9f, 0xe4, 0xe5, 0xb3, 0xdf, 0x98, 0x01, 0x25, 0x26,
0x3e, 0x80, 0xf3, 0x19, 0x29, 0x5e, 0xa9, 0x9f, 0x91, 0x9f, 0x0e, 0x9e, 0x65, 0x01, 0xc5, 0x62,
0xa9, 0x0c, 0x6e, 0xce, 0x62, 0x59, 0xd9, 0xde, 0x59, 0x8b, 0x0d, 0x60, 0x39, 0x95, 0x21, 0x93,
0x9a, 0xc0, 0xac, 0x3c, 0xda, 0xac, 0x05, 0x46, 0xf0, 0x8a, 0x34, 0x1b, 0x24, 0xf5, 0x4e, 0xf2,
0xf2, 0x46, 0xb3, 0x16, 0x1a, 0xc2, 0x59, 0x49, 0x0e, 0x48, 0x6a, 0xe5, 0xb2, 0x73, 0x45, 0xb3,
0x16, 0xd9, 0x83, 0xde, 0xba, 0xeb, 0xe8, 0xc6, 0x50, 0xf7, 0x7c, 0x9a, 0x97, 0x21, 0xa1, 0x62,
0xe0, 0x1e, 0xca, 0x63, 0x07, 0x69, 0xf6, 0x66, 0xd6, 0x3a, 0x4f, 0xa0, 0x49, 0x8f, 0x92, 0xfd,
0x23, 0x09, 0x92, 0xdb, 0x88, 0x08, 0x44, 0x86, 0xe2, 0x91, 0x01, 0x0a, 0xa6, 0xde, 0x85, 0xe6,
0x06, 0x2d, 0x70, 0xf5, 0x6d, 0x03, 0x3f, 0x4b, 0xda, 0x2b, 0xfa, 0x2c, 0xfb, 0x56, 0x04, 0xa0,
0x30, 0x85, 0x16, 0xa9, 0xd7, 0x6e, 0xe0, 0x67, 0xec, 0x9c, 0x57, 0x65, 0xf3, 0xc6, 0x40, 0x32,
0xa2, 0x1c, 0x29, 0x64, 0xc4, 0xd2, 0x9f, 0x8b, 0xfa, 0xb2, 0x62, 0xb9, 0xdb, 0x19, 0x93, 0xa4,
0x20, 0x83, 0x55, 0xdf, 0x29, 0x3e, 0x20, 0x6a, 0x19, 0x02, 0xbc, 0xfa, 0xb4, 0xba, 0xf6, 0x7a,
0x1e, 0xea, 0x51, 0x07, 0x75, 0x75, 0x36, 0xa0, 0x58, 0x65, 0x1b, 0x1a, 0x84, 0x3b, 0xd9, 0xf1,
0x5c, 0x97, 0x0d, 0x14, 0x9f, 0x8b, 0x1f, 0xce, 0x26, 0xf6, 0x86, 0xae, 0xf9, 0x84, 0x1f, 0xba,
0x14, 0x9d, 0x18, 0x48, 0xee, 0xe1, 0x24, 0x20, 0x05, 0xe6, 0x53, 0xea, 0x35, 0x08, 0xd2, 0x71,
0x55, 0xf9, 0xf6, 0xac, 0xf3, 0x8d, 0xab, 0xc9, 0x5b, 0x45, 0xc1, 0xc5, 0xb2, 0xbf, 0x44, 0x23,
0x21, 0xfa, 0x7d, 0x7d, 0x6a, 0x5a, 0xc6, 0x36, 0xbf, 0x10, 0x8d, 0xde, 0xc9, 0x9b, 0x2a, 0x06,
0x9a, 0xe9, 0x00, 0xe6, 0x8c, 0x10, 0xeb, 0xff, 0x3c, 0x34, 0x44, 0x86, 0x10, 0xc9, 0x6e, 0xf4,
0x25, 0x73, 0x93, 0xbd, 0xeb, 0xf9, 0x40, 0x62, 0x66, 0x0c, 0xe7, 0x64, 0xf9, 0x40, 0x69, 0x90,
0x9d, 0x93, 0x38, 0x9c, 0xc1, 0x1f, 0x6b, 0x5f, 0x36, 0xa0, 0x1e, 0x0c, 0xfc, 0x8a, 0x53, 0x57,
0x2f, 0x21, 0x97, 0xf4, 0x7d, 0x58, 0x4a, 0xfc, 0xd1, 0x84, 0x54, 0x83, 0xcb, 0xff, 0x8c, 0x62,
0x96, 0xa8, 0x7d, 0xca, 0xff, 0x07, 0x51, 0x04, 0x79, 0xaf, 0x67, 0xe5, 0xa3, 0x92, 0xf1, 0xdd,
0x8c, 0x89, 0xff, 0x6f, 0x87, 0x38, 0x0f, 0x01, 0x22, 0xc1, 0x4d, 0xfe, 0x95, 0x6c, 0xe2, 0xaf,
0xcf, 0xa2, 0xd6, 0x58, 0x1a, 0xbf, 0xbc, 0x51, 0xe4, 0x7a, 0x6b, 0xb6, 0x07, 0x9a, 0x1d, 0xb5,
0x3c, 0x82, 0x56, 0xf4, 0xb1, 0x04, 0x92, 0xfe, 0xeb, 0x5e, 0xfa, 0x35, 0xc5, 0xac, 0x5d, 0x6c,
0x9d, 0xd0, 0xb1, 0x9d, 0x31, 0x9d, 0x07, 0x28, 0x5d, 0x29, 0x97, 0x06, 0x02, 0x99, 0xf5, 0x79,
0x69, 0x20, 0x90, 0x5d, 0x7e, 0x67, 0x69, 0xc9, 0x64, 0xf9, 0x57, 0x9a, 0x96, 0xcc, 0x28, 0xa8,
0x4b, 0xd3, 0x92, 0x59, 0xf5, 0x64, 0xf5, 0xcc, 0xfa, 0x9d, 0xcf, 0xde, 0x1d, 0x99, 0xfe, 0xfe,
0xf4, 0x09, 0xd9, 0xfd, 0x6d, 0x36, 0xf4, 0x6d, 0xd3, 0xe1, 0xbf, 0x6e, 0x07, 0xec, 0x7e, 0x9b,
0xce, 0x76, 0x9b, 0xcc, 0x36, 0x79, 0xf2, 0xa4, 0x46, 0x5b, 0x77, 0xfe, 0x37, 0x00, 0x00, 0xff,
0xff, 0x92, 0xb1, 0xb4, 0x00, 0x03, 0x56, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.

View File

@ -1316,19 +1316,9 @@ func getDefaultPartitionsInPartitionKeyMode(ctx context.Context, dbName string,
}
// Make sure the order of the partition names got every time is the same
partitionNames := make([]string, len(partitions))
for partitionName := range partitions {
splits := strings.Split(partitionName, "_")
if len(splits) < 2 {
err = fmt.Errorf("bad default partion name in partition ket mode: %s", partitionName)
return nil, err
}
index, err := strconv.ParseInt(splits[len(splits)-1], 10, 64)
if err != nil {
return nil, err
}
partitionNames[index] = partitionName
partitionNames, _, err := typeutil.RearrangePartitionsForPartitionKey(partitions)
if err != nil {
return nil, err
}
return partitionNames, nil

View File

@ -205,6 +205,7 @@ func (m *importManager) sendOutTasks(ctx context.Context) error {
TaskId: task.GetId(),
Files: task.GetFiles(),
Infos: task.GetInfos(),
DatabaseName: task.GetDatabaseName(),
}
// Get all busy dataNodes for reference.
@ -461,7 +462,8 @@ func (m *importManager) importJob(ctx context.Context, req *milvuspb.ImportReque
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPending,
},
Infos: req.Options,
Infos: req.Options,
DatabaseName: req.GetDbName(),
}
// Here no need to check error returned by setCollectionPartitionName(),
@ -499,7 +501,8 @@ func (m *importManager) importJob(ctx context.Context, req *milvuspb.ImportReque
State: &datapb.ImportTaskState{
StateCode: commonpb.ImportState_ImportPending,
},
Infos: req.Options,
Infos: req.Options,
DatabaseName: req.GetDbName(),
}
// Here no need to check error returned by setCollectionPartitionName(),
// since here we always return task list to client no matter something missed.

View File

@ -1820,37 +1820,31 @@ func (c *Core) Import(ctx context.Context, req *milvuspb.ImportRequest) (*milvus
}
return ret, nil
}
} else {
// In v2.2.9, bulkdinsert cannot support partition key, return error to client.
// Remove the following lines after bulkinsert can support partition key
for _, field := range colInfo.Fields {
if field.IsPartitionKey {
log.Info("partition key is not yet supported by bulkinsert",
zap.String("collection name", req.GetCollectionName()),
zap.String("partition key", field.Name))
ret := &milvuspb.ImportResponse{
Status: failStatus(commonpb.ErrorCode_UnexpectedError,
fmt.Sprintf("the collection '%s' contains partition key '%s', partition key is not yet supported by bulkinsert",
req.GetCollectionName(), field.Name)),
}
return ret, nil
}
}
// Remove the upper lines after bulkinsert can support partition key
}
cID := colInfo.CollectionID
req.ChannelNames = c.meta.GetCollectionVirtualChannels(cID)
if req.GetPartitionName() == "" {
req.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue()
hasPartitionKey := false
for _, field := range colInfo.Fields {
if field.IsPartitionKey {
hasPartitionKey = true
break
}
}
var pID UniqueID
if pID, err = c.meta.GetPartitionByName(cID, req.GetPartitionName(), typeutil.MaxTimestamp); err != nil {
log.Error("failed to get partition ID from its name",
zap.String("partition name", req.GetPartitionName()),
zap.Error(err))
return nil, err
if !hasPartitionKey {
if req.GetPartitionName() == "" {
req.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue()
}
if pID, err = c.meta.GetPartitionByName(cID, req.GetPartitionName(), typeutil.MaxTimestamp); err != nil {
log.Error("failed to get partition ID from its name",
zap.String("partition name", req.GetPartitionName()),
zap.Error(err))
return nil, err
}
}
log.Info("RootCoord receive import request",
zap.String("collection name", req.GetCollectionName()),
zap.Int64("collection ID", cID),

View File

@ -1031,8 +1031,9 @@ func TestCore_Import(t *testing.T) {
ctx := context.Background()
c := newTestCore(withHealthyCode(),
withMeta(meta))
meta.GetCollectionIDByNameFunc = func(name string) (UniqueID, error) {
return 100, nil
coll := &model.Collection{Name: "a-good-name"}
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
return coll, nil
}
meta.GetCollectionVirtualChannelsFunc = func(colID int64) []string {
return []string{"ch-1", "ch-2"}

View File

@ -42,20 +42,17 @@ type SegmentFilesHolder struct {
// 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.
// 1. read insert log of each field, then constructs SegmentData in memory.
// 2. read delta log to remove deleted entities(TimeStampField is used to apply or skip the operation).
// 3. split data according to shard number
// 4. call the callFlushFunc function to flush data into new binlog file if data size reaches blockSize.
type BinlogAdapter struct {
ctx context.Context // for canceling parse process
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
blockSize int64 // maximum size of a read block(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
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)
// 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
@ -71,39 +68,43 @@ type BinlogAdapter struct {
}
func NewBinlogAdapter(ctx context.Context,
collectionSchema *schemapb.CollectionSchema,
shardNum int32,
collectionInfo *CollectionInfo,
blockSize int64,
maxTotalSize int64,
chunkManager storage.ChunkManager,
flushFunc ImportFlushFunc,
tsStartPoint uint64,
tsEndPoint uint64) (*BinlogAdapter, error) {
if collectionSchema == nil {
log.Error("Binlog adapter: collection schema is nil")
if collectionInfo == nil {
log.Warn("Binlog adapter: collection schema is nil")
return nil, errors.New("collection schema is nil")
}
// binlog import doesn't support partition key, the caller must specify one partition for importing
if len(collectionInfo.PartitionIDs) != 1 {
log.Warn("Binlog adapter: target partition must be only one", zap.Int("partitions", len(collectionInfo.PartitionIDs)))
return nil, errors.New("target partition must be only one")
}
if chunkManager == nil {
log.Error("Binlog adapter: chunk manager pointer is nil")
log.Warn("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")
log.Warn("Binlog adapter: flush function is nil")
return nil, errors.New("flush function is nil")
}
adapter := &BinlogAdapter{
ctx: ctx,
collectionSchema: collectionSchema,
chunkManager: chunkManager,
callFlushFunc: flushFunc,
shardNum: shardNum,
blockSize: blockSize,
maxTotalSize: maxTotalSize,
tsStartPoint: tsStartPoint,
tsEndPoint: tsEndPoint,
ctx: ctx,
collectionInfo: collectionInfo,
chunkManager: chunkManager,
callFlushFunc: flushFunc,
blockSize: blockSize,
maxTotalSize: maxTotalSize,
tsStartPoint: tsStartPoint,
tsEndPoint: tsEndPoint,
}
// amend the segment size to avoid portential OOM risk
@ -111,28 +112,12 @@ func NewBinlogAdapter(ctx context.Context,
adapter.blockSize = 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")
log.Warn("Binlog adapter: segment files holder is nil")
return errors.New("segment files holder is nil")
}
@ -163,25 +148,26 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error {
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")
// 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))
return fmt.Errorf("fail to initialize in-memory segment data for shard id %d", i)
}
segmentsData = append(segmentsData, segmentData)
shardsData = append(shardsData, shardData)
}
// read binlog files batch by batch
primaryKey := p.collectionInfo.PrimaryKey
for i := 0; i < batchCount; i++ {
// batchFiles excludes the primary key field and the timestamp field
// 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 {
if fieldID == primaryKey.GetFieldID() || fieldID == common.TimeStampField {
continue
}
batchFiles[fieldID] = files[i]
@ -189,8 +175,9 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error {
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))
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))
intList, strList, err := p.readPrimaryKeys(primaryLog)
if err != nil {
return err
@ -205,21 +192,21 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error {
}
var shardList []int32
if p.primaryType == schemapb.DataType_Int64 {
if primaryKey.GetDataType() == schemapb.DataType_Int64 {
// calculate a shard num list by primary keys and deleted entities
shardList, err = p.getShardingListByPrimaryInt64(intList, timestampList, segmentsData, intDeletedList)
shardList, err = p.getShardingListByPrimaryInt64(intList, timestampList, shardsData, intDeletedList)
if err != nil {
return err
}
} else if p.primaryType == schemapb.DataType_VarChar {
} else if primaryKey.GetDataType() == schemapb.DataType_VarChar {
// calculate a shard num list by primary keys and deleted entities
shardList, err = p.getShardingListByPrimaryVarchar(strList, timestampList, segmentsData, strDeletedList)
shardList, err = p.getShardingListByPrimaryVarchar(strList, timestampList, shardsData, strDeletedList)
if err != nil {
return err
}
} else {
log.Error("Binlog adapter: unsupported primary key type", zap.Int("type", int(p.primaryType)))
return fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", p.primaryType)
log.Warn("Binlog adapter: unsupported primary key type", zap.Int("type", int(primaryKey.GetDataType())))
return fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType())
}
// if shardList is empty, that means all the primary keys have been deleted(or skipped), no need to read other files
@ -231,25 +218,25 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error {
for fieldID, file := range batchFiles {
// outside context might be canceled(service stop, or future enhancement for canceling import task)
if isCanceled(p.ctx) {
log.Error("Binlog adapter: import task was canceled")
log.Warn("Binlog adapter: import task was canceled")
return errors.New("import task was canceled")
}
err = p.readInsertlog(fieldID, file, segmentsData, shardList)
err = p.readInsertlog(fieldID, file, shardsData, shardList)
if err != nil {
return err
}
}
// flush segment whose size exceed blockSize
err = tryFlushBlocks(p.ctx, segmentsData, p.collectionSchema, p.callFlushFunc, p.blockSize, p.maxTotalSize, false)
err = tryFlushBlocks(p.ctx, shardsData, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, p.maxTotalSize, false)
if err != nil {
return err
}
}
// finally, force to flush
return tryFlushBlocks(p.ctx, segmentsData, p.collectionSchema, p.callFlushFunc, p.blockSize, p.maxTotalSize, true)
return tryFlushBlocks(p.ctx, shardsData, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, p.maxTotalSize, true)
}
// verify method verify the schema and binlog files
@ -259,18 +246,18 @@ func (p *BinlogAdapter) Read(segmentHolder *SegmentFilesHolder) error {
// 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")
log.Warn("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]
for i := 0; i < len(p.collectionInfo.Schema.Fields); i++ {
schema := p.collectionInfo.Schema.Fields[i]
files, ok := segmentHolder.fieldFiles[schema.FieldID]
if !ok {
log.Error("Binlog adapter: a field has no binlog file", zap.Int64("fieldID", schema.FieldID))
log.Warn("Binlog adapter: a field has no binlog file", zap.Int64("fieldID", schema.FieldID))
return fmt.Errorf("the field %d has no binlog file", schema.FieldID)
}
@ -282,21 +269,21 @@ func (p *BinlogAdapter) verify(segmentHolder *SegmentFilesHolder) error {
// 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")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog adapter: file count of each field must be equal", zap.Int("firstFieldFileCount", firstFieldFileCount))
return fmt.Errorf("binlog file count of each field must be equal, first field files count: %d, other field files count: %d",
firstFieldFileCount, len(files))
}
@ -319,14 +306,15 @@ func (p *BinlogAdapter) readDeltalogs(segmentHolder *SegmentFilesHolder) (map[in
return nil, nil, nil // no deletion
}
if p.primaryType == schemapb.DataType_Int64 {
primaryKey := p.collectionInfo.PrimaryKey
if primaryKey.GetDataType() == 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 {
} else if primaryKey.GetDataType() == schemapb.DataType_VarChar {
deletedIDDict := make(map[string]uint64)
for _, deleteLog := range deleteLogs {
deletedIDDict[deleteLog.Pk.GetValue().(string)] = deleteLog.Ts
@ -334,8 +322,8 @@ func (p *BinlogAdapter) readDeltalogs(segmentHolder *SegmentFilesHolder) (map[in
log.Info("Binlog adapter: count of deleted entities", zap.Int("deletedCount", len(deletedIDDict)))
return nil, deletedIDDict, nil
} else {
log.Error("Binlog adapter: unsupported primary key type", zap.Int("type", int(p.primaryType)))
return nil, nil, fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", p.primaryType)
log.Warn("Binlog adapter: unsupported primary key type", zap.Int("type", int(primaryKey.GetDataType())))
return nil, nil, fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType())
}
}
@ -375,13 +363,14 @@ func (p *BinlogAdapter) decodeDeleteLogs(segmentHolder *SegmentFilesHolder) ([]*
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
primaryKey := p.collectionInfo.PrimaryKey
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",
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",
zap.Int64("deltaDataType", deleteLogs[i].PkType),
zap.Int64("pkDataType", int64(p.primaryType)))
zap.Int64("pkDataType", int64(primaryKey.GetDataType())))
return nil, fmt.Errorf("delta log data type %d is not equal to collection's primary key data type %d",
deleteLogs[i].PkType, p.primaryType)
deleteLogs[i].PkType, primaryKey.GetDataType())
}
}
@ -398,12 +387,12 @@ func (p *BinlogAdapter) decodeDeleteLog(deltaStr string) (*storage.DeleteLog, er
// 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))
log.Warn("Binlog adapter: the format of deletion string is incorrect", zap.String("deltaStr", deltaStr))
return nil, fmt.Errorf("the format of deletion string is incorrect, '%s' can not be split", deltaStr)
}
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",
log.Warn("Binlog adapter: failed to parse primary key of deletion string from old version",
zap.String("deltaStr", deltaStr), zap.Error(err))
return nil, fmt.Errorf("failed to parse primary key of deletion string '%s' from old version, error: %w", deltaStr, err)
}
@ -413,7 +402,7 @@ func (p *BinlogAdapter) decodeDeleteLog(deltaStr string) (*storage.DeleteLog, er
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",
log.Warn("Binlog adapter: failed to parse timestamp of deletion string from old version",
zap.String("deltaStr", deltaStr), zap.Error(err))
return nil, fmt.Errorf("failed to parse timestamp of deletion string '%s' from old version, error: %w", deltaStr, err)
}
@ -427,13 +416,13 @@ 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))
log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err))
return nil, fmt.Errorf("failed to initialize binlog file '%s', error: %w", logPath, err)
}
err = binlogFile.Open(logPath)
if err != nil {
log.Error("Binlog adapter: failed to open delta log", zap.String("logPath", logPath), zap.Error(err))
log.Warn("Binlog adapter: failed to open delta log", zap.String("logPath", logPath), zap.Error(err))
return nil, fmt.Errorf("failed to open delta log '%s', error: %w", logPath, err)
}
defer binlogFile.Close()
@ -441,7 +430,7 @@ func (p *BinlogAdapter) readDeltalog(logPath string) ([]string, error) {
// 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))
log.Warn("Binlog adapter: failed to read delta log", zap.String("logPath", logPath), zap.Error(err))
return nil, fmt.Errorf("failed to read delta log '%s', error: %w", logPath, err)
}
log.Info("Binlog adapter: successfully read deltalog", zap.Int("deleteCount", len(data)))
@ -454,13 +443,13 @@ 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))
log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err))
return nil, fmt.Errorf("failed to initialize binlog file '%s', error: %w", logPath, err)
}
err = binlogFile.Open(logPath)
if err != nil {
log.Error("Binlog adapter: failed to open timestamp log file", zap.String("logPath", logPath))
log.Warn("Binlog adapter: failed to open timestamp log file", zap.String("logPath", logPath))
return nil, fmt.Errorf("failed to open timestamp log file '%s', error: %w", logPath, err)
}
defer binlogFile.Close()
@ -468,7 +457,7 @@ func (p *BinlogAdapter) readTimestamp(logPath string) ([]int64, error) {
// 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))
log.Warn("Binlog adapter: failed to read timestamp data from log file", zap.String("logPath", logPath))
return nil, fmt.Errorf("failed to read timestamp data from log file '%s', error: %w", logPath, err)
}
@ -482,37 +471,38 @@ func (p *BinlogAdapter) readPrimaryKeys(logPath string) ([]int64, []string, erro
// 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))
log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err))
return nil, nil, fmt.Errorf("failed to initialize binlog file '%s', error: %w", logPath, err)
}
err = binlogFile.Open(logPath)
if err != nil {
log.Error("Binlog adapter: failed to open primary key binlog", zap.String("logPath", logPath))
log.Warn("Binlog adapter: failed to open primary key binlog", zap.String("logPath", logPath))
return nil, nil, fmt.Errorf("failed to open primary key binlog '%s', error: %w", logPath, err)
}
defer binlogFile.Close()
// primary key can be int64 or varchar, we need to handle the two cases
if p.primaryType == schemapb.DataType_Int64 {
primaryKey := p.collectionInfo.PrimaryKey
if primaryKey.GetDataType() == 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))
log.Warn("Binlog adapter: failed to read int64 primary key from binlog", zap.String("logPath", logPath), zap.Error(err))
return nil, nil, fmt.Errorf("failed to read int64 primary key from binlog '%s', error: %w", logPath, err)
}
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 {
} else if primaryKey.GetDataType() == 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))
log.Warn("Binlog adapter: failed to read varchar primary key from binlog", zap.String("logPath", logPath), zap.Error(err))
return nil, nil, fmt.Errorf("failed to read varchar primary key from binlog '%s', error: %w", logPath, err)
}
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: unsupported primary key type", zap.Int("type", int(p.primaryType)))
return nil, nil, fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", p.primaryType)
log.Warn("Binlog adapter: unsupported primary key type", zap.Int("type", int(primaryKey.GetDataType())))
return nil, nil, fmt.Errorf("unsupported primary key type %d, primary key should be int64 or varchar", primaryKey.GetDataType())
}
}
@ -522,10 +512,10 @@ func (p *BinlogAdapter) readPrimaryKeys(logPath string) ([]int64, []string, erro
// Compare timestampList with tsEndPoint to skip some rows.
func (p *BinlogAdapter) getShardingListByPrimaryInt64(primaryKeys []int64,
timestampList []int64,
memoryData []map[storage.FieldID]storage.FieldData,
memoryData []ShardData,
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",
log.Warn("Binlog adapter: primary key length is not equal to timestamp list length",
zap.Int("primaryKeysLen", len(primaryKeys)), zap.Int("timestampLen", len(timestampList)))
return nil, fmt.Errorf("primary key length %d is not equal to timestamp list length %d", len(primaryKeys), len(timestampList))
}
@ -535,6 +525,7 @@ func (p *BinlogAdapter) getShardingListByPrimaryInt64(primaryKeys []int64,
actualDeleted := 0
excluded := 0
shardList := make([]int32, 0, len(primaryKeys))
primaryKey := p.collectionInfo.PrimaryKey
for i, key := range primaryKeys {
// if this entity's timestamp is greater than the tsEndPoint, or less than tsStartPoint, set shardID = -1 to skip this entity
// timestamp is stored as int64 type in log file, actually it is uint64, compare with uint64
@ -552,9 +543,10 @@ func (p *BinlogAdapter) getShardingListByPrimaryInt64(primaryKeys []int64,
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
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
// append the entity to primary key's FieldData
field.(*storage.Int64FieldData).Data = append(field.(*storage.Int64FieldData).Data, key)
@ -573,10 +565,10 @@ func (p *BinlogAdapter) getShardingListByPrimaryInt64(primaryKeys []int64,
// [0, 1, -1, 1, 0, 1, -1, 1, 0, 1]
func (p *BinlogAdapter) getShardingListByPrimaryVarchar(primaryKeys []string,
timestampList []int64,
memoryData []map[storage.FieldID]storage.FieldData,
memoryData []ShardData,
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",
log.Warn("Binlog adapter: primary key length is not equal to timestamp list length",
zap.Int("primaryKeysLen", len(primaryKeys)), zap.Int("timestampLen", len(timestampList)))
return nil, fmt.Errorf("primary key length %d is not equal to timestamp list length %d", len(primaryKeys), len(timestampList))
}
@ -586,6 +578,7 @@ func (p *BinlogAdapter) getShardingListByPrimaryVarchar(primaryKeys []string,
actualDeleted := 0
excluded := 0
shardList := make([]int32, 0, len(primaryKeys))
primaryKey := p.collectionInfo.PrimaryKey
for i, key := range primaryKeys {
// if this entity's timestamp is greater than the tsEndPoint, or less than tsStartPoint, set shardID = -1 to skip this entity
// timestamp is stored as int64 type in log file, actually it is uint64, compare with uint64
@ -603,9 +596,10 @@ func (p *BinlogAdapter) getShardingListByPrimaryVarchar(primaryKeys []string,
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
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
// append the entity to primary key's FieldData
field.(*storage.StringFieldData).Data = append(field.(*storage.StringFieldData).Data, key)
@ -621,11 +615,12 @@ func (p *BinlogAdapter) getShardingListByPrimaryVarchar(primaryKeys []string,
// 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
func (p *BinlogAdapter) verifyField(fieldID storage.FieldID, memoryData []map[storage.FieldID]storage.FieldData) error {
for _, fields := range memoryData {
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
_, ok := fields[fieldID]
if !ok {
log.Error("Binlog adapter: the field ID doesn't exist in collection schema", zap.Int64("fieldID", fieldID))
log.Warn("Binlog adapter: the field ID doesn't exist in collection schema", zap.Int64("fieldID", fieldID))
return fmt.Errorf("the field ID %d doesn't exist in collection schema", fieldID)
}
}
@ -642,23 +637,23 @@ func (p *BinlogAdapter) verifyField(fieldID storage.FieldID, memoryData []map[st
// 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 {
memoryData []ShardData, shardList []int32) error {
err := p.verifyField(fieldID, memoryData)
if err != nil {
log.Error("Binlog adapter: could not read binlog file", zap.String("logPath", logPath), zap.Error(err))
log.Warn("Binlog adapter: could not read binlog file", zap.String("logPath", logPath), zap.Error(err))
return fmt.Errorf("could not read binlog file %s, error: %w", logPath, err)
}
// 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))
log.Warn("Binlog adapter: failed to initialize binlog file", zap.String("logPath", logPath), zap.Error(err))
return fmt.Errorf("failed to initialize binlog file %s, error: %w", logPath, err)
}
err = binlogFile.Open(logPath)
if err != nil {
log.Error("Binlog adapter: failed to open insert log", zap.String("logPath", logPath), zap.Error(err))
log.Warn("Binlog adapter: failed to open insert log", zap.String("logPath", logPath), zap.Error(err))
return fmt.Errorf("failed to open insert log %s, error: %w", logPath, err)
}
defer binlogFile.Close()
@ -783,11 +778,11 @@ func (p *BinlogAdapter) readInsertlog(fieldID storage.FieldID, logPath string,
return nil
}
func (p *BinlogAdapter) dispatchBoolToShards(data []bool, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchBoolToShards(data []bool, memoryData []ShardData,
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 shard list row count %d", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: bool field row count is not equal to shard list row count %d", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("bool field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -797,20 +792,31 @@ func (p *BinlogAdapter) dispatchBoolToShards(data []bool, memoryData []map[stora
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: bool field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("bool field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is bool type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchInt8ToShards(data []int8, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchInt8ToShards(data []int8, memoryData []ShardData,
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 shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: int8 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("int8 field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -820,20 +826,31 @@ func (p *BinlogAdapter) dispatchInt8ToShards(data []int8, memoryData []map[stora
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: int8 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("int8 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is int8 type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchInt16ToShards(data []int16, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchInt16ToShards(data []int16, memoryData []ShardData,
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 shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: int16 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("int16 field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -843,20 +860,31 @@ func (p *BinlogAdapter) dispatchInt16ToShards(data []int16, memoryData []map[sto
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: int16 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("int16 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is int16 type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchInt32ToShards(data []int32, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchInt32ToShards(data []int32, memoryData []ShardData,
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 shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: int32 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("int32 field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -866,20 +894,31 @@ func (p *BinlogAdapter) dispatchInt32ToShards(data []int32, memoryData []map[sto
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: int32 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("int32 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is int32 type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchInt64ToShards(data []int64, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchInt64ToShards(data []int64, memoryData []ShardData,
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 shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: int64 field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("int64 field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -889,20 +928,31 @@ func (p *BinlogAdapter) dispatchInt64ToShards(data []int64, memoryData []map[sto
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: int64 field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("int64 field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is int64 type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchFloatToShards(data []float32, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchFloatToShards(data []float32, memoryData []ShardData,
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 shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: float field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("float field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -912,20 +962,31 @@ func (p *BinlogAdapter) dispatchFloatToShards(data []float32, memoryData []map[s
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: float field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("float field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is float type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchDoubleToShards(data []float64, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchDoubleToShards(data []float64, memoryData []ShardData,
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 shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: double field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("double field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -935,20 +996,31 @@ func (p *BinlogAdapter) dispatchDoubleToShards(data []float64, memoryData []map[
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: double field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("double field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is double type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchVarcharToShards(data []string, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchVarcharToShards(data []string, memoryData []ShardData,
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 shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: varchar field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("varchar field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -958,20 +1030,31 @@ func (p *BinlogAdapter) dispatchVarcharToShards(data []string, memoryData []map[
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: varchar field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("varchar field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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)
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))
return fmt.Errorf("binlog is varchar type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchBytesToShards(data [][]byte, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchBytesToShards(data [][]byte, memoryData []ShardData,
shardList []int32, fieldID storage.FieldID) error {
// verify row count
if len(data) != len(shardList) {
log.Error("Binlog adapter: JSON field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
log.Warn("Binlog adapter: JSON field row count is not equal to shard list row count", zap.Int("dataLen", len(data)), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("varchar JSON row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -981,22 +1064,33 @@ func (p *BinlogAdapter) dispatchBytesToShards(data [][]byte, memoryData []map[st
if shardID < 0 {
continue // this entity has been deleted or excluded by timestamp
}
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: JSON field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("JSON field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
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.JSONFieldData).Data = append(field.(*storage.JSONFieldData).Data, val)
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))
return fmt.Errorf("binlog is JSON type, unequal to field %d", fieldID)
}
fieldData.Data = append(fieldData.Data, val)
}
return nil
}
func (p *BinlogAdapter) dispatchBinaryVecToShards(data []byte, dim int, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchBinaryVecToShards(data []byte, dim int, memoryData []ShardData,
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 shard list row count",
log.Warn("Binlog adapter: binary vector field row count is not equal to shard list row count",
zap.Int("dataLen", count), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("binary vector field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -1007,36 +1101,42 @@ func (p *BinlogAdapter) dispatchBinaryVecToShards(data []byte, dim int, memoryDa
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",
zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID))
return fmt.Errorf("the in-memory field is not a binary vector field, field id: %d", fieldID)
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: binary vector field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("binary vector field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
if binVecField.Dim != dim {
log.Error("Binlog adapter: binary vector dimension mismatch",
zap.Int("sourceDim", dim), zap.Int("schemaDim", binVecField.Dim))
return fmt.Errorf("binary vector dimension %d is not equal to schema dimension %d", dim, binVecField.Dim)
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",
zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID))
return fmt.Errorf("binlog is binary vector type, unequal to field %d", fieldID)
}
if fieldData.Dim != dim {
log.Warn("Binlog adapter: binary vector dimension mismatch",
zap.Int("sourceDim", dim), zap.Int("schemaDim", fieldData.Dim))
return fmt.Errorf("binary vector dimension %d is not equal to schema dimension %d", dim, fieldData.Dim)
}
for j := 0; j < bytesPerVector; j++ {
val := data[bytesPerVector*i+j]
binVecField.Data = append(binVecField.Data, val)
fieldData.Data = append(fieldData.Data, val)
}
}
return nil
}
func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memoryData []map[storage.FieldID]storage.FieldData,
func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memoryData []ShardData,
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 shard list row count",
log.Warn("Binlog adapter: float vector field row count is not equal to shard list row count",
zap.Int("dataLen", count), zap.Int("shardLen", len(shardList)))
return fmt.Errorf("float vector field row count %d is not equal to shard list row count %d", len(data), len(shardList))
}
@ -1047,23 +1147,29 @@ func (p *BinlogAdapter) dispatchFloatVecToShards(data []float32, dim int, memory
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",
zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID))
return fmt.Errorf("the in-memory field is not a float vector field, field id: %d", fieldID)
if shardID >= int32(len(memoryData)) {
log.Warn("Binlog adapter: float vector field's shard ID is illegal", zap.Int32("shardID", shardID), zap.Int("shardsCount", len(memoryData)))
return fmt.Errorf("float vector field's shard ID %d is larger than shards number %d", shardID, len(memoryData))
}
if floatVecField.Dim != dim {
log.Error("Binlog adapter: float vector dimension mismatch",
zap.Int("sourceDim", dim), zap.Int("schemaDim", floatVecField.Dim))
return fmt.Errorf("binary vector dimension %d is not equal to schema dimension %d", dim, floatVecField.Dim)
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",
zap.Int64("fieldID", fieldID), zap.Int32("shardID", shardID))
return fmt.Errorf("binlog is float vector type, unequal to field %d", fieldID)
}
if fieldData.Dim != dim {
log.Warn("Binlog adapter: float vector dimension mismatch",
zap.Int("sourceDim", dim), zap.Int("schemaDim", fieldData.Dim))
return fmt.Errorf("binary vector dimension %d is not equal to schema dimension %d", dim, fieldData.Dim)
}
for j := 0; j < dim; j++ {
val := data[dim*i+j]
floatVecField.Data = append(floatVecField.Data, val)
fieldData.Data = append(fieldData.Data, val)
}
}

File diff suppressed because it is too large Load Diff

View File

@ -39,7 +39,7 @@ type BinlogFile struct {
func NewBinlogFile(chunkManager storage.ChunkManager) (*BinlogFile, error) {
if chunkManager == nil {
log.Error("Binlog file: chunk manager pointer is nil")
log.Warn("Binlog file: chunk manager pointer is nil")
return nil, errors.New("chunk manager pointer is nil")
}
@ -53,20 +53,20 @@ func NewBinlogFile(chunkManager storage.ChunkManager) (*BinlogFile, error) {
func (p *BinlogFile) Open(filePath string) error {
p.Close()
if len(filePath) == 0 {
log.Error("Binlog file: binlog path is empty")
log.Warn("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))
log.Warn("Binlog file: failed to open binlog", zap.String("filePath", filePath), zap.Error(err))
return fmt.Errorf("failed to open binlog %s", filePath)
}
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))
log.Warn("Binlog file: failed to initialize binlog reader", zap.String("filePath", filePath), zap.Error(err))
return fmt.Errorf("failed to initialize binlog reader for binlog %s, error: %w", filePath, err)
}
@ -94,7 +94,7 @@ func (p *BinlogFile) DataType() schemapb.DataType {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadBool() ([]bool, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -102,7 +102,7 @@ func (p *BinlogFile) ReadBool() ([]bool, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -112,18 +112,18 @@ func (p *BinlogFile) ReadBool() ([]bool, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read bool data", zap.Error(err))
return nil, fmt.Errorf("failed to read bool data, error: %w", err)
}
@ -137,7 +137,7 @@ func (p *BinlogFile) ReadBool() ([]bool, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadInt8() ([]int8, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -145,7 +145,7 @@ func (p *BinlogFile) ReadInt8() ([]int8, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -155,18 +155,18 @@ func (p *BinlogFile) ReadInt8() ([]int8, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read int8 data", zap.Error(err))
return nil, fmt.Errorf("failed to read int8 data, error: %w", err)
}
@ -180,7 +180,7 @@ func (p *BinlogFile) ReadInt8() ([]int8, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadInt16() ([]int16, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -188,7 +188,7 @@ func (p *BinlogFile) ReadInt16() ([]int16, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -198,18 +198,18 @@ func (p *BinlogFile) ReadInt16() ([]int16, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read int16 data", zap.Error(err))
return nil, fmt.Errorf("failed to read int16 data, error: %w", err)
}
@ -223,7 +223,7 @@ func (p *BinlogFile) ReadInt16() ([]int16, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadInt32() ([]int32, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -231,7 +231,7 @@ func (p *BinlogFile) ReadInt32() ([]int32, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -241,18 +241,18 @@ func (p *BinlogFile) ReadInt32() ([]int32, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read int32 data", zap.Error(err))
return nil, fmt.Errorf("failed to read int32 data, error: %w", err)
}
@ -266,7 +266,7 @@ func (p *BinlogFile) ReadInt32() ([]int32, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadInt64() ([]int64, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -274,7 +274,7 @@ func (p *BinlogFile) ReadInt64() ([]int64, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -284,18 +284,18 @@ func (p *BinlogFile) ReadInt64() ([]int64, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read int64 data", zap.Error(err))
return nil, fmt.Errorf("failed to read int64 data, error: %w", err)
}
@ -309,7 +309,7 @@ func (p *BinlogFile) ReadInt64() ([]int64, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadFloat() ([]float32, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -317,7 +317,7 @@ func (p *BinlogFile) ReadFloat() ([]float32, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -327,18 +327,18 @@ func (p *BinlogFile) ReadFloat() ([]float32, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read float data", zap.Error(err))
return nil, fmt.Errorf("failed to read float data, error: %w", err)
}
@ -352,7 +352,7 @@ func (p *BinlogFile) ReadFloat() ([]float32, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadDouble() ([]float64, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -360,7 +360,7 @@ func (p *BinlogFile) ReadDouble() ([]float64, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -370,18 +370,18 @@ func (p *BinlogFile) ReadDouble() ([]float64, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read double data", zap.Error(err))
return nil, fmt.Errorf("failed to read double data, error: %w", err)
}
@ -395,7 +395,7 @@ func (p *BinlogFile) ReadDouble() ([]float64, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadVarchar() ([]string, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -403,7 +403,7 @@ func (p *BinlogFile) ReadVarchar() ([]string, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -414,18 +414,18 @@ func (p *BinlogFile) ReadVarchar() ([]string, error) {
// special case: delete event data type is varchar
if event.TypeCode != storage.InsertEventType && event.TypeCode != storage.DeleteEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read varchar data", zap.Error(err))
return nil, fmt.Errorf("failed to read varchar data, error: %w", err)
}
@ -439,7 +439,7 @@ func (p *BinlogFile) ReadVarchar() ([]string, error) {
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
func (p *BinlogFile) ReadJSON() ([][]byte, error) {
if p.reader == nil {
log.Error("Binlog file: binlog reader not yet initialized")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, errors.New("binlog reader not yet initialized")
}
@ -447,7 +447,7 @@ func (p *BinlogFile) ReadJSON() ([][]byte, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -457,18 +457,18 @@ func (p *BinlogFile) ReadJSON() ([][]byte, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("Binlog file: binlog file is not insert log")
return nil, errors.New("binlog file is not insert log")
}
if p.DataType() != schemapb.DataType_JSON {
log.Error("Binlog file: binlog data type is not JSON")
log.Warn("Binlog file: binlog data type is not JSON")
return nil, errors.New("binlog data type is not JSON")
}
data, err := event.PayloadReaderInterface.GetJSONFromPayload()
if err != nil {
log.Error("Binlog file: failed to read JSON data", zap.Error(err))
log.Warn("Binlog file: failed to read JSON data", zap.Error(err))
return nil, fmt.Errorf("failed to read JSON data, error: %w", err)
}
@ -483,7 +483,7 @@ func (p *BinlogFile) ReadJSON() ([][]byte, error) {
// 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")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, 0, errors.New("binlog reader not yet initialized")
}
@ -492,7 +492,7 @@ func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, 0, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -502,18 +502,18 @@ func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read binary vector data", zap.Error(err))
return nil, 0, fmt.Errorf("failed to read binary vector data, error: %w", err)
}
@ -529,7 +529,7 @@ func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) {
// 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")
log.Warn("Binlog file: binlog reader not yet initialized")
return nil, 0, errors.New("binlog reader not yet initialized")
}
@ -538,7 +538,7 @@ func (p *BinlogFile) ReadFloatVector() ([]float32, int, error) {
for {
event, err := p.reader.NextEventReader()
if err != nil {
log.Error("Binlog file: failed to iterate events reader", zap.Error(err))
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
return nil, 0, fmt.Errorf("failed to iterate events reader, error: %w", err)
}
@ -548,18 +548,18 @@ func (p *BinlogFile) ReadFloatVector() ([]float32, int, error) {
}
if event.TypeCode != storage.InsertEventType {
log.Error("Binlog file: binlog file is not insert log")
log.Warn("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")
log.Warn("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))
log.Warn("Binlog file: failed to read float vector data", zap.Error(err))
return nil, 0, fmt.Errorf("failed to read float vector data, error: %w", err)
}

View File

@ -159,7 +159,7 @@ func createBinlogBuf(t *testing.T, dataType schemapb.DataType, data interface{})
return buf
}
func Test_NewBinlogFile(t *testing.T) {
func Test_BinlogFileNew(t *testing.T) {
// nil chunkManager
file, err := NewBinlogFile(nil)
assert.Error(t, err)
@ -304,6 +304,12 @@ func Test_BinlogFileBool(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadBool()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -355,6 +361,12 @@ func Test_BinlogFileInt8(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadInt8()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -407,6 +419,12 @@ func Test_BinlogFileInt16(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadInt16()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -458,6 +476,12 @@ func Test_BinlogFileInt32(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadInt32()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -509,6 +533,12 @@ func Test_BinlogFileInt64(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadInt64()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -560,6 +590,12 @@ func Test_BinlogFileFloat(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadFloat()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -611,6 +647,12 @@ func Test_BinlogFileDouble(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadDouble()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -651,6 +693,12 @@ func Test_BinlogFileVarchar(t *testing.T) {
assert.Zero(t, len(d))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadVarchar()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -703,6 +751,12 @@ func Test_BinlogFileJSON(t *testing.T) {
assert.Zero(t, len(data))
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, err = binlogFile.ReadJSON()
assert.Zero(t, len(data))
assert.Error(t, err)
binlogFile.Close()
}
@ -764,6 +818,13 @@ func Test_BinlogFileBinaryVector(t *testing.T) {
assert.Zero(t, d)
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, d, err = binlogFile.ReadBinaryVector()
assert.Zero(t, len(data))
assert.Zero(t, d)
assert.Error(t, err)
binlogFile.Close()
}
@ -824,5 +885,12 @@ func Test_BinlogFileFloatVector(t *testing.T) {
assert.Zero(t, d)
assert.Error(t, err)
// failed to iterate events reader
binlogFile.reader.Close()
data, d, err = binlogFile.ReadFloatVector()
assert.Zero(t, len(data))
assert.Zero(t, d)
assert.Error(t, err)
binlogFile.Close()
}

View File

@ -26,20 +26,19 @@ import (
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/log"
"go.uber.org/zap"
)
type BinlogParser struct {
ctx context.Context // for canceling parse process
collectionSchema *schemapb.CollectionSchema // collection schema
shardNum int32 // sharding number of the collection
blockSize int64 // maximum size of a read block(unit:byte)
chunkManager storage.ChunkManager // storage interfaces to browse/read the files
callFlushFunc ImportFlushFunc // call back function to flush segment
updateProgressFunc func(percent int64) // update working progress percent value
ctx context.Context // for canceling parse process
collectionInfo *CollectionInfo // collection details including schema
shardNum int32 // sharding number of the collection
blockSize int64 // maximum size of a read block(unit:byte)
chunkManager storage.ChunkManager // storage interfaces to browse/read the files
callFlushFunc ImportFlushFunc // call back function to flush segment
updateProgressFunc func(percent int64) // update working progress percent value
// 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
@ -55,44 +54,43 @@ type BinlogParser struct {
}
func NewBinlogParser(ctx context.Context,
collectionSchema *schemapb.CollectionSchema,
shardNum int32,
collectionInfo *CollectionInfo,
blockSize int64,
chunkManager storage.ChunkManager,
flushFunc ImportFlushFunc,
updateProgressFunc func(percent int64),
tsStartPoint uint64,
tsEndPoint uint64) (*BinlogParser, error) {
if collectionSchema == nil {
log.Error("Binlog parser: collection schema is nil")
if collectionInfo == nil {
log.Warn("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")
log.Warn("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")
log.Warn("Binlog parser: flush function is nil")
return nil, errors.New("flush function is nil")
}
if tsStartPoint > tsEndPoint {
log.Error("Binlog parser: the tsStartPoint should be less than tsEndPoint",
log.Warn("Binlog parser: the tsStartPoint should be less than tsEndPoint",
zap.Uint64("tsStartPoint", tsStartPoint), zap.Uint64("tsEndPoint", tsEndPoint))
return nil, fmt.Errorf("Binlog parser: the tsStartPoint %d should be less than tsEndPoint %d", tsStartPoint, tsEndPoint)
}
v := &BinlogParser{
ctx: ctx,
collectionSchema: collectionSchema,
shardNum: shardNum,
blockSize: blockSize,
chunkManager: chunkManager,
callFlushFunc: flushFunc,
tsStartPoint: tsStartPoint,
tsEndPoint: tsEndPoint,
ctx: ctx,
collectionInfo: collectionInfo,
blockSize: blockSize,
chunkManager: chunkManager,
callFlushFunc: flushFunc,
updateProgressFunc: updateProgressFunc,
tsStartPoint: tsStartPoint,
tsEndPoint: tsEndPoint,
}
return v, nil
@ -121,7 +119,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo
// 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))
log.Warn("Binlog parser: list insert logs error", zap.Error(err))
return nil, fmt.Errorf("failed to list insert logs with root path %s, error: %w", insertlogRoot, err)
}
@ -139,7 +137,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo
fieldStrID := path.Base(fieldPath)
fieldID, err := strconv.ParseInt(fieldStrID, 10, 64)
if err != nil {
log.Error("Binlog parser: failed to parse field id", zap.String("fieldPath", fieldPath), zap.Error(err))
log.Warn("Binlog parser: failed to parse field id", zap.String("fieldPath", fieldPath), zap.Error(err))
return nil, fmt.Errorf("failed to parse field id from insert log path %s, error: %w", insertlog, err)
}
@ -147,7 +145,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo
segmentStrID := path.Base(segmentPath)
segmentID, err := strconv.ParseInt(segmentStrID, 10, 64)
if err != nil {
log.Error("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err))
log.Warn("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err))
return nil, fmt.Errorf("failed to parse segment id from insert log path %s, error: %w", insertlog, err)
}
@ -186,7 +184,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo
// TODO add context
deltalogs, _, err := p.chunkManager.ListWithPrefix(context.TODO(), deltalogRoot, true)
if err != nil {
log.Error("Binlog parser: failed to list delta logs", zap.Error(err))
log.Warn("Binlog parser: failed to list delta logs", zap.Error(err))
return nil, fmt.Errorf("failed to list delta logs, error: %w", err)
}
@ -197,7 +195,7 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo
segmentStrID := path.Base(segmentPath)
segmentID, err := strconv.ParseInt(segmentStrID, 10, 64)
if err != nil {
log.Error("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err))
log.Warn("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err))
return nil, fmt.Errorf("failed to parse segment id from delta log path %s, error: %w", deltalog, err)
}
@ -221,14 +219,14 @@ func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoo
func (p *BinlogParser) parseSegmentFiles(segmentHolder *SegmentFilesHolder) error {
if segmentHolder == nil {
log.Error("Binlog parser: segment files holder is nil")
log.Warn("Binlog parser: segment files holder is nil")
return errors.New("segment files holder is nil")
}
adapter, err := NewBinlogAdapter(p.ctx, p.collectionSchema, p.shardNum, p.blockSize,
adapter, err := NewBinlogAdapter(p.ctx, p.collectionInfo, p.blockSize,
MaxTotalSizeInMemory, p.chunkManager, p.callFlushFunc, p.tsStartPoint, p.tsEndPoint)
if err != nil {
log.Error("Binlog parser: failed to create binlog adapter", zap.Error(err))
log.Warn("Binlog parser: failed to create binlog adapter", zap.Error(err))
return fmt.Errorf("failed to create binlog adapter, error: %w", err)
}
@ -240,7 +238,7 @@ func (p *BinlogParser) parseSegmentFiles(segmentHolder *SegmentFilesHolder) erro
// 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, partition binlog path and delta path are required")
log.Warn("Binlog parser: illegal paths for binlog import, partition binlog path and delta path are required")
return errors.New("illegal paths for binlog import, partition binlog path and delta path are required")
}
@ -272,7 +270,6 @@ func (p *BinlogParser) Parse(filePaths []string) error {
if err != nil {
return err
}
updateProgress(i + 1)
// trigger gb after each segment finished

View File

@ -25,38 +25,40 @@ import (
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/stretchr/testify/assert"
)
func Test_NewBinlogParser(t *testing.T) {
func Test_BinlogParserNew(t *testing.T) {
ctx := context.Background()
// nil schema
parser, err := NewBinlogParser(ctx, nil, 2, 1024, nil, nil, nil, 0, math.MaxUint64)
parser, err := NewBinlogParser(ctx, nil, 1024, nil, nil, nil, 0, math.MaxUint64)
assert.Nil(t, parser)
assert.Error(t, err)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
// nil chunkmanager
parser, err = NewBinlogParser(ctx, sampleSchema(), 2, 1024, nil, nil, nil, 0, math.MaxUint64)
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, nil, nil, nil, 0, math.MaxUint64)
assert.Nil(t, parser)
assert.Error(t, err)
// nil flushfunc
parser, err = NewBinlogParser(ctx, sampleSchema(), 2, 1024, &MockChunkManager{}, nil, nil, 0, math.MaxUint64)
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, nil, nil, 0, math.MaxUint64)
assert.Nil(t, parser)
assert.Error(t, err)
// succeed
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
parser, err = NewBinlogParser(ctx, sampleSchema(), 2, 1024, &MockChunkManager{}, flushFunc, nil, 0, math.MaxUint64)
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, flushFunc, nil, 0, math.MaxUint64)
assert.NotNil(t, parser)
assert.NoError(t, err)
// tsStartPoint larger than tsEndPoint
parser, err = NewBinlogParser(ctx, sampleSchema(), 2, 1024, &MockChunkManager{}, flushFunc, nil, 2, 1)
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, flushFunc, nil, 2, 1)
assert.Nil(t, parser)
assert.Error(t, err)
}
@ -64,7 +66,7 @@ func Test_NewBinlogParser(t *testing.T) {
func Test_BinlogParserConstructHolders(t *testing.T) {
ctx := context.Background()
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
@ -128,7 +130,10 @@ func Test_BinlogParserConstructHolders(t *testing.T) {
"backup/bak1/data/delta_log/435978159196147009/435978159196147010/435978159261483009/434574382554415105",
}
parser, err := NewBinlogParser(ctx, sampleSchema(), 2, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
assert.NotNil(t, parser)
assert.NoError(t, err)
@ -179,7 +184,7 @@ func Test_BinlogParserConstructHolders(t *testing.T) {
func Test_BinlogParserConstructHoldersFailed(t *testing.T) {
ctx := context.Background()
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
@ -188,7 +193,10 @@ func Test_BinlogParserConstructHoldersFailed(t *testing.T) {
listResult: make(map[string][]string),
}
parser, err := NewBinlogParser(ctx, sampleSchema(), 2, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
assert.NotNil(t, parser)
assert.NoError(t, err)
@ -230,18 +238,21 @@ func Test_BinlogParserConstructHoldersFailed(t *testing.T) {
func Test_BinlogParserParseFilesFailed(t *testing.T) {
ctx := context.Background()
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
parser, err := NewBinlogParser(ctx, sampleSchema(), 2, 1024, &MockChunkManager{}, flushFunc, nil, 0, math.MaxUint64)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, flushFunc, nil, 0, math.MaxUint64)
assert.NotNil(t, parser)
assert.NoError(t, err)
err = parser.parseSegmentFiles(nil)
assert.Error(t, err)
parser.collectionSchema = nil
parser.collectionInfo = nil
err = parser.parseSegmentFiles(&SegmentFilesHolder{})
assert.Error(t, err)
}
@ -249,7 +260,7 @@ func Test_BinlogParserParseFilesFailed(t *testing.T) {
func Test_BinlogParserParse(t *testing.T) {
ctx := context.Background()
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
@ -268,11 +279,10 @@ func Test_BinlogParserParse(t *testing.T) {
},
},
}
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
updateProgress := func(percent int64) {
assert.Greater(t, percent, int64(0))
}
parser, err := NewBinlogParser(ctx, schema, 2, 1024, chunkManager, flushFunc, updateProgress, 0, math.MaxUint64)
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
assert.NotNil(t, parser)
assert.NoError(t, err)
@ -302,18 +312,69 @@ func Test_BinlogParserParse(t *testing.T) {
// 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",
"123/0/a",
"123/1/a",
"123/101/a",
}
err = parser.Parse(paths)
assert.Error(t, err)
// progress
rowCount := 100
fieldsData := createFieldsData(sampleSchema(), rowCount)
chunkManager.listResult["deltaPath"] = []string{}
chunkManager.listResult["insertPath"] = []string{
"123/0/a",
"123/1/a",
"123/102/a",
"123/103/a",
"123/104/a",
"123/105/a",
"123/106/a",
"123/107/a",
"123/108/a",
"123/109/a",
"123/110/a",
"123/111/a",
"123/112/a",
}
chunkManager.readBuf = map[string][]byte{
"123/0/a": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)),
"123/1/a": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)),
"123/102/a": createBinlogBuf(t, schemapb.DataType_Bool, fieldsData[102].([]bool)),
"123/103/a": createBinlogBuf(t, schemapb.DataType_Int8, fieldsData[103].([]int8)),
"123/104/a": createBinlogBuf(t, schemapb.DataType_Int16, fieldsData[104].([]int16)),
"123/105/a": createBinlogBuf(t, schemapb.DataType_Int32, fieldsData[105].([]int32)),
"123/106/a": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)), // this is primary key
"123/107/a": createBinlogBuf(t, schemapb.DataType_Float, fieldsData[107].([]float32)),
"123/108/a": createBinlogBuf(t, schemapb.DataType_Double, fieldsData[108].([]float64)),
"123/109/a": createBinlogBuf(t, schemapb.DataType_VarChar, fieldsData[109].([]string)),
"123/110/a": createBinlogBuf(t, schemapb.DataType_BinaryVector, fieldsData[110].([][]byte)),
"123/111/a": createBinlogBuf(t, schemapb.DataType_FloatVector, fieldsData[111].([][]float32)),
"123/112/a": createBinlogBuf(t, schemapb.DataType_JSON, fieldsData[112].([][]byte)),
}
callTime := 0
updateProgress := func(percent int64) {
assert.GreaterOrEqual(t, percent, int64(0))
assert.LessOrEqual(t, percent, int64(100))
callTime++
}
collectionInfo, err = NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, updateProgress, 0, math.MaxUint64)
assert.NotNil(t, parser)
assert.NoError(t, err)
err = parser.Parse(paths)
assert.NoError(t, err)
assert.Equal(t, 1, callTime)
}
func Test_BinlogParserSkipFlagFile(t *testing.T) {
ctx := context.Background()
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
@ -322,7 +383,10 @@ func Test_BinlogParserSkipFlagFile(t *testing.T) {
listResult: make(map[string][]string),
}
parser, err := NewBinlogParser(ctx, sampleSchema(), 2, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
assert.NotNil(t, parser)
assert.NoError(t, err)

View File

@ -0,0 +1,130 @@
// 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 (
"fmt"
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type CollectionInfo struct {
Schema *schemapb.CollectionSchema
ShardNum int32
PartitionIDs []int64 // target partitions of bulkinsert, one partition for non-partition-key collection, or all partiitons for partition-key collection
PrimaryKey *schemapb.FieldSchema
PartitionKey *schemapb.FieldSchema
DynamicField *schemapb.FieldSchema
Name2FieldID map[string]int64 // this member is for Numpy file name validation and JSON row validation
}
func DeduceTargetPartitions(partitions map[string]int64, collectionSchema *schemapb.CollectionSchema, defaultPartition int64) ([]int64, error) {
// if no partition key, rutrn the default partition ID as target partition
_, err := typeutil.GetPartitionKeyFieldSchema(collectionSchema)
if err != nil {
return []int64{defaultPartition}, nil
}
_, partitionIDs, err := typeutil.RearrangePartitionsForPartitionKey(partitions)
if err != nil {
return nil, err
}
return partitionIDs, nil
}
func NewCollectionInfo(collectionSchema *schemapb.CollectionSchema,
shardNum int32,
partitionIDs []int64) (*CollectionInfo, error) {
if shardNum <= 0 {
return nil, fmt.Errorf("illegal shard number %d", shardNum)
}
if len(partitionIDs) == 0 {
return nil, errors.New("partition list is empty")
}
info := &CollectionInfo{
ShardNum: shardNum,
PartitionIDs: partitionIDs,
}
err := info.resetSchema(collectionSchema)
if err != nil {
return nil, err
}
return info, nil
}
func (c *CollectionInfo) resetSchema(collectionSchema *schemapb.CollectionSchema) error {
if collectionSchema == nil {
return errors.New("collection schema is null")
}
fields := make([]*schemapb.FieldSchema, 0)
name2FieldID := make(map[string]int64)
var primaryKey *schemapb.FieldSchema
var dynamicField *schemapb.FieldSchema
var partitionKey *schemapb.FieldSchema
for i := 0; i < len(collectionSchema.Fields); i++ {
schema := collectionSchema.Fields[i]
// RowIDField and TimeStampField is internal field, no need to parse
if schema.GetName() == common.RowIDFieldName || schema.GetName() == common.TimeStampFieldName {
continue
}
fields = append(fields, schema)
name2FieldID[schema.GetName()] = schema.GetFieldID()
if schema.GetIsPrimaryKey() {
primaryKey = schema
} else if schema.GetIsDynamic() {
dynamicField = schema
} else if schema.GetIsPartitionKey() {
partitionKey = schema
}
}
if primaryKey == nil {
return errors.New("collection schema has no primary key")
}
if partitionKey == nil && len(c.PartitionIDs) != 1 {
return errors.New("only allow one partition when there is no partition key")
}
c.Schema = &schemapb.CollectionSchema{
Name: collectionSchema.GetName(),
Description: collectionSchema.GetDescription(),
AutoID: collectionSchema.GetAutoID(),
Fields: fields,
EnableDynamicField: collectionSchema.GetEnableDynamicField(),
}
c.PrimaryKey = primaryKey
c.DynamicField = dynamicField
c.PartitionKey = partitionKey
c.Name2FieldID = name2FieldID
return nil
}

View File

@ -0,0 +1,142 @@
// 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 (
"testing"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/stretchr/testify/assert"
)
func Test_DeduceTargetPartitions(t *testing.T) {
schema := sampleSchema()
partitions := map[string]int64{
"part_0": 100,
"part_1": 200,
}
partitionIDs, err := DeduceTargetPartitions(partitions, schema, int64(1))
assert.NoError(t, err)
assert.Equal(t, 1, len(partitionIDs))
assert.Equal(t, int64(1), partitionIDs[0])
schema.Fields[7].IsPartitionKey = true
partitionIDs, err = DeduceTargetPartitions(partitions, schema, int64(1))
assert.NoError(t, err)
assert.Equal(t, len(partitions), len(partitionIDs))
partitions = map[string]int64{
"part_a": 100,
}
partitionIDs, err = DeduceTargetPartitions(partitions, schema, int64(1))
assert.Error(t, err)
assert.Nil(t, partitionIDs)
}
func Test_CollectionInfoNew(t *testing.T) {
t.Run("succeed", func(t *testing.T) {
info, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
assert.NotNil(t, info)
assert.Greater(t, len(info.Name2FieldID), 0)
assert.Nil(t, info.PartitionKey)
assert.Nil(t, info.DynamicField)
assert.NotNil(t, info.PrimaryKey)
assert.Equal(t, int32(2), info.ShardNum)
assert.Equal(t, 1, len(info.PartitionIDs))
// has partition key, has dynamic field
schema := &schemapb.CollectionSchema{
Name: "schema",
Description: "schema",
Fields: []*schemapb.FieldSchema{
{
FieldID: 0,
Name: "RowID",
DataType: schemapb.DataType_Int64,
},
{
FieldID: 100,
Name: "ID",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
},
{
FieldID: 101,
Name: "PartitionKey",
IsPartitionKey: true,
DataType: schemapb.DataType_VarChar,
},
{
FieldID: 102,
Name: "$meta",
IsDynamic: true,
DataType: schemapb.DataType_JSON,
},
},
}
info, err = NewCollectionInfo(schema, 2, []int64{1, 2})
assert.NoError(t, err)
assert.NotNil(t, info)
assert.NotNil(t, info.PrimaryKey)
assert.NotNil(t, int64(100), info.PrimaryKey.GetFieldID())
assert.False(t, info.PrimaryKey.GetAutoID())
assert.NotNil(t, info.DynamicField)
assert.Equal(t, int64(102), info.DynamicField.GetFieldID())
assert.NotNil(t, info.PartitionKey)
assert.Equal(t, int64(101), info.PartitionKey.GetFieldID())
})
t.Run("error cases", func(t *testing.T) {
schema := sampleSchema()
// shard number is 0
info, err := NewCollectionInfo(schema, 0, []int64{1})
assert.Error(t, err)
assert.Nil(t, info)
// partiton ID list is empty
info, err = NewCollectionInfo(schema, 2, []int64{})
assert.Error(t, err)
assert.Nil(t, info)
// only allow one partition when there is no partition key
info, err = NewCollectionInfo(schema, 2, []int64{1, 2})
assert.Error(t, err)
assert.Nil(t, info)
// collection schema is nil
info, err = NewCollectionInfo(nil, 2, []int64{1})
assert.Error(t, err)
assert.Nil(t, info)
// no primary key
schema = &schemapb.CollectionSchema{
Name: "schema",
Description: "schema",
AutoID: true,
Fields: make([]*schemapb.FieldSchema, 0),
}
info, err = NewCollectionInfo(schema, 2, []int64{1})
assert.Error(t, err)
assert.Nil(t, info)
// partition key is nil
info, err = NewCollectionInfo(schema, 2, []int64{1, 2})
assert.Error(t, err)
assert.Nil(t, info)
})
}

View File

@ -22,10 +22,10 @@ import (
"strings"
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
)
// Extra option keys to pass through import API

View File

@ -24,7 +24,7 @@ import (
"github.com/stretchr/testify/assert"
)
func TestValidateOptions(t *testing.T) {
func Test_ValidateOptions(t *testing.T) {
assert.NoError(t, ValidateOptions([]*commonpb.KeyValuePair{}))
assert.NoError(t, ValidateOptions([]*commonpb.KeyValuePair{
@ -51,9 +51,13 @@ func TestValidateOptions(t *testing.T) {
{Key: "start_ts", Value: "3.14"},
{Key: "end_ts", Value: "1666007457"},
}))
assert.Error(t, ValidateOptions([]*commonpb.KeyValuePair{
{Key: "start_ts", Value: "1666007457"},
{Key: "end_ts", Value: "3.14"},
}))
}
func TestParseTSFromOptions(t *testing.T) {
func Test_ParseTSFromOptions(t *testing.T) {
var tsStart uint64
var tsEnd uint64
var err error
@ -88,7 +92,7 @@ func TestParseTSFromOptions(t *testing.T) {
assert.Error(t, err)
}
func TestIsBackup(t *testing.T) {
func Test_IsBackup(t *testing.T) {
isBackup := IsBackup([]*commonpb.KeyValuePair{
{Key: "backup", Value: "true"},
})

View File

@ -37,6 +37,9 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type BlockData map[storage.FieldID]storage.FieldData // a map of field ID to field data
type ShardData map[int64]BlockData // a map of partition ID to block data
func isCanceled(ctx context.Context) bool {
// canceled?
select {
@ -48,11 +51,11 @@ func isCanceled(ctx context.Context) bool {
return false
}
func initSegmentData(collectionSchema *schemapb.CollectionSchema) map[storage.FieldID]storage.FieldData {
segmentData := make(map[storage.FieldID]storage.FieldData)
func initBlockData(collectionSchema *schemapb.CollectionSchema) BlockData {
blockData := make(BlockData)
// rowID field is a hidden field with fieldID=0, it is always auto-generated by IDAllocator
// if primary key is int64 and autoID=true, primary key field is equal to rowID field
segmentData[common.RowIDField] = &storage.Int64FieldData{
blockData[common.RowIDField] = &storage.Int64FieldData{
Data: make([]int64, 0),
}
@ -60,60 +63,73 @@ func initSegmentData(collectionSchema *schemapb.CollectionSchema) map[storage.Fi
schema := collectionSchema.Fields[i]
switch schema.DataType {
case schemapb.DataType_Bool:
segmentData[schema.GetFieldID()] = &storage.BoolFieldData{
blockData[schema.GetFieldID()] = &storage.BoolFieldData{
Data: make([]bool, 0),
}
case schemapb.DataType_Float:
segmentData[schema.GetFieldID()] = &storage.FloatFieldData{
blockData[schema.GetFieldID()] = &storage.FloatFieldData{
Data: make([]float32, 0),
}
case schemapb.DataType_Double:
segmentData[schema.GetFieldID()] = &storage.DoubleFieldData{
blockData[schema.GetFieldID()] = &storage.DoubleFieldData{
Data: make([]float64, 0),
}
case schemapb.DataType_Int8:
segmentData[schema.GetFieldID()] = &storage.Int8FieldData{
blockData[schema.GetFieldID()] = &storage.Int8FieldData{
Data: make([]int8, 0),
}
case schemapb.DataType_Int16:
segmentData[schema.GetFieldID()] = &storage.Int16FieldData{
blockData[schema.GetFieldID()] = &storage.Int16FieldData{
Data: make([]int16, 0),
}
case schemapb.DataType_Int32:
segmentData[schema.GetFieldID()] = &storage.Int32FieldData{
blockData[schema.GetFieldID()] = &storage.Int32FieldData{
Data: make([]int32, 0),
}
case schemapb.DataType_Int64:
segmentData[schema.GetFieldID()] = &storage.Int64FieldData{
blockData[schema.GetFieldID()] = &storage.Int64FieldData{
Data: make([]int64, 0),
}
case schemapb.DataType_BinaryVector:
dim, _ := getFieldDimension(schema)
segmentData[schema.GetFieldID()] = &storage.BinaryVectorFieldData{
blockData[schema.GetFieldID()] = &storage.BinaryVectorFieldData{
Data: make([]byte, 0),
Dim: dim,
}
case schemapb.DataType_FloatVector:
dim, _ := getFieldDimension(schema)
segmentData[schema.GetFieldID()] = &storage.FloatVectorFieldData{
blockData[schema.GetFieldID()] = &storage.FloatVectorFieldData{
Data: make([]float32, 0),
Dim: dim,
}
case schemapb.DataType_String, schemapb.DataType_VarChar:
segmentData[schema.GetFieldID()] = &storage.StringFieldData{
blockData[schema.GetFieldID()] = &storage.StringFieldData{
Data: make([]string, 0),
}
case schemapb.DataType_JSON:
segmentData[schema.GetFieldID()] = &storage.JSONFieldData{
blockData[schema.GetFieldID()] = &storage.JSONFieldData{
Data: make([][]byte, 0),
}
default:
log.Error("Import util: unsupported data type", zap.String("DataType", getTypeName(schema.DataType)))
log.Warn("Import util: unsupported data type", zap.String("DataType", getTypeName(schema.DataType)))
return nil
}
}
return segmentData
return blockData
}
func initShardData(collectionSchema *schemapb.CollectionSchema, partitionIDs []int64) ShardData {
shardData := make(ShardData)
for i := 0; i < len(partitionIDs); i++ {
blockData := initBlockData(collectionSchema)
if blockData == nil {
return nil
}
shardData[partitionIDs[i]] = blockData
}
return shardData
}
func parseFloat(s string, bitsize int, fieldName string) (float64, error) {
@ -130,6 +146,17 @@ func parseFloat(s string, bitsize int, fieldName string) (float64, error) {
return value, nil
}
// Validator is field value validator
type Validator struct {
convertFunc func(obj interface{}, field storage.FieldData) error // convert data function
primaryKey bool // true for primary key
autoID bool // only for primary key field
isString bool // for string field
dimension int // only for vector field
fieldName string // field name
fieldID int64 //field ID
}
// initValidators constructs valiator methods and data conversion methods
func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[storage.FieldID]*Validator) error {
if collectionSchema == nil {
@ -143,6 +170,7 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[
validators[schema.GetFieldID()].primaryKey = schema.GetIsPrimaryKey()
validators[schema.GetFieldID()].autoID = schema.GetAutoID()
validators[schema.GetFieldID()].fieldName = schema.GetName()
validators[schema.GetFieldID()].fieldID = schema.GetFieldID()
validators[schema.GetFieldID()].isString = false
switch schema.DataType {
@ -337,7 +365,7 @@ func initValidators(collectionSchema *schemapb.CollectionSchema, validators map[
return nil
}
func printFieldsDataInfo(fieldsData map[storage.FieldID]storage.FieldData, msg string, files []string) {
func printFieldsDataInfo(fieldsData BlockData, msg string, files []string) {
stats := make([]zapcore.Field, 0)
for k, v := range fieldsData {
stats = append(stats, zap.Int(strconv.FormatInt(k, 10), v.RowNum()))
@ -381,7 +409,7 @@ func triggerGC() {
}
// if user didn't provide dynamic data, fill the dynamic field by "{}"
func fillDynamicData(blockData map[storage.FieldID]storage.FieldData, collectionSchema *schemapb.CollectionSchema) error {
func fillDynamicData(blockData BlockData, collectionSchema *schemapb.CollectionSchema) error {
if !collectionSchema.GetEnableDynamicField() {
return nil
}
@ -435,7 +463,7 @@ func fillDynamicData(blockData map[storage.FieldID]storage.FieldData, collection
// 1. if accumulate data of a block exceed blockSize, call callFlushFunc to generate new binlog file
// 2. if total accumulate data exceed maxTotalSize, call callFlushFUnc to flush the biggest block
func tryFlushBlocks(ctx context.Context,
blocksData []map[storage.FieldID]storage.FieldData,
shardsData []ShardData,
collectionSchema *schemapb.CollectionSchema,
callFlushFunc ImportFlushFunc,
blockSize int64,
@ -445,89 +473,96 @@ func tryFlushBlocks(ctx context.Context,
totalSize := 0
biggestSize := 0
biggestItem := -1
biggestPartition := int64(-1)
// 1. if accumulate data of a block exceed blockSize, call callFlushFunc to generate new binlog file
for i := 0; i < len(blocksData); i++ {
for i := 0; i < len(shardsData); i++ {
// outside context might be canceled(service stop, or future enhancement for canceling import task)
if isCanceled(ctx) {
log.Error("Import util: import task was canceled")
log.Warn("Import util: import task was canceled")
return errors.New("import task was canceled")
}
blockData := blocksData[i]
err := fillDynamicData(blockData, collectionSchema)
if err != nil {
log.Error("Import util: failed to fill dynamic field", zap.Error(err))
return fmt.Errorf("failed to fill dynamic field, error: %w", err)
}
// Note: even rowCount is 0, the size is still non-zero
size := 0
rowCount := 0
for _, fieldData := range blockData {
size += fieldData.GetMemorySize()
rowCount = fieldData.RowNum()
}
// force to flush, called at the end of Read()
if force && rowCount > 0 {
printFieldsDataInfo(blockData, "import util: prepare to force flush a block", nil)
err := callFlushFunc(blockData, i)
shardData := shardsData[i]
for partitionID, blockData := range shardData {
err := fillDynamicData(blockData, collectionSchema)
if err != nil {
log.Error("Import util: failed to force flush block data", zap.Int("shardID", i), zap.Error(err))
return fmt.Errorf("failed to force flush block data for shard id %d, error: %w", i, err)
log.Warn("Import util: failed to fill dynamic field", zap.Error(err))
return fmt.Errorf("failed to fill dynamic field, error: %w", err)
}
log.Info("Import util: force flush", zap.Int("rowCount", rowCount), zap.Int("size", size), zap.Int("shardID", i))
blocksData[i] = initSegmentData(collectionSchema)
if blocksData[i] == nil {
log.Error("Import util: failed to initialize FieldData list", zap.Int("shardID", i))
return fmt.Errorf("failed to initialize FieldData list for shard id %d", i)
// Note: even rowCount is 0, the size is still non-zero
size := 0
rowCount := 0
for _, fieldData := range blockData {
size += fieldData.GetMemorySize()
rowCount = fieldData.RowNum()
}
continue
}
// if segment size is larger than predefined blockSize, flush to create a new binlog file
// initialize a new FieldData list for next round batch read
if size > int(blockSize) && rowCount > 0 {
printFieldsDataInfo(blockData, "import util: prepare to flush block larger than blockSize", nil)
err := callFlushFunc(blockData, i)
if err != nil {
log.Error("Import util: failed to flush block data", zap.Int("shardID", i), zap.Error(err))
return fmt.Errorf("failed to flush block data for shard id %d, error: %w", i, err)
// force to flush, called at the end of Read()
if force && rowCount > 0 {
printFieldsDataInfo(blockData, "import util: prepare to force flush a block", nil)
err := callFlushFunc(blockData, i, partitionID)
if err != nil {
log.Warn("Import util: failed to force flush block data", zap.Int("shardID", i),
zap.Int64("partitionID", partitionID), zap.Error(err))
return fmt.Errorf("failed to force flush block data for shard id %d to partition %d, error: %w", i, partitionID, err)
}
log.Info("Import util: force flush", zap.Int("rowCount", rowCount), zap.Int("size", size),
zap.Int("shardID", i), zap.Int64("partitionID", partitionID))
shardData[partitionID] = initBlockData(collectionSchema)
if shardData[partitionID] == nil {
log.Warn("Import util: failed to initialize FieldData list", zap.Int("shardID", i), zap.Int64("partitionID", partitionID))
return fmt.Errorf("failed to initialize FieldData list for shard id %d to partition %d", i, partitionID)
}
continue
}
log.Info("Import util: block size exceed limit and flush", zap.Int("rowCount", rowCount),
zap.Int("size", size), zap.Int("shardID", i), zap.Int64("blockSize", blockSize))
blocksData[i] = initSegmentData(collectionSchema)
if blocksData[i] == nil {
log.Error("Import util: failed to initialize FieldData list", zap.Int("shardID", i))
return fmt.Errorf("failed to initialize FieldData list for shard id %d", i)
// if segment size is larger than predefined blockSize, flush to create a new binlog file
// initialize a new FieldData list for next round batch read
if size > int(blockSize) && rowCount > 0 {
printFieldsDataInfo(blockData, "import util: prepare to flush block larger than blockSize", nil)
err := callFlushFunc(blockData, i, partitionID)
if err != nil {
log.Warn("Import util: failed to flush block data", zap.Int("shardID", i),
zap.Int64("partitionID", partitionID), zap.Error(err))
return fmt.Errorf("failed to flush block data for shard id %d to partition %d, error: %w", i, partitionID, err)
}
log.Info("Import util: block size exceed limit and flush", zap.Int("rowCount", rowCount), zap.Int("size", size),
zap.Int("shardID", i), zap.Int64("partitionID", partitionID), zap.Int64("blockSize", blockSize))
shardData[partitionID] = initBlockData(collectionSchema)
if shardData[partitionID] == nil {
log.Warn("Import util: failed to initialize FieldData list", zap.Int("shardID", i), zap.Int64("partitionID", partitionID))
return fmt.Errorf("failed to initialize FieldData list for shard id %d to partition %d", i, partitionID)
}
continue
}
continue
}
// calculate the total size(ignore the flushed blocks)
// find out the biggest block for the step 2
totalSize += size
if size > biggestSize {
biggestSize = size
biggestItem = i
// calculate the total size(ignore the flushed blocks)
// find out the biggest block for the step 2
totalSize += size
if size > biggestSize {
biggestSize = size
biggestItem = i
biggestPartition = partitionID
}
}
}
// 2. if total accumulate data exceed maxTotalSize, call callFlushFUnc to flush the biggest block
if totalSize > int(maxTotalSize) && biggestItem >= 0 {
if totalSize > int(maxTotalSize) && biggestItem >= 0 && biggestPartition >= 0 {
// outside context might be canceled(service stop, or future enhancement for canceling import task)
if isCanceled(ctx) {
log.Error("Import util: import task was canceled")
log.Warn("Import util: import task was canceled")
return errors.New("import task was canceled")
}
blockData := blocksData[biggestItem]
blockData := shardsData[biggestItem][biggestPartition]
err := fillDynamicData(blockData, collectionSchema)
if err != nil {
log.Error("Import util: failed to fill dynamic field", zap.Error(err))
log.Warn("Import util: failed to fill dynamic field", zap.Error(err))
return fmt.Errorf("failed to fill dynamic field, error: %w", err)
}
@ -541,18 +576,21 @@ func tryFlushBlocks(ctx context.Context,
if rowCount > 0 {
printFieldsDataInfo(blockData, "import util: prepare to flush biggest block", nil)
err = callFlushFunc(blockData, biggestItem)
err = callFlushFunc(blockData, biggestItem, biggestPartition)
if err != nil {
log.Error("Import util: failed to flush biggest block data", zap.Int("shardID", biggestItem))
return fmt.Errorf("failed to flush biggest block data for shard id %d, error: %w", biggestItem, err)
log.Warn("Import util: failed to flush biggest block data", zap.Int("shardID", biggestItem),
zap.Int64("partitionID", biggestPartition))
return fmt.Errorf("failed to flush biggest block data for shard id %d to partition %d, error: %w",
biggestItem, biggestPartition, err)
}
log.Info("Import util: total size exceed limit and flush", zap.Int("rowCount", rowCount),
zap.Int("size", size), zap.Int("totalSize", totalSize), zap.Int("shardID", biggestItem))
blocksData[biggestItem] = initSegmentData(collectionSchema)
if blocksData[biggestItem] == nil {
log.Error("Import util: failed to initialize FieldData list", zap.Int("shardID", biggestItem))
return fmt.Errorf("failed to initialize FieldData list for shard id %d", biggestItem)
shardsData[biggestItem][biggestPartition] = initBlockData(collectionSchema)
if shardsData[biggestItem][biggestPartition] == nil {
log.Warn("Import util: failed to initialize FieldData list", zap.Int("shardID", biggestItem),
zap.Int64("partitionID", biggestPartition))
return fmt.Errorf("failed to initialize FieldData list for shard id %d to partition %d", biggestItem, biggestPartition)
}
}
}
@ -600,7 +638,7 @@ func pkToShard(pk interface{}, shardNum uint32) (uint32, error) {
} else {
intPK, ok := pk.(int64)
if !ok {
log.Error("Numpy parser: primary key field must be int64 or varchar")
log.Warn("Numpy parser: primary key field must be int64 or varchar")
return 0, fmt.Errorf("primary key field must be int64 or varchar")
}
hash, _ := typeutil.Hash32Int64(intPK)

View File

@ -18,7 +18,9 @@ package importutil
import (
"context"
"encoding/json"
"fmt"
"math"
"strconv"
"testing"
"github.com/cockroachdb/errors"
@ -229,6 +231,171 @@ func jsonNumber(value string) json.Number {
return json.Number(value)
}
func createFieldsData(collectionSchema *schemapb.CollectionSchema, rowCount int) map[storage.FieldID]interface{} {
fieldsData := make(map[storage.FieldID]interface{})
// internal fields
rowIDData := make([]int64, 0)
timestampData := make([]int64, 0)
for i := 0; i < rowCount; i++ {
rowIDData = append(rowIDData, int64(i))
timestampData = append(timestampData, baseTimestamp+int64(i))
}
fieldsData[0] = rowIDData
fieldsData[1] = timestampData
// user-defined fields
for i := 0; i < len(collectionSchema.Fields); i++ {
schema := collectionSchema.Fields[i]
switch schema.DataType {
case schemapb.DataType_Bool:
boolData := make([]bool, 0)
for i := 0; i < rowCount; i++ {
boolData = append(boolData, (i%3 != 0))
}
fieldsData[schema.GetFieldID()] = boolData
case schemapb.DataType_Float:
floatData := make([]float32, 0)
for i := 0; i < rowCount; i++ {
floatData = append(floatData, float32(i/2))
}
fieldsData[schema.GetFieldID()] = floatData
case schemapb.DataType_Double:
doubleData := make([]float64, 0)
for i := 0; i < rowCount; i++ {
doubleData = append(doubleData, float64(i/5))
}
fieldsData[schema.GetFieldID()] = doubleData
case schemapb.DataType_Int8:
int8Data := make([]int8, 0)
for i := 0; i < rowCount; i++ {
int8Data = append(int8Data, int8(i%256))
}
fieldsData[schema.GetFieldID()] = int8Data
case schemapb.DataType_Int16:
int16Data := make([]int16, 0)
for i := 0; i < rowCount; i++ {
int16Data = append(int16Data, int16(i%65536))
}
fieldsData[schema.GetFieldID()] = int16Data
case schemapb.DataType_Int32:
int32Data := make([]int32, 0)
for i := 0; i < rowCount; i++ {
int32Data = append(int32Data, int32(i%1000))
}
fieldsData[schema.GetFieldID()] = int32Data
case schemapb.DataType_Int64:
int64Data := make([]int64, 0)
for i := 0; i < rowCount; i++ {
int64Data = append(int64Data, int64(i))
}
fieldsData[schema.GetFieldID()] = int64Data
case schemapb.DataType_BinaryVector:
dim, _ := getFieldDimension(schema)
binVecData := make([][]byte, 0)
for i := 0; i < rowCount; i++ {
vec := make([]byte, 0)
for k := 0; k < dim/8; k++ {
vec = append(vec, byte(i%256))
}
binVecData = append(binVecData, vec)
}
fieldsData[schema.GetFieldID()] = binVecData
case schemapb.DataType_FloatVector:
dim, _ := getFieldDimension(schema)
floatVecData := make([][]float32, 0)
for i := 0; i < rowCount; i++ {
vec := make([]float32, 0)
for k := 0; k < dim; k++ {
vec = append(vec, float32((i+k)/5))
}
floatVecData = append(floatVecData, vec)
}
fieldsData[schema.GetFieldID()] = floatVecData
case schemapb.DataType_String, schemapb.DataType_VarChar:
varcharData := make([]string, 0)
for i := 0; i < rowCount; i++ {
varcharData = append(varcharData, "no."+strconv.Itoa(i))
}
fieldsData[schema.GetFieldID()] = varcharData
case schemapb.DataType_JSON:
jsonData := make([][]byte, 0)
for i := 0; i < rowCount; i++ {
jsonData = append(jsonData, []byte(fmt.Sprintf("{\"y\": %d}", i)))
}
fieldsData[schema.GetFieldID()] = jsonData
default:
return nil
}
}
return fieldsData
}
func createBlockData(collectionSchema *schemapb.CollectionSchema, fieldsData map[storage.FieldID]interface{}) BlockData {
blockData := initBlockData(collectionSchema)
if fieldsData != nil {
// internal field
blockData[common.RowIDField].(*storage.Int64FieldData).Data = append(blockData[common.RowIDField].(*storage.Int64FieldData).Data, fieldsData[common.RowIDField].([]int64)...)
// user custom fields
for i := 0; i < len(collectionSchema.Fields); i++ {
schema := collectionSchema.Fields[i]
fieldID := schema.GetFieldID()
switch schema.DataType {
case schemapb.DataType_Bool:
blockData[fieldID].(*storage.BoolFieldData).Data = append(blockData[fieldID].(*storage.BoolFieldData).Data, fieldsData[fieldID].([]bool)...)
case schemapb.DataType_Float:
blockData[fieldID].(*storage.FloatFieldData).Data = append(blockData[fieldID].(*storage.FloatFieldData).Data, fieldsData[fieldID].([]float32)...)
case schemapb.DataType_Double:
blockData[fieldID].(*storage.DoubleFieldData).Data = append(blockData[fieldID].(*storage.DoubleFieldData).Data, fieldsData[fieldID].([]float64)...)
case schemapb.DataType_Int8:
blockData[fieldID].(*storage.Int8FieldData).Data = append(blockData[fieldID].(*storage.Int8FieldData).Data, fieldsData[fieldID].([]int8)...)
case schemapb.DataType_Int16:
blockData[fieldID].(*storage.Int16FieldData).Data = append(blockData[fieldID].(*storage.Int16FieldData).Data, fieldsData[fieldID].([]int16)...)
case schemapb.DataType_Int32:
blockData[fieldID].(*storage.Int32FieldData).Data = append(blockData[fieldID].(*storage.Int32FieldData).Data, fieldsData[fieldID].([]int32)...)
case schemapb.DataType_Int64:
blockData[fieldID].(*storage.Int64FieldData).Data = append(blockData[fieldID].(*storage.Int64FieldData).Data, fieldsData[fieldID].([]int64)...)
case schemapb.DataType_BinaryVector:
binVectors := fieldsData[fieldID].([][]byte)
for _, vec := range binVectors {
blockData[fieldID].(*storage.BinaryVectorFieldData).Data = append(blockData[fieldID].(*storage.BinaryVectorFieldData).Data, vec...)
}
case schemapb.DataType_FloatVector:
floatVectors := fieldsData[fieldID].([][]float32)
for _, vec := range floatVectors {
blockData[fieldID].(*storage.FloatVectorFieldData).Data = append(blockData[fieldID].(*storage.FloatVectorFieldData).Data, vec...)
}
case schemapb.DataType_String, schemapb.DataType_VarChar:
blockData[fieldID].(*storage.StringFieldData).Data = append(blockData[fieldID].(*storage.StringFieldData).Data, fieldsData[fieldID].([]string)...)
case schemapb.DataType_JSON:
blockData[fieldID].(*storage.JSONFieldData).Data = append(blockData[fieldID].(*storage.JSONFieldData).Data, fieldsData[fieldID].([][]byte)...)
default:
return nil
}
}
}
return blockData
}
func createShardsData(collectionSchema *schemapb.CollectionSchema, fieldsData map[storage.FieldID]interface{},
shardNum int32, partitionIDs []int64) []ShardData {
shardsData := make([]ShardData, 0, shardNum)
for i := 0; i < int(shardNum); i++ {
shardData := make(ShardData)
for p := 0; p < len(partitionIDs); p++ {
blockData := createBlockData(collectionSchema, fieldsData)
shardData[partitionIDs[p]] = blockData
}
shardsData = append(shardsData, shardData)
}
return shardsData
}
func Test_IsCanceled(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
@ -239,7 +406,7 @@ func Test_IsCanceled(t *testing.T) {
func Test_InitSegmentData(t *testing.T) {
testFunc := func(schema *schemapb.CollectionSchema) {
fields := initSegmentData(schema)
fields := initBlockData(schema)
assert.Equal(t, len(schema.Fields)+1, len(fields))
for _, field := range schema.Fields {
@ -272,7 +439,7 @@ func Test_InitSegmentData(t *testing.T) {
},
},
}
data := initSegmentData(schema)
data := initBlockData(schema)
assert.Nil(t, data)
}
@ -345,7 +512,7 @@ func Test_InitValidators(t *testing.T) {
name2ID[field.GetName()] = field.GetFieldID()
}
fields := initSegmentData(schema)
fields := initBlockData(schema)
assert.NotNil(t, fields)
checkConvertFunc := func(funcName string, validVal interface{}, invalidVal interface{}) {
@ -490,7 +657,7 @@ func Test_InitValidators(t *testing.T) {
v, ok := validators[102]
assert.True(t, ok)
fields := initSegmentData(schema)
fields := initBlockData(schema)
assert.NotNil(t, fields)
fieldData := fields[102]
@ -571,7 +738,9 @@ func Test_FillDynamicData(t *testing.T) {
},
}
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
partitionID := int64(1)
flushFunc := func(fields BlockData, shardID int, partID int64) error {
assert.Equal(t, partitionID, partID)
return nil
}
@ -584,12 +753,14 @@ func Test_FillDynamicData(t *testing.T) {
}
t.Run("dynamic field is filled", func(t *testing.T) {
blockData := map[storage.FieldID]storage.FieldData{
blockData := BlockData{
106: idData,
}
segmentsData := []map[storage.FieldID]storage.FieldData{
blockData,
shardsData := []ShardData{
{
partitionID: blockData,
},
}
err := fillDynamicData(blockData, schema)
@ -599,34 +770,36 @@ func Test_FillDynamicData(t *testing.T) {
assert.Equal(t, rowCount, blockData[113].RowNum())
assert.Equal(t, []byte("{}"), blockData[113].GetRow(0).([]byte))
err = tryFlushBlocks(ctx, segmentsData, schema, flushFunc, 1, 1, false)
err = tryFlushBlocks(ctx, shardsData, schema, flushFunc, 1, 1, false)
assert.NoError(t, err)
})
t.Run("collection is dynamic by no dynamic field", func(t *testing.T) {
blockData := map[storage.FieldID]storage.FieldData{
blockData := BlockData{
106: idData,
}
schema.Fields[1].IsDynamic = false
err := fillDynamicData(blockData, schema)
assert.Error(t, err)
segmentsData := []map[storage.FieldID]storage.FieldData{
blockData,
shardsData := []ShardData{
{
partitionID: blockData,
},
}
err = tryFlushBlocks(ctx, segmentsData, schema, flushFunc, 1024*1024, 1, true)
err = tryFlushBlocks(ctx, shardsData, schema, flushFunc, 1024*1024, 1, true)
assert.Error(t, err)
err = tryFlushBlocks(ctx, segmentsData, schema, flushFunc, 1024, 1, false)
err = tryFlushBlocks(ctx, shardsData, schema, flushFunc, 1024, 1, false)
assert.Error(t, err)
err = tryFlushBlocks(ctx, segmentsData, schema, flushFunc, 1024*1024, 1, false)
err = tryFlushBlocks(ctx, shardsData, schema, flushFunc, 1024*1024, 1, false)
assert.Error(t, err)
})
t.Run("collection is not dynamic", func(t *testing.T) {
blockData := map[storage.FieldID]storage.FieldData{
blockData := BlockData{
106: idData,
}
schema.EnableDynamicField = false
@ -640,7 +813,9 @@ func Test_TryFlushBlocks(t *testing.T) {
flushCounter := 0
flushRowCount := 0
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
partitionID := int64(1)
flushFunc := func(fields BlockData, shardID int, partID int64) error {
assert.Equal(t, partitionID, partID)
flushCounter++
rowCount := 0
for _, v := range fields {
@ -657,84 +832,128 @@ func Test_TryFlushBlocks(t *testing.T) {
blockSize := int64(1024)
maxTotalSize := int64(4096)
shardNum := int32(3)
schema := sampleSchema()
// prepare flush data, 3 shards, each shard 10 rows
rowCount := 10
fieldsData := createFieldsData(rowCount)
fieldsData := createFieldsData(schema, rowCount)
shardsData := createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
// non-force flush
segmentsData := createSegmentsData(fieldsData, shardNum)
err := tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, false)
assert.NoError(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
t.Run("non-force flush", func(t *testing.T) {
err := tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, false)
assert.NoError(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
})
// force flush
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, true)
assert.NoError(t, err)
assert.Equal(t, int(shardNum), flushCounter)
assert.Equal(t, rowCount*int(shardNum), flushRowCount)
t.Run("force flush", func(t *testing.T) {
err := tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, true)
assert.NoError(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 = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, true)
assert.NoError(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
t.Run("after force flush, no data left", func(t *testing.T) {
flushCounter = 0
flushRowCount = 0
err := tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, true)
assert.NoError(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
})
// flush when segment size exceeds blockSize
segmentsData = createSegmentsData(fieldsData, shardNum)
blockSize = 100 // blockSize is 100 bytes, less than the 10 rows size
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, false)
assert.NoError(t, err)
assert.Equal(t, int(shardNum), flushCounter)
assert.Equal(t, rowCount*int(shardNum), flushRowCount)
t.Run("flush when segment size exceeds blockSize", func(t *testing.T) {
shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
blockSize = 100 // blockSize is 100 bytes, less than the 10 rows size
err := tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, false)
assert.NoError(t, err)
assert.Equal(t, int(shardNum), flushCounter)
assert.Equal(t, rowCount*int(shardNum), flushRowCount)
flushCounter = 0
flushRowCount = 0
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, true) // no data left
assert.NoError(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
flushCounter = 0
flushRowCount = 0
err = tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, true) // no data left
assert.NoError(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
})
// flush when segments total size exceeds maxTotalSize
segmentsData = createSegmentsData(fieldsData, shardNum)
blockSize = 4096 // blockSize is 4096 bytes, larger than the 10 rows size
maxTotalSize = 100 // maxTotalSize is 100 bytes, less than the 30 rows size
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, false)
assert.NoError(t, err)
assert.Equal(t, 1, flushCounter) // only the max segment is flushed
assert.Equal(t, 10, flushRowCount)
t.Run("flush when segments total size exceeds maxTotalSize", func(t *testing.T) {
shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
blockSize = 4096 // blockSize is 4096 bytes, larger than the 10 rows size
maxTotalSize = 100 // maxTotalSize is 100 bytes, less than the 30 rows size
err := tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, false)
assert.NoError(t, err)
assert.Equal(t, 1, flushCounter) // only the max segment is flushed
assert.Equal(t, 10, flushRowCount)
flushCounter = 0
flushRowCount = 0
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, true) // two segments left
assert.NoError(t, err)
assert.Equal(t, 2, flushCounter)
assert.Equal(t, 20, flushRowCount)
flushCounter = 0
flushRowCount = 0
err = tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, true) // two segments left
assert.NoError(t, err)
assert.Equal(t, 2, flushCounter)
assert.Equal(t, 20, flushRowCount)
})
// call flush function failed
flushFunc = func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
return errors.New("error")
}
segmentsData = createSegmentsData(fieldsData, shardNum)
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, true) // failed to force flush
assert.Error(t, err)
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, 1, maxTotalSize, false) // failed to flush block larger than blockSize
assert.Error(t, err)
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, false) // failed to flush biggest block
assert.Error(t, err)
t.Run("call flush function failed", func(t *testing.T) {
flushErrFunc := func(fields BlockData, shardID int, partID int64) error {
return errors.New("error")
}
shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
err := tryFlushBlocks(ctx, shardsData, schema, flushErrFunc, blockSize, maxTotalSize, true) // failed to force flush
assert.Error(t, err)
err = tryFlushBlocks(ctx, shardsData, schema, flushErrFunc, 1, maxTotalSize, false) // failed to flush block larger than blockSize
assert.Error(t, err)
err = tryFlushBlocks(ctx, shardsData, schema, flushErrFunc, blockSize, maxTotalSize, false) // failed to flush biggest block
assert.Error(t, err)
})
// canceled
cancel()
flushCounter = 0
flushRowCount = 0
segmentsData = createSegmentsData(fieldsData, shardNum)
err = tryFlushBlocks(ctx, segmentsData, sampleSchema(), flushFunc, blockSize, maxTotalSize, true)
assert.Error(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
t.Run("illegal schema", func(t *testing.T) {
illegalSchema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
FieldID: 106,
Name: "ID",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
},
{
FieldID: 108,
Name: "FieldDouble",
DataType: schemapb.DataType_Double,
},
},
}
shardsData = createShardsData(illegalSchema, fieldsData, shardNum, []int64{partitionID})
illegalSchema.Fields[1].DataType = schemapb.DataType_None
err := tryFlushBlocks(ctx, shardsData, illegalSchema, flushFunc, 100, maxTotalSize, true)
assert.Error(t, err)
illegalSchema.Fields[1].DataType = schemapb.DataType_Double
shardsData = createShardsData(illegalSchema, fieldsData, shardNum, []int64{partitionID})
illegalSchema.Fields[1].DataType = schemapb.DataType_None
err = tryFlushBlocks(ctx, shardsData, illegalSchema, flushFunc, 100, maxTotalSize, false)
assert.Error(t, err)
illegalSchema.Fields[1].DataType = schemapb.DataType_Double
shardsData = createShardsData(illegalSchema, fieldsData, shardNum, []int64{partitionID})
illegalSchema.Fields[1].DataType = schemapb.DataType_None
err = tryFlushBlocks(ctx, shardsData, illegalSchema, flushFunc, 4096, maxTotalSize, false)
assert.Error(t, err)
})
t.Run("canceled", func(t *testing.T) {
cancel()
flushCounter = 0
flushRowCount = 0
shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
err := tryFlushBlocks(ctx, shardsData, schema, flushFunc, blockSize, maxTotalSize, true)
assert.Error(t, err)
assert.Equal(t, 0, flushCounter)
assert.Equal(t, 0, flushRowCount)
})
}
func Test_GetTypeName(t *testing.T) {

View File

@ -22,16 +22,14 @@ import (
"fmt"
"strconv"
"github.com/milvus-io/milvus/internal/allocator"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/querycoordv2/params"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/retry"
"github.com/milvus-io/milvus/pkg/util/timerecord"
@ -70,14 +68,16 @@ const (
// ReportImportAttempts is the maximum # of attempts to retry when import fails.
var ReportImportAttempts uint = 10
type ImportFlushFunc func(fields map[storage.FieldID]storage.FieldData, shardID int) error
type AssignSegmentFunc func(shardID int) (int64, string, error)
type CreateBinlogsFunc func(fields map[storage.FieldID]storage.FieldData, segmentID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error)
type SaveSegmentFunc func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64) error
type ImportFlushFunc func(fields BlockData, shardID int, partID int64) error
type AssignSegmentFunc func(shardID int, partID int64) (int64, string, error)
type CreateBinlogsFunc func(fields BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error)
type SaveSegmentFunc func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64, partID int64) error
type ReportFunc func(res *rootcoordpb.ImportResult) error
type WorkingSegment struct {
segmentID int64 // segment ID
shardID int // shard id
shardID int // shard ID
partitionID int64 // partition ID
targetChName string // target dml channel
rowCount int64 // accumulate row count
memSize int // total memory size of all binlogs
@ -86,64 +86,48 @@ type WorkingSegment struct {
}
type ImportWrapper struct {
ctx context.Context // for canceling parse process
cancel context.CancelFunc // for canceling parse process
collectionSchema *schemapb.CollectionSchema // collection schema
shardNum int32 // sharding number of the collection
segmentSize int64 // maximum size of a segment(unit:byte) defined by dataCoord.segment.maxSize (milvus.yml)
rowIDAllocator *allocator.IDAllocator // autoid allocator
chunkManager storage.ChunkManager
ctx context.Context // for canceling parse process
cancel context.CancelFunc // for canceling parse process
collectionInfo *CollectionInfo // collection details including schema
segmentSize int64 // maximum size of a segment(unit:byte) defined by dataCoord.segment.maxSize (milvus.yml)
rowIDAllocator *allocator.IDAllocator // autoid allocator
chunkManager storage.ChunkManager
assignSegmentFunc AssignSegmentFunc // function to prepare a new segment
createBinlogsFunc CreateBinlogsFunc // function to create binlog for a segment
saveSegmentFunc SaveSegmentFunc // function to persist a segment
importResult *rootcoordpb.ImportResult // import result
reportFunc func(res *rootcoordpb.ImportResult) error // report import state to rootcoord
reportImportAttempts uint // attempts count if report function get error
importResult *rootcoordpb.ImportResult // import result
reportFunc ReportFunc // report import state to rootcoord
reportImportAttempts uint // attempts count if report function get error
workingSegments map[int]*WorkingSegment // a map shard id to working segments
progressPercent int64 // working progress percent
workingSegments map[int]map[int64]*WorkingSegment // two-level map shard id and partition id to working segments
progressPercent int64 // working progress percent
}
func NewImportWrapper(ctx context.Context, collectionSchema *schemapb.CollectionSchema, shardNum int32, segmentSize int64,
func NewImportWrapper(ctx context.Context, collectionInfo *CollectionInfo, segmentSize int64,
idAlloc *allocator.IDAllocator, cm storage.ChunkManager, importResult *rootcoordpb.ImportResult,
reportFunc func(res *rootcoordpb.ImportResult) error) *ImportWrapper {
if collectionSchema == nil {
log.Error("import wrapper: collection schema is nil")
if collectionInfo == nil || collectionInfo.Schema == nil {
log.Warn("import wrapper: collection schema is nil")
return nil
}
// ignore the RowID field and Timestamp field
realSchema := &schemapb.CollectionSchema{
Name: collectionSchema.GetName(),
Description: collectionSchema.GetDescription(),
AutoID: collectionSchema.GetAutoID(),
Fields: make([]*schemapb.FieldSchema, 0),
EnableDynamicField: collectionSchema.GetEnableDynamicField(),
}
for i := 0; i < len(collectionSchema.Fields); i++ {
schema := collectionSchema.Fields[i]
if schema.GetName() == common.RowIDFieldName || schema.GetName() == common.TimeStampFieldName {
continue
}
realSchema.Fields = append(realSchema.Fields, schema)
}
log.Info("import wrapper: collection info", zap.Int32("ShardNum", collectionInfo.ShardNum),
zap.Int("PartitionsNum", len(collectionInfo.PartitionIDs)), zap.Any("Fields", collectionInfo.Name2FieldID))
ctx, cancel := context.WithCancel(ctx)
wrapper := &ImportWrapper{
ctx: ctx,
cancel: cancel,
collectionSchema: realSchema,
shardNum: shardNum,
collectionInfo: collectionInfo,
segmentSize: segmentSize,
rowIDAllocator: idAlloc,
chunkManager: cm,
importResult: importResult,
reportFunc: reportFunc,
reportImportAttempts: ReportImportAttempts,
workingSegments: make(map[int]*WorkingSegment),
workingSegments: make(map[int]map[int64]*WorkingSegment),
}
return wrapper
@ -151,17 +135,17 @@ func NewImportWrapper(ctx context.Context, collectionSchema *schemapb.Collection
func (p *ImportWrapper) SetCallbackFunctions(assignSegmentFunc AssignSegmentFunc, createBinlogsFunc CreateBinlogsFunc, saveSegmentFunc SaveSegmentFunc) error {
if assignSegmentFunc == nil {
log.Error("import wrapper: callback function AssignSegmentFunc is nil")
log.Warn("import wrapper: callback function AssignSegmentFunc is nil")
return fmt.Errorf("callback function AssignSegmentFunc is nil")
}
if createBinlogsFunc == nil {
log.Error("import wrapper: callback function CreateBinlogsFunc is nil")
log.Warn("import wrapper: callback function CreateBinlogsFunc is nil")
return fmt.Errorf("callback function CreateBinlogsFunc is nil")
}
if saveSegmentFunc == nil {
log.Error("import wrapper: callback function SaveSegmentFunc is nil")
log.Warn("import wrapper: callback function SaveSegmentFunc is nil")
return fmt.Errorf("callback function SaveSegmentFunc is nil")
}
@ -192,7 +176,7 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) {
// only allow json file or numpy file
if fileType != JSONFileExt && fileType != NumpyFileExt {
log.Error("import wrapper: unsupported file type", zap.String("filePath", filePath))
log.Warn("import wrapper: unsupported file type", zap.String("filePath", filePath))
return false, fmt.Errorf("unsupported file type: '%s'", filePath)
}
@ -205,12 +189,12 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) {
// row-based only support json type, column-based only support numpy type
if rowBased {
if fileType != JSONFileExt {
log.Error("import wrapper: unsupported file type for row-based mode", zap.String("filePath", filePath))
log.Warn("import wrapper: unsupported file type for row-based mode", zap.String("filePath", filePath))
return rowBased, fmt.Errorf("unsupported file type for row-based mode: '%s'", filePath)
}
} else {
if fileType != NumpyFileExt {
log.Error("import wrapper: unsupported file type for column-based mode", zap.String("filePath", filePath))
log.Warn("import wrapper: unsupported file type for column-based mode", zap.String("filePath", filePath))
return rowBased, fmt.Errorf("unsupported file type for column-based mode: '%s'", filePath)
}
}
@ -218,7 +202,7 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) {
// check dupliate file
_, ok := fileNames[name]
if ok {
log.Error("import wrapper: duplicate file name", zap.String("filePath", filePath))
log.Warn("import wrapper: duplicate file name", zap.String("filePath", filePath))
return rowBased, fmt.Errorf("duplicate file: '%s'", filePath)
}
fileNames[name] = struct{}{}
@ -226,20 +210,21 @@ func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) {
// check file size, single file size cannot exceed MaxFileSize
size, err := p.chunkManager.Size(p.ctx, filePath)
if err != nil {
log.Error("import wrapper: failed to get file size", zap.String("filePath", filePath), zap.Error(err))
log.Warn("import wrapper: failed to get file size", zap.String("filePath", filePath), zap.Error(err))
return rowBased, fmt.Errorf("failed to get file size of '%s', error:%w", filePath, err)
}
// empty file
if size == 0 {
log.Error("import wrapper: file size is zero", zap.String("filePath", filePath))
log.Warn("import wrapper: file size is zero", zap.String("filePath", filePath))
return rowBased, fmt.Errorf("the file '%s' size is zero", filePath)
}
if size > params.Params.CommonCfg.ImportMaxFileSize.GetAsInt64() {
log.Error("import wrapper: file size exceeds the maximum size", zap.String("filePath", filePath),
zap.Int64("fileSize", size), zap.Int64("MaxFileSize", params.Params.CommonCfg.ImportMaxFileSize.GetAsInt64()))
return rowBased, fmt.Errorf("the file '%s' size exceeds the maximum size: %d bytes", filePath, params.Params.CommonCfg.ImportMaxFileSize.GetAsInt64())
log.Warn("import wrapper: file size exceeds the maximum size", zap.String("filePath", filePath),
zap.Int64("fileSize", size), zap.String("MaxFileSize", params.Params.CommonCfg.ImportMaxFileSize.GetValue()))
return rowBased, fmt.Errorf("the file '%s' size exceeds the maximum size: %s bytes",
filePath, params.Params.CommonCfg.ImportMaxFileSize.GetValue())
}
totalSize += size
}
@ -278,7 +263,7 @@ func (p *ImportWrapper) Import(filePaths []string, options ImportOptions) error
if fileType == JSONFileExt {
err = p.parseRowBasedJSON(filePath, options.OnlyValidate)
if err != nil {
log.Error("import wrapper: failed to parse row-based json file", zap.Error(err), zap.String("filePath", filePath))
log.Warn("import wrapper: failed to parse row-based json file", zap.Error(err), zap.String("filePath", filePath))
return err
}
} // no need to check else, since the fileValidation() already do this
@ -290,11 +275,11 @@ func (p *ImportWrapper) Import(filePaths []string, options ImportOptions) error
// parse and consume column-based files(currently support numpy)
// for column-based files, the NumpyParser will generate autoid for primary key, and split rows into segments
// according to shard number, so the flushFunc will be called in the NumpyParser
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partitionID int64) error {
printFieldsDataInfo(fields, "import wrapper: prepare to flush binlog data", filePaths)
return p.flushFunc(fields, shardID)
return p.flushFunc(fields, shardID, partitionID)
}
parser, err := NewNumpyParser(p.ctx, p.collectionSchema, p.rowIDAllocator, p.shardNum, SingleBlockSize,
parser, err := NewNumpyParser(p.ctx, p.collectionInfo, p.rowIDAllocator, SingleBlockSize,
p.chunkManager, flushFunc, p.updateProgressPercent)
if err != nil {
return err
@ -392,12 +377,12 @@ func (p *ImportWrapper) isBinlogImport(filePaths []string) bool {
func (p *ImportWrapper) doBinlogImport(filePaths []string, tsStartPoint uint64, tsEndPoint uint64) error {
tr := timerecord.NewTimeRecorder("Import task")
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partitionID int64) error {
printFieldsDataInfo(fields, "import wrapper: prepare to flush binlog data", filePaths)
return p.flushFunc(fields, shardID)
return p.flushFunc(fields, shardID, partitionID)
}
parser, err := NewBinlogParser(p.ctx, p.collectionSchema, p.shardNum, SingleBlockSize, p.chunkManager, flushFunc,
p.updateProgressPercent, tsStartPoint, tsEndPoint)
parser, err := NewBinlogParser(p.ctx, p.collectionInfo, SingleBlockSize,
p.chunkManager, flushFunc, p.updateProgressPercent, tsStartPoint, tsEndPoint)
if err != nil {
return err
}
@ -429,23 +414,23 @@ func (p *ImportWrapper) parseRowBasedJSON(filePath string, onlyValidate bool) er
// parse file
reader := bufio.NewReader(file)
parser := NewJSONParser(p.ctx, p.collectionSchema, p.updateProgressPercent)
parser := NewJSONParser(p.ctx, p.collectionInfo, p.updateProgressPercent)
// if only validate, we input a empty flushFunc so that the consumer do nothing but only validation.
var flushFunc ImportFlushFunc
if onlyValidate {
flushFunc = func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc = func(fields BlockData, shardID int, partitionID int64) error {
return nil
}
} else {
flushFunc = func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc = func(fields BlockData, shardID int, partitionID int64) error {
var filePaths = []string{filePath}
printFieldsDataInfo(fields, "import wrapper: prepare to flush binlogs", filePaths)
return p.flushFunc(fields, shardID)
return p.flushFunc(fields, shardID, partitionID)
}
}
consumer, err := NewJSONRowConsumer(p.collectionSchema, p.rowIDAllocator, p.shardNum, SingleBlockSize, flushFunc)
consumer, err := NewJSONRowConsumer(p.ctx, p.collectionInfo, p.rowIDAllocator, SingleBlockSize, flushFunc)
if err != nil {
return err
}
@ -463,7 +448,12 @@ func (p *ImportWrapper) parseRowBasedJSON(filePath string, onlyValidate bool) er
}
// flushFunc is the callback function for parsers generate segment and save binlog files
func (p *ImportWrapper) flushFunc(fields map[storage.FieldID]storage.FieldData, shardID int) error {
func (p *ImportWrapper) flushFunc(fields BlockData, shardID int, partitionID int64) error {
logFields := []zap.Field{
zap.Int("shardID", shardID),
zap.Int64("partitionID", partitionID),
}
// if fields data is empty, do nothing
var rowNum int
memSize := 0
@ -473,52 +463,59 @@ func (p *ImportWrapper) flushFunc(fields map[storage.FieldID]storage.FieldData,
break
}
if rowNum <= 0 {
log.Warn("import wrapper: fields data is empty", zap.Int("shardID", shardID))
log.Warn("import wrapper: fields data is empty", logFields...)
return nil
}
// if there is no segment for this shard, create a new one
// if the segment exists and its size almost exceed segmentSize, close it and create a new one
var segment *WorkingSegment
segment, ok := p.workingSegments[shardID]
if ok {
// the segment already exists, check its size, if the size exceeds(or almost) segmentSize, close the segment
if int64(segment.memSize)+int64(memSize) >= p.segmentSize {
err := p.closeWorkingSegment(segment)
if err != nil {
return err
if shard, ok := p.workingSegments[shardID]; ok {
if segment, exists := shard[partitionID]; exists {
// the segment already exists, check its size, if the size exceeds(or almost) segmentSize, close the segment
if int64(segment.memSize)+int64(memSize) >= p.segmentSize {
err := p.closeWorkingSegment(segment)
if err != nil {
logFields = append(logFields, zap.Error(err))
log.Warn("import wrapper: failed to close working segment", logFields...)
return err
}
segment = nil
p.workingSegments[shardID][partitionID] = nil
}
segment = nil
p.workingSegments[shardID] = nil
}
} else {
p.workingSegments[shardID] = make(map[int64]*WorkingSegment)
}
if segment == nil {
// create a new segment
segID, channelName, err := p.assignSegmentFunc(shardID)
segID, channelName, err := p.assignSegmentFunc(shardID, partitionID)
if err != nil {
log.Error("import wrapper: failed to assign a new segment", zap.Error(err), zap.Int("shardID", shardID))
logFields = append(logFields, zap.Error(err))
log.Warn("import wrapper: failed to assign a new segment", logFields...)
return fmt.Errorf("failed to assign a new segment for shard id %d, error: %w", shardID, err)
}
segment = &WorkingSegment{
segmentID: segID,
shardID: shardID,
partitionID: partitionID,
targetChName: channelName,
rowCount: int64(0),
memSize: 0,
fieldsInsert: make([]*datapb.FieldBinlog, 0),
fieldsStats: make([]*datapb.FieldBinlog, 0),
}
p.workingSegments[shardID] = segment
p.workingSegments[shardID][partitionID] = segment
}
// save binlogs
fieldsInsert, fieldsStats, err := p.createBinlogsFunc(fields, segment.segmentID)
fieldsInsert, fieldsStats, err := p.createBinlogsFunc(fields, segment.segmentID, partitionID)
if err != nil {
log.Error("import wrapper: failed to save binlogs", zap.Error(err), zap.Int("shardID", shardID),
zap.Int64("segmentID", segment.segmentID), zap.String("targetChannel", segment.targetChName))
logFields = append(logFields, zap.Error(err), zap.Int64("segmentID", segment.segmentID),
zap.String("targetChannel", segment.targetChName))
log.Warn("import wrapper: failed to save binlogs", logFields...)
return fmt.Errorf("failed to save binlogs, shard id %d, segment id %d, channel '%s', error: %w",
shardID, segment.segmentID, segment.targetChName, err)
}
@ -536,7 +533,8 @@ func (p *ImportWrapper) flushFunc(fields map[storage.FieldID]storage.FieldData,
return p.reportFunc(p.importResult)
}, retry.Attempts(p.reportImportAttempts))
if reportErr != nil {
log.Warn("import wrapper: fail to report working progress percent value to RootCoord", zap.Error(reportErr))
logFields = append(logFields, zap.Error(err))
log.Warn("import wrapper: fail to report working progress percent value to RootCoord", logFields...)
}
return nil
@ -544,21 +542,21 @@ func (p *ImportWrapper) flushFunc(fields map[storage.FieldID]storage.FieldData,
// closeWorkingSegment marks a segment to be sealed
func (p *ImportWrapper) closeWorkingSegment(segment *WorkingSegment) error {
log.Info("import wrapper: adding segment to the correct DataNode flow graph and saving binlog paths",
logFields := []zap.Field{
zap.Int("shardID", segment.shardID),
zap.Int64("segmentID", segment.segmentID),
zap.String("targetChannel", segment.targetChName),
zap.Int64("rowCount", segment.rowCount),
zap.Int("insertLogCount", len(segment.fieldsInsert)),
zap.Int("statsLogCount", len(segment.fieldsStats)))
zap.Int("statsLogCount", len(segment.fieldsStats)),
}
log.Info("import wrapper: adding segment to the correct DataNode flow graph and saving binlog paths", logFields...)
err := p.saveSegmentFunc(segment.fieldsInsert, segment.fieldsStats, segment.segmentID, segment.targetChName, segment.rowCount)
err := p.saveSegmentFunc(segment.fieldsInsert, segment.fieldsStats, segment.segmentID, segment.targetChName,
segment.rowCount, segment.partitionID)
if err != nil {
log.Error("import wrapper: failed to seal segment",
zap.Error(err),
zap.Int("shardID", segment.shardID),
zap.Int64("segmentID", segment.segmentID),
zap.String("targetChannel", segment.targetChName))
logFields = append(logFields, zap.Error(err))
log.Warn("import wrapper: failed to seal segment", logFields...)
return fmt.Errorf("failed to seal segment, shard id %d, segment id %d, channel '%s', error: %w",
segment.shardID, segment.segmentID, segment.targetChName, err)
}
@ -568,13 +566,15 @@ func (p *ImportWrapper) closeWorkingSegment(segment *WorkingSegment) error {
// closeAllWorkingSegments mark all segments to be sealed at the end of import operation
func (p *ImportWrapper) closeAllWorkingSegments() error {
for _, segment := range p.workingSegments {
err := p.closeWorkingSegment(segment)
if err != nil {
return err
for _, shard := range p.workingSegments {
for _, segment := range shard {
err := p.closeWorkingSegment(segment)
if err != nil {
return err
}
}
}
p.workingSegments = make(map[int]*WorkingSegment)
p.workingSegments = make(map[int]map[int64]*WorkingSegment)
return nil
}

View File

@ -48,6 +48,7 @@ const (
)
type MockChunkManager struct {
readerErr error
size int64
sizeErr error
readBuf map[string][]byte
@ -65,7 +66,7 @@ func (mc *MockChunkManager) Path(ctx context.Context, filePath string) (string,
}
func (mc *MockChunkManager) Reader(ctx context.Context, filePath string) (storage.FileReader, error) {
return nil, nil
return nil, mc.readerErr
}
func (mc *MockChunkManager) Write(ctx context.Context, filePath string, content []byte) error {
@ -148,7 +149,7 @@ type rowCounterTest struct {
}
func createMockCallbackFunctions(t *testing.T, rowCounter *rowCounterTest) (AssignSegmentFunc, CreateBinlogsFunc, SaveSegmentFunc) {
createBinlogFunc := func(fields map[storage.FieldID]storage.FieldData, segmentID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
createBinlogFunc := func(fields BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
count := 0
for _, data := range fields {
assert.Less(t, 0, data.RowNum())
@ -163,25 +164,26 @@ func createMockCallbackFunctions(t *testing.T, rowCounter *rowCounterTest) (Assi
return nil, nil, nil
}
assignSegmentFunc := func(shardID int) (int64, string, error) {
assignSegmentFunc := func(shardID int, partID int64) (int64, string, error) {
return 100, "ch", nil
}
saveSegmentFunc := func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64) error {
saveSegmentFunc := func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog,
segmentID int64, targetChName string, rowCount int64, partID int64) error {
return nil
}
return assignSegmentFunc, createBinlogFunc, saveSegmentFunc
}
func Test_NewImportWrapper(t *testing.T) {
func Test_ImportWrapperNew(t *testing.T) {
// NewDefaultFactory() use "/tmp/milvus" as default root path, and cannot specify root path
// NewChunkManagerFactory() can specify the root path
f := storage.NewChunkManagerFactory("local", storage.RootPath(TempFilesPath))
ctx := context.Background()
cm, err := f.NewPersistentStorageChunkManager(ctx)
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, nil, 2, 1, nil, cm, nil, nil)
wrapper := NewImportWrapper(ctx, nil, 1, nil, cm, nil, nil)
assert.Nil(t, wrapper)
schema := &schemapb.CollectionSchema{
@ -199,16 +201,19 @@ func Test_NewImportWrapper(t *testing.T) {
Description: "int64",
DataType: schemapb.DataType_Int64,
})
wrapper = NewImportWrapper(ctx, schema, 2, 1, nil, cm, nil, nil)
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
wrapper = NewImportWrapper(ctx, collectionInfo, 1, nil, cm, nil, nil)
assert.NotNil(t, wrapper)
assignSegFunc := func(shardID int) (int64, string, error) {
assignSegFunc := func(shardID int, partID int64) (int64, string, error) {
return 0, "", nil
}
createBinFunc := func(fields map[storage.FieldID]storage.FieldData, segmentID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
createBinFunc := func(fields BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
return nil, nil, nil
}
saveBinFunc := func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64) error {
saveBinFunc := func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog,
segmentID int64, targetChName string, rowCount int64, partID int64) error {
return nil
}
@ -242,11 +247,11 @@ func Test_ImportWrapperRowBased(t *testing.T) {
content := []byte(`{
"rows":[
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": {"x": 2}, "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]},
{"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{\"k\": 2.5}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4]},
{"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": {"y": "hello"}, "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4]},
{"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4]},
{"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4]}
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": {"x": 2}, "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 7, "b": true}},
{"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{\"k\": 2.5}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4], "FieldJSON": {"a": 8, "b": 2}},
{"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": {"y": "hello"}, "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4], "FieldJSON": {"a": 9, "b": false}},
{"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4], "FieldJSON": {"a": 10, "b": 2.15}},
{"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4], "FieldJSON": {"a": 11, "b": "s"}}
]
}`)
@ -258,7 +263,6 @@ func Test_ImportWrapperRowBased(t *testing.T) {
rowCounter := &rowCounterTest{}
assignSegmentFunc, flushFunc, saveSegmentFunc := createMockCallbackFunctions(t, rowCounter)
// success case
importResult := &rootcoordpb.ImportResult{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -273,128 +277,52 @@ func Test_ImportWrapperRowBased(t *testing.T) {
reportFunc := func(res *rootcoordpb.ImportResult) error {
return nil
}
wrapper := NewImportWrapper(ctx, sampleSchema(), 2, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
files := make([]string, 0)
files = append(files, filePath)
err = wrapper.Import(files, ImportOptions{OnlyValidate: true})
assert.NoError(t, err)
assert.Equal(t, 0, rowCounter.rowCount)
err = wrapper.Import(files, DefaultImportOptions())
assert.NoError(t, err)
assert.Equal(t, 5, rowCounter.rowCount)
assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State)
// parse error
content = []byte(`{
"rows":[
{"FieldBool": true, "FieldInt8": false, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": "{\"x\": 2}", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]},
]
}`)
filePath = TempFilesPath + "rows_2.json"
err = cm.Write(ctx, filePath, content)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
importResult.State = commonpb.ImportState_ImportStarted
wrapper = NewImportWrapper(ctx, sampleSchema(), 2, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
files = make([]string, 0)
files = append(files, filePath)
err = wrapper.Import(files, ImportOptions{OnlyValidate: true})
assert.Error(t, err)
assert.NotEqual(t, commonpb.ImportState_ImportPersisted, importResult.State)
t.Run("success case", func(t *testing.T) {
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
files := make([]string, 0)
files = append(files, filePath)
err = wrapper.Import(files, ImportOptions{OnlyValidate: true})
assert.NoError(t, err)
assert.Equal(t, 0, rowCounter.rowCount)
// file doesn't exist
files = make([]string, 0)
files = append(files, "/dummy/dummy.json")
err = wrapper.Import(files, ImportOptions{OnlyValidate: true})
assert.Error(t, err)
}
err = wrapper.Import(files, DefaultImportOptions())
assert.NoError(t, err)
assert.Equal(t, 5, rowCounter.rowCount)
assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State)
})
func createSampleNumpyFiles(t *testing.T, cm storage.ChunkManager) []string {
ctx := context.Background()
files := make([]string, 0)
t.Run("parse error", func(t *testing.T) {
content = []byte(`{
"rows":[
{"FieldBool": true, "FieldInt8": false, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": "{\"x\": 2}", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 9, "b": false}},
]
}`)
filePath := path.Join(cm.RootPath(), "FieldBool.npy")
content, err := CreateNumpyData([]bool{true, false, true, true, true})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = TempFilesPath + "rows_2.json"
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
filePath = path.Join(cm.RootPath(), "FieldInt8.npy")
content, err = CreateNumpyData([]int8{10, 11, 12, 13, 14})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
importResult.State = commonpb.ImportState_ImportStarted
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
files := make([]string, 0)
files = append(files, filePath)
err = wrapper.Import(files, ImportOptions{OnlyValidate: true})
assert.Error(t, err)
assert.NotEqual(t, commonpb.ImportState_ImportPersisted, importResult.State)
})
filePath = path.Join(cm.RootPath(), "FieldInt16.npy")
content, err = CreateNumpyData([]int16{100, 101, 102, 103, 104})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldInt32.npy")
content, err = CreateNumpyData([]int32{1000, 1001, 1002, 1003, 1004})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldInt64.npy")
content, err = CreateNumpyData([]int64{10000, 10001, 10002, 10003, 10004})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldFloat.npy")
content, err = CreateNumpyData([]float32{3.14, 3.15, 3.16, 3.17, 3.18})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldDouble.npy")
content, err = CreateNumpyData([]float64{5.1, 5.2, 5.3, 5.4, 5.5})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldString.npy")
content, err = CreateNumpyData([]string{"a", "bb", "ccc", "dd", "e"})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldJSON.npy")
content, err = CreateNumpyData([]string{"{\"x\": 10, \"y\": 5}", "{\"z\": 5}", "{}", "{}", "{\"x\": 3}"})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldBinaryVector.npy")
content, err = CreateNumpyData([][2]uint8{{1, 2}, {3, 4}, {5, 6}, {7, 8}, {9, 10}})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldFloatVector.npy")
content, err = CreateNumpyData([][4]float32{{1, 2, 3, 4}, {3, 4, 5, 6}, {5, 6, 7, 8}, {7, 8, 9, 10}, {9, 10, 11, 12}})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
return files
t.Run("file doesn't exist", func(t *testing.T) {
files := make([]string, 0)
files = append(files, "/dummy/dummy.json")
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
err = wrapper.Import(files, ImportOptions{OnlyValidate: true})
assert.Error(t, err)
})
}
func Test_ImportWrapperColumnBased_numpy(t *testing.T) {
@ -415,7 +343,6 @@ func Test_ImportWrapperColumnBased_numpy(t *testing.T) {
rowCounter := &rowCounterTest{}
assignSegmentFunc, flushFunc, saveSegmentFunc := createMockCallbackFunctions(t, rowCounter)
// success case
importResult := &rootcoordpb.ImportResult{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
@ -430,37 +357,45 @@ func Test_ImportWrapperColumnBased_numpy(t *testing.T) {
reportFunc := func(res *rootcoordpb.ImportResult) error {
return nil
}
schema := sampleSchema()
wrapper := NewImportWrapper(ctx, schema, 2, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
files := createSampleNumpyFiles(t, cm)
err = wrapper.Import(files, DefaultImportOptions())
assert.NoError(t, err)
assert.Equal(t, 5, rowCounter.rowCount)
assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State)
// row count of fields not equal
filePath := path.Join(cm.RootPath(), "FieldInt8.npy")
content, err := CreateNumpyData([]int8{10})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files[1] = filePath
t.Run("success case", func(t *testing.T) {
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
importResult.State = commonpb.ImportState_ImportStarted
wrapper = NewImportWrapper(ctx, sampleSchema(), 2, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
err = wrapper.Import(files, DefaultImportOptions())
assert.NoError(t, err)
assert.Equal(t, 5, rowCounter.rowCount)
assert.Equal(t, commonpb.ImportState_ImportPersisted, importResult.State)
})
err = wrapper.Import(files, DefaultImportOptions())
assert.Error(t, err)
assert.NotEqual(t, commonpb.ImportState_ImportPersisted, importResult.State)
t.Run("row count of fields not equal", func(t *testing.T) {
filePath := path.Join(cm.RootPath(), "FieldInt8.npy")
content, err := CreateNumpyData([]int8{10})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files[1] = filePath
// file doesn't exist
files = make([]string, 0)
files = append(files, "/dummy/dummy.npy")
err = wrapper.Import(files, DefaultImportOptions())
assert.Error(t, err)
importResult.State = commonpb.ImportState_ImportStarted
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
err = wrapper.Import(files, DefaultImportOptions())
assert.Error(t, err)
assert.NotEqual(t, commonpb.ImportState_ImportPersisted, importResult.State)
})
t.Run("file doesn't exist", func(t *testing.T) {
files := make([]string, 0)
files = append(files, "/dummy/dummy.npy")
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
err = wrapper.Import(files, DefaultImportOptions())
assert.Error(t, err)
})
}
func perfSchema(dim int) *schemapb.CollectionSchema {
@ -577,7 +512,9 @@ func Test_ImportWrapperRowBased_perf(t *testing.T) {
reportFunc := func(res *rootcoordpb.ImportResult) error {
return nil
}
wrapper := NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, importResult, reportFunc)
collectionInfo, err := NewCollectionInfo(schema, int32(shardNum), []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, int64(segmentSize), idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
files := make([]string, 0)
@ -619,8 +556,10 @@ func Test_ImportWrapperFileValidation(t *testing.T) {
shardNum := 2
segmentSize := 512 // unit: MB
wrapper := NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil)
params.Params.Init()
collectionInfo, err := NewCollectionInfo(schema, int32(shardNum), []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, int64(segmentSize), idAllocator, cm, nil, nil)
t.Run("unsupported file type", func(t *testing.T) {
files := []string{"uid.txt"}
rowBased, err := wrapper.fileValidation(files)
@ -666,10 +605,10 @@ func Test_ImportWrapperFileValidation(t *testing.T) {
assert.False(t, rowBased)
})
t.Run("empty file", func(t *testing.T) {
t.Run("empty file list", func(t *testing.T) {
files := []string{}
cm.size = 0
wrapper = NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil)
wrapper = NewImportWrapper(ctx, collectionInfo, int64(segmentSize), idAllocator, cm, nil, nil)
rowBased, err := wrapper.fileValidation(files)
assert.NoError(t, err)
assert.False(t, rowBased)
@ -678,7 +617,7 @@ func Test_ImportWrapperFileValidation(t *testing.T) {
t.Run("file size exceed MaxFileSize limit", func(t *testing.T) {
files := []string{"a/1.json"}
cm.size = params.Params.CommonCfg.ImportMaxFileSize.GetAsInt64() + 1
wrapper = NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil)
wrapper = NewImportWrapper(ctx, collectionInfo, int64(segmentSize), idAllocator, cm, nil, nil)
rowBased, err := wrapper.fileValidation(files)
assert.Error(t, err)
assert.True(t, rowBased)
@ -691,6 +630,15 @@ func Test_ImportWrapperFileValidation(t *testing.T) {
assert.Error(t, err)
assert.True(t, rowBased)
})
t.Run("file size is zero", func(t *testing.T) {
files := []string{"a/1.json"}
cm.sizeErr = nil
cm.size = int64(0)
rowBased, err := wrapper.fileValidation(files)
assert.Error(t, err)
assert.True(t, rowBased)
})
}
func Test_ImportWrapperReportFailRowBased(t *testing.T) {
@ -709,11 +657,11 @@ func Test_ImportWrapperReportFailRowBased(t *testing.T) {
content := []byte(`{
"rows":[
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": "{\"x\": \"aaa\"}", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]},
{"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4]},
{"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": "{\"x\": 2, \"y\": 5}", "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4]},
{"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4]},
{"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4]}
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldJSON": "{\"x\": \"aaa\"}", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 9, "b": false}},
{"FieldBool": false, "FieldInt8": 11, "FieldInt16": 102, "FieldInt32": 1002, "FieldInt64": 10002, "FieldFloat": 3.15, "FieldDouble": 2.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [253, 0], "FieldFloatVector": [2.1, 2.2, 2.3, 2.4], "FieldJSON": {"a": 9, "b": false}},
{"FieldBool": true, "FieldInt8": 12, "FieldInt16": 103, "FieldInt32": 1003, "FieldInt64": 10003, "FieldFloat": 3.16, "FieldDouble": 3.56, "FieldString": "hello world", "FieldJSON": "{\"x\": 2, \"y\": 5}", "FieldBinaryVector": [252, 0], "FieldFloatVector": [3.1, 3.2, 3.3, 3.4], "FieldJSON": {"a": 9, "b": false}},
{"FieldBool": false, "FieldInt8": 13, "FieldInt16": 104, "FieldInt32": 1004, "FieldInt64": 10004, "FieldFloat": 3.17, "FieldDouble": 4.56, "FieldString": "hello world", "FieldJSON": "{\"x\": true}", "FieldBinaryVector": [251, 0], "FieldFloatVector": [4.1, 4.2, 4.3, 4.4], "FieldJSON": {"a": 9, "b": false}},
{"FieldBool": true, "FieldInt8": 14, "FieldInt16": 105, "FieldInt32": 1005, "FieldInt64": 10005, "FieldFloat": 3.18, "FieldDouble": 5.56, "FieldString": "hello world", "FieldJSON": "{}", "FieldBinaryVector": [250, 0], "FieldFloatVector": [5.1, 5.2, 5.3, 5.4], "FieldJSON": {"a": 9, "b": false}}
]
}`)
@ -740,7 +688,9 @@ func Test_ImportWrapperReportFailRowBased(t *testing.T) {
reportFunc := func(res *rootcoordpb.ImportResult) error {
return nil
}
wrapper := NewImportWrapper(ctx, sampleSchema(), 2, 1, idAllocator, cm, importResult, reportFunc)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
files := []string{filePath}
@ -787,8 +737,9 @@ func Test_ImportWrapperReportFailColumnBased_numpy(t *testing.T) {
reportFunc := func(res *rootcoordpb.ImportResult) error {
return nil
}
schema := sampleSchema()
wrapper := NewImportWrapper(ctx, schema, 2, 1, idAllocator, cm, importResult, reportFunc)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, 1, idAllocator, cm, importResult, reportFunc)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
wrapper.reportImportAttempts = 2
@ -821,7 +772,9 @@ func Test_ImportWrapperIsBinlogImport(t *testing.T) {
shardNum := 2
segmentSize := 512 // unit: MB
wrapper := NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil)
collectionInfo, err := NewCollectionInfo(schema, int32(shardNum), []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, int64(segmentSize), idAllocator, cm, nil, nil)
// empty paths
paths := []string{}
@ -883,7 +836,9 @@ func Test_ImportWrapperDoBinlogImport(t *testing.T) {
shardNum := 2
segmentSize := 512 // unit: MB
wrapper := NewImportWrapper(ctx, schema, int32(shardNum), int64(segmentSize), idAllocator, cm, nil, nil)
collectionInfo, err := NewCollectionInfo(schema, int32(shardNum), []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, int64(segmentSize), idAllocator, cm, nil, nil)
paths := []string{
"/tmp",
"/tmp",
@ -891,7 +846,7 @@ func Test_ImportWrapperDoBinlogImport(t *testing.T) {
wrapper.chunkManager = nil
// failed to create new BinlogParser
err := wrapper.doBinlogImport(paths, 0, math.MaxUint64)
err = wrapper.doBinlogImport(paths, 0, math.MaxUint64)
assert.Error(t, err)
cm.listErr = errors.New("error")
@ -948,12 +903,14 @@ func Test_ImportWrapperReportPersisted(t *testing.T) {
reportFunc := func(res *rootcoordpb.ImportResult) error {
return nil
}
wrapper := NewImportWrapper(ctx, sampleSchema(), int32(2), int64(1024), nil, nil, importResult, reportFunc)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, int64(1024), nil, nil, importResult, reportFunc)
assert.NotNil(t, wrapper)
rowCounter := &rowCounterTest{}
assignSegmentFunc, flushFunc, saveSegmentFunc := createMockCallbackFunctions(t, rowCounter)
err := wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
err = wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
assert.NoError(t, err)
// success
@ -962,15 +919,19 @@ func Test_ImportWrapperReportPersisted(t *testing.T) {
assert.NotEmpty(t, wrapper.importResult.GetInfos())
// error when closing segments
wrapper.saveSegmentFunc = func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64) error {
wrapper.saveSegmentFunc = func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog,
segmentID int64, targetChName string, rowCount int64, partID int64) error {
return errors.New("error")
}
wrapper.workingSegments[0] = &WorkingSegment{}
wrapper.workingSegments[0] = map[int64]*WorkingSegment{
int64(1): {},
}
err = wrapper.reportPersisted(2, tr)
assert.Error(t, err)
// failed to report
wrapper.saveSegmentFunc = func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64) error {
wrapper.saveSegmentFunc = func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog,
segmentID int64, targetChName string, rowCount int64, partID int64) error {
return nil
}
wrapper.reportFunc = func(res *rootcoordpb.ImportResult) error {
@ -983,7 +944,9 @@ func Test_ImportWrapperReportPersisted(t *testing.T) {
func Test_ImportWrapperUpdateProgressPercent(t *testing.T) {
ctx := context.Background()
wrapper := NewImportWrapper(ctx, sampleSchema(), 2, 1, nil, nil, nil, nil)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, 1, nil, nil, nil, nil)
assert.NotNil(t, wrapper)
assert.Equal(t, int64(0), wrapper.progressPercent)
@ -996,3 +959,91 @@ func Test_ImportWrapperUpdateProgressPercent(t *testing.T) {
wrapper.updateProgressPercent(100)
assert.Equal(t, int64(100), wrapper.progressPercent)
}
func Test_ImportWrapperFlushFunc(t *testing.T) {
ctx := context.Background()
params.Params.Init()
shardID := 0
partitionID := int64(1)
rowCounter := &rowCounterTest{}
assignSegmentFunc, flushFunc, saveSegmentFunc := createMockCallbackFunctions(t, rowCounter)
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,
}
reportFunc := func(res *rootcoordpb.ImportResult) error {
return nil
}
schema := sampleSchema()
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
wrapper := NewImportWrapper(ctx, collectionInfo, 1, nil, nil, importResult, reportFunc)
assert.NotNil(t, wrapper)
wrapper.SetCallbackFunctions(assignSegmentFunc, flushFunc, saveSegmentFunc)
t.Run("fieldsData is empty", func(t *testing.T) {
blockData := initBlockData(schema)
err = wrapper.flushFunc(blockData, shardID, partitionID)
assert.NoError(t, err)
})
fieldsData := createFieldsData(schema, 5)
blockData := createBlockData(schema, fieldsData)
t.Run("fieldsData is not empty", func(t *testing.T) {
err = wrapper.flushFunc(blockData, shardID, partitionID)
assert.NoError(t, err)
assert.Contains(t, wrapper.workingSegments, shardID)
assert.Contains(t, wrapper.workingSegments[shardID], partitionID)
assert.NotNil(t, wrapper.workingSegments[shardID][partitionID])
})
t.Run("close segment, saveSegmentFunc returns error", func(t *testing.T) {
wrapper.saveSegmentFunc = func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog,
segmentID int64, targetChName string, rowCount int64, partID int64) error {
return errors.New("error")
}
wrapper.segmentSize = 1
wrapper.workingSegments = make(map[int]map[int64]*WorkingSegment)
wrapper.workingSegments[shardID] = map[int64]*WorkingSegment{
int64(1): {
memSize: 100,
},
}
err = wrapper.flushFunc(blockData, shardID, partitionID)
assert.Error(t, err)
})
t.Run("assignSegmentFunc returns error", func(t *testing.T) {
wrapper.assignSegmentFunc = func(shardID int, partID int64) (int64, string, error) {
return 100, "ch", errors.New("error")
}
err = wrapper.flushFunc(blockData, 99, partitionID)
assert.Error(t, err)
})
t.Run("createBinlogsFunc returns error", func(t *testing.T) {
wrapper.saveSegmentFunc = func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog,
segmentID int64, targetChName string, rowCount int64, partID int64) error {
return nil
}
wrapper.assignSegmentFunc = func(shardID int, partID int64) (int64, string, error) {
return 100, "ch", nil
}
wrapper.createBinlogsFunc = func(fields BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
return nil, nil, errors.New("error")
}
err = wrapper.flushFunc(blockData, shardID, partitionID)
assert.Error(t, err)
})
}

View File

@ -17,16 +17,16 @@
package importutil
import (
"context"
"encoding/json"
"fmt"
"strconv"
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus/internal/allocator"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
@ -38,97 +38,76 @@ type JSONRowHandler interface {
Handle(rows []map[storage.FieldID]interface{}) error
}
// Validator is field value validator
type Validator struct {
convertFunc func(obj interface{}, field storage.FieldData) error // convert data function
primaryKey bool // true for primary key
autoID bool // only for primary key field
isString bool // for string field
dimension int // only for vector field
fieldName string // field name
}
func getPrimaryKey(obj interface{}, fieldName string, isString bool) (string, error) {
func getKeyValue(obj interface{}, fieldName string, isString bool) (string, error) {
// varchar type primary field, the value must be a string
if isString {
if value, ok := obj.(string); ok {
return value, nil
}
return "", fmt.Errorf("illegal value '%v' for varchar type primary key field '%s'", obj, fieldName)
return "", fmt.Errorf("illegal value '%v' for varchar type key field '%s'", obj, fieldName)
}
// int64 type primary field, the value must be json.Number
if num, ok := obj.(json.Number); ok {
return string(num), nil
}
return "", fmt.Errorf("illegal value '%v' for int64 type primary key field '%s'", obj, fieldName)
return "", fmt.Errorf("illegal value '%v' for int64 type key field '%s'", obj, fieldName)
}
// JSONRowConsumer is row-based json format consumer class
type JSONRowConsumer struct {
collectionSchema *schemapb.CollectionSchema // collection schema
rowIDAllocator *allocator.IDAllocator // autoid allocator
validators map[storage.FieldID]*Validator // validators for each field
rowCounter int64 // how many rows have been consumed
shardNum int32 // sharding number of the collection
segmentsData []map[storage.FieldID]storage.FieldData // in-memory segments data
blockSize int64 // maximum size of a read block(unit:byte)
primaryKey storage.FieldID // name of primary key
autoIDRange []int64 // auto-generated id range, for example: [1, 10, 20, 25] means id from 1 to 10 and 20 to 25
ctx context.Context // for canceling parse process
collectionInfo *CollectionInfo // collection details including schema
rowIDAllocator *allocator.IDAllocator // autoid allocator
validators map[storage.FieldID]*Validator // validators for each field
rowCounter int64 // how many rows have been consumed
shardsData []ShardData // in-memory shards data
blockSize int64 // maximum size of a read block(unit:byte)
autoIDRange []int64 // auto-generated id range, for example: [1, 10, 20, 25] means id from 1 to 10 and 20 to 25
callFlushFunc ImportFlushFunc // call back function to flush segment
}
func NewJSONRowConsumer(collectionSchema *schemapb.CollectionSchema, idAlloc *allocator.IDAllocator, shardNum int32, blockSize int64,
func NewJSONRowConsumer(ctx context.Context,
collectionInfo *CollectionInfo,
idAlloc *allocator.IDAllocator,
blockSize int64,
flushFunc ImportFlushFunc) (*JSONRowConsumer, error) {
if collectionSchema == nil {
log.Error("JSON row consumer: collection schema is nil")
if collectionInfo == nil {
log.Warn("JSON row consumer: collection schema is nil")
return nil, errors.New("collection schema is nil")
}
v := &JSONRowConsumer{
collectionSchema: collectionSchema,
rowIDAllocator: idAlloc,
validators: make(map[storage.FieldID]*Validator),
shardNum: shardNum,
blockSize: blockSize,
rowCounter: 0,
primaryKey: -1,
autoIDRange: make([]int64, 0),
callFlushFunc: flushFunc,
ctx: ctx,
collectionInfo: collectionInfo,
rowIDAllocator: idAlloc,
validators: make(map[storage.FieldID]*Validator),
blockSize: blockSize,
rowCounter: 0,
autoIDRange: make([]int64, 0),
callFlushFunc: flushFunc,
}
err := initValidators(collectionSchema, v.validators)
err := initValidators(collectionInfo.Schema, v.validators)
if err != nil {
log.Error("JSON row consumer: fail to initialize json row-based consumer", zap.Error(err))
log.Warn("JSON row consumer: fail to initialize json row-based consumer", zap.Error(err))
return nil, fmt.Errorf("fail to initialize json row-based consumer, error: %w", err)
}
v.segmentsData = make([]map[storage.FieldID]storage.FieldData, 0, shardNum)
for i := 0; i < int(shardNum); i++ {
segmentData := initSegmentData(collectionSchema)
if segmentData == nil {
log.Error("JSON row consumer: fail to initialize in-memory segment data", zap.Int("shardID", i))
v.shardsData = make([]ShardData, 0, collectionInfo.ShardNum)
for i := 0; i < int(collectionInfo.ShardNum); i++ {
shardData := initShardData(collectionInfo.Schema, collectionInfo.PartitionIDs)
if shardData == nil {
log.Warn("JSON row consumer: fail to initialize in-memory segment data", zap.Int("shardID", i))
return nil, fmt.Errorf("fail to initialize in-memory segment data for shard id %d", i)
}
v.segmentsData = append(v.segmentsData, segmentData)
v.shardsData = append(v.shardsData, shardData)
}
for i := 0; i < len(collectionSchema.Fields); i++ {
schema := collectionSchema.Fields[i]
if schema.GetIsPrimaryKey() {
v.primaryKey = schema.GetFieldID()
break
}
}
// primary key not found
if v.primaryKey == -1 {
log.Error("JSON row consumer: collection schema has no primary key")
return nil, errors.New("collection schema has no primary key")
}
// primary key is autoid, id generator is required
if v.validators[v.primaryKey].autoID && idAlloc == nil {
log.Error("JSON row consumer: ID allocator is nil")
if v.collectionInfo.PrimaryKey.GetAutoID() && idAlloc == nil {
log.Warn("JSON row consumer: ID allocator is nil")
return nil, errors.New("ID allocator is nil")
}
@ -143,92 +122,46 @@ func (v *JSONRowConsumer) RowCount() int64 {
return v.rowCounter
}
func (v *JSONRowConsumer) flush(force bool) error {
// force flush all data
if force {
for i := 0; i < len(v.segmentsData); i++ {
segmentData := v.segmentsData[i]
rowNum := segmentData[v.primaryKey].RowNum()
if rowNum > 0 {
log.Info("JSON row consumer: force flush binlog", zap.Int("rows", rowNum))
err := v.callFlushFunc(segmentData, i)
if err != nil {
return err
}
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")
}
}
}
return nil
}
// segment size can be flushed
for i := 0; i < len(v.segmentsData); i++ {
segmentData := v.segmentsData[i]
rowNum := segmentData[v.primaryKey].RowNum()
memSize := 0
for _, field := range segmentData {
memSize += field.GetMemorySize()
}
if memSize >= int(v.blockSize) && rowNum > 0 {
log.Info("JSON row consumer: flush fulled binlog", zap.Int("bytes", memSize), zap.Int("rowNum", rowNum))
err := v.callFlushFunc(segmentData, i)
if err != nil {
return err
}
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")
}
}
}
return nil
}
func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
if v == nil || v.validators == nil || len(v.validators) == 0 {
log.Error("JSON row consumer is not initialized")
log.Warn("JSON row consumer is not initialized")
return errors.New("JSON row consumer is not initialized")
}
// flush in necessery
// if rows is nil, that means read to end of file, force flush all data
if rows == nil {
err := v.flush(true)
err := tryFlushBlocks(v.ctx, v.shardsData, v.collectionInfo.Schema, v.callFlushFunc, v.blockSize, MaxTotalSizeInMemory, true)
log.Info("JSON row consumer finished")
return err
}
err := v.flush(false)
// rows is not nil, flush in necessary:
// 1. data block size larger than v.blockSize will be flushed
// 2. total data size exceeds MaxTotalSizeInMemory, the largest data block will be flushed
err := tryFlushBlocks(v.ctx, v.shardsData, v.collectionInfo.Schema, v.callFlushFunc, v.blockSize, MaxTotalSizeInMemory, false)
if err != nil {
log.Error("JSON row consumer: try flush data but failed", zap.Error(err))
log.Warn("JSON row consumer: try flush data but failed", zap.Error(err))
return fmt.Errorf("try flush data but failed, error: %w", err)
}
// prepare autoid, no matter int64 or varchar pk, we always generate autoid since the hidden field RowIDField requires them
primaryValidator := v.validators[v.primaryKey]
primaryKeyID := v.collectionInfo.PrimaryKey.FieldID
primaryValidator := v.validators[primaryKeyID]
var rowIDBegin typeutil.UniqueID
var rowIDEnd typeutil.UniqueID
if primaryValidator.autoID {
if v.rowIDAllocator == nil {
log.Error("JSON row consumer: primary keys is auto-generated but IDAllocator is nil")
log.Warn("JSON row consumer: primary keys is auto-generated but IDAllocator is nil")
return fmt.Errorf("primary keys is auto-generated but IDAllocator is nil")
}
var err error
rowIDBegin, rowIDEnd, err = v.rowIDAllocator.Alloc(uint32(len(rows)))
if err != nil {
log.Error("JSON row consumer: failed to generate primary keys", zap.Int("count", len(rows)), zap.Error(err))
log.Warn("JSON row consumer: failed to generate primary keys", zap.Int("count", len(rows)), zap.Error(err))
return fmt.Errorf("failed to generate %d primary keys, error: %w", len(rows), err)
}
if rowIDEnd-rowIDBegin != int64(len(rows)) {
log.Error("JSON row consumer: try to generate primary keys but allocated ids are not enough",
log.Warn("JSON row consumer: try to generate primary keys but allocated ids are not enough",
zap.Int("count", len(rows)), zap.Int64("generated", rowIDEnd-rowIDBegin))
return fmt.Errorf("try to generate %d primary keys but only %d keys were allocated", len(rows), rowIDEnd-rowIDBegin)
}
@ -246,23 +179,30 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
// hash to a shard number
var shard uint32
var partitionID int64
if primaryValidator.isString {
if primaryValidator.autoID {
log.Error("JSON row consumer: string type primary key cannot be auto-generated")
log.Warn("JSON row consumer: string type primary key cannot be auto-generated")
return errors.New("string type primary key cannot be auto-generated")
}
value := row[v.primaryKey]
pk, err := getPrimaryKey(value, primaryValidator.fieldName, primaryValidator.isString)
value := row[primaryKeyID]
pk, err := getKeyValue(value, primaryValidator.fieldName, primaryValidator.isString)
if err != nil {
log.Error("JSON row consumer: failed to parse primary key at the row",
log.Warn("JSON row consumer: failed to parse primary key at the row",
zap.Int64("rowNumber", rowNumber), zap.Error(err))
return fmt.Errorf("failed to parse primary key at the row %d, error: %w", rowNumber, err)
}
// hash to shard based on pk, hash to partition if partition key exist
hash := typeutil.HashString2Uint32(pk)
shard = hash % uint32(v.shardNum)
pkArray := v.segmentsData[shard][v.primaryKey].(*storage.StringFieldData)
shard = hash % uint32(v.collectionInfo.ShardNum)
partitionID, err = v.hashToPartition(row, rowNumber)
if err != nil {
return err
}
pkArray := v.shardsData[shard][partitionID][primaryKeyID].(*storage.StringFieldData)
pkArray.Data = append(pkArray.Data, pk)
} else {
// get/generate the row id
@ -270,10 +210,10 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
if primaryValidator.autoID {
pk = rowIDBegin + int64(i)
} else {
value := row[v.primaryKey]
strValue, err := getPrimaryKey(value, primaryValidator.fieldName, primaryValidator.isString)
value := row[primaryKeyID]
strValue, err := getKeyValue(value, primaryValidator.fieldName, primaryValidator.isString)
if err != nil {
log.Error("JSON row consumer: failed to parse primary key at the row",
log.Warn("JSON row consumer: failed to parse primary key at the row",
zap.Int64("rowNumber", rowNumber), zap.Error(err))
return fmt.Errorf("failed to parse primary key at the row %d, error: %w", rowNumber, err)
}
@ -281,7 +221,7 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
// parse the pk from a string
pk, err = strconv.ParseInt(strValue, 10, 64)
if err != nil {
log.Error("JSON row consumer: failed to parse primary key at the row",
log.Warn("JSON row consumer: failed to parse primary key at the row",
zap.String("value", strValue), zap.Int64("rowNumber", rowNumber), zap.Error(err))
return fmt.Errorf("failed to parse primary key '%s' at the row %d, error: %w",
strValue, rowNumber, err)
@ -290,28 +230,34 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
hash, err := typeutil.Hash32Int64(pk)
if err != nil {
log.Error("JSON row consumer: failed to hash primary key at the row",
log.Warn("JSON row consumer: failed to hash primary key at the row",
zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err))
return fmt.Errorf("failed to hash primary key %d at the row %d, error: %w", pk, rowNumber, err)
}
shard = hash % uint32(v.shardNum)
pkArray := v.segmentsData[shard][v.primaryKey].(*storage.Int64FieldData)
// hash to shard based on pk, hash to partition if partition key exist
shard = hash % uint32(v.collectionInfo.ShardNum)
partitionID, err = v.hashToPartition(row, rowNumber)
if err != nil {
return err
}
pkArray := v.shardsData[shard][partitionID][primaryKeyID].(*storage.Int64FieldData)
pkArray.Data = append(pkArray.Data, pk)
}
// set rowid field
rowIDField := v.segmentsData[shard][common.RowIDField].(*storage.Int64FieldData)
rowIDField := v.shardsData[shard][partitionID][common.RowIDField].(*storage.Int64FieldData)
rowIDField.Data = append(rowIDField.Data, rowIDBegin+int64(i))
// convert value and consume
for name, validator := range v.validators {
for fieldID, validator := range v.validators {
if validator.primaryKey {
continue
}
value := row[name]
if err := validator.convertFunc(value, v.segmentsData[shard][name]); err != nil {
log.Error("JSON row consumer: failed to convert value for field at the row",
value := row[fieldID]
if err := validator.convertFunc(value, v.shardsData[shard][partitionID][fieldID]); err != nil {
log.Warn("JSON row consumer: failed to convert value for field at the row",
zap.String("fieldName", validator.fieldName), zap.Int64("rowNumber", rowNumber), zap.Error(err))
return fmt.Errorf("failed to convert value for field '%s' at the row %d, error: %w",
validator.fieldName, rowNumber, err)
@ -323,3 +269,49 @@ func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
return nil
}
// hashToPartition hash partition key to get an partition ID, return the first partition ID if no partition key exist
// CollectionInfo ensures only one partition ID in the PartitionIDs if no partition key exist
func (v *JSONRowConsumer) hashToPartition(row map[storage.FieldID]interface{}, rowNumber int64) (int64, error) {
if v.collectionInfo.PartitionKey == nil {
if len(v.collectionInfo.PartitionIDs) != 1 {
return 0, fmt.Errorf("collection '%s' partition list is empty", v.collectionInfo.Schema.Name)
}
// no partition key, directly return the target partition id
return v.collectionInfo.PartitionIDs[0], nil
}
partitionKeyID := v.collectionInfo.PartitionKey.GetFieldID()
partitionKeyValidator := v.validators[partitionKeyID]
value := row[partitionKeyID]
strValue, err := getKeyValue(value, partitionKeyValidator.fieldName, partitionKeyValidator.isString)
if err != nil {
log.Warn("JSON row consumer: failed to parse partition key at the row",
zap.Int64("rowNumber", rowNumber), zap.Error(err))
return 0, fmt.Errorf("failed to parse partition key at the row %d, error: %w", rowNumber, err)
}
var hashValue uint32
if partitionKeyValidator.isString {
hashValue = typeutil.HashString2Uint32(strValue)
} else {
// parse the value from a string
pk, err := strconv.ParseInt(strValue, 10, 64)
if err != nil {
log.Warn("JSON row consumer: failed to parse partition key at the row",
zap.String("value", strValue), zap.Int64("rowNumber", rowNumber), zap.Error(err))
return 0, fmt.Errorf("failed to parse partition key '%s' at the row %d, error: %w",
strValue, rowNumber, err)
}
hashValue, err = typeutil.Hash32Int64(pk)
if err != nil {
log.Warn("JSON row consumer: failed to hash partition key at the row",
zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err))
return 0, fmt.Errorf("failed to hash partition key %d at the row %d, error: %w", pk, rowNumber, err)
}
}
index := int64(hashValue % uint32(len(v.collectionInfo.PartitionIDs)))
return v.collectionInfo.PartitionIDs[index], nil
}

View File

@ -19,6 +19,7 @@ package importutil
import (
"context"
"encoding/json"
"strconv"
"testing"
"github.com/cockroachdb/errors"
@ -30,7 +31,6 @@ import (
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/common"
)
type mockIDAllocator struct {
@ -61,216 +61,61 @@ func newIDAllocator(ctx context.Context, t *testing.T, allocErr error) *allocato
return idAllocator
}
func Test_NewJSONRowConsumer(t *testing.T) {
// nil schema
consumer, err := NewJSONRowConsumer(nil, nil, 2, 16, nil)
assert.Error(t, err)
assert.Nil(t, consumer)
// wrong schema
schema := &schemapb.CollectionSchema{
Name: "schema",
AutoID: true,
Fields: []*schemapb.FieldSchema{
{
FieldID: 101,
Name: "uid",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_None,
},
},
}
consumer, err = NewJSONRowConsumer(schema, nil, 2, 16, nil)
assert.Error(t, err)
assert.Nil(t, consumer)
// no primary key
schema.Fields[0].IsPrimaryKey = false
schema.Fields[0].DataType = schemapb.DataType_Int64
consumer, err = NewJSONRowConsumer(schema, nil, 2, 16, nil)
assert.Error(t, err)
assert.Nil(t, consumer)
// primary key is autoid, but no IDAllocator
schema.Fields[0].IsPrimaryKey = true
schema.Fields[0].AutoID = true
consumer, err = NewJSONRowConsumer(schema, nil, 2, 16, nil)
assert.Error(t, err)
assert.Nil(t, consumer)
// success
consumer, err = NewJSONRowConsumer(sampleSchema(), nil, 2, 16, nil)
assert.NotNil(t, consumer)
func Test_GetKeyValue(t *testing.T) {
fieldName := "dummy"
var obj1 interface{} = "aa"
val, err := getKeyValue(obj1, fieldName, true)
assert.Equal(t, val, "aa")
assert.NoError(t, err)
val, err = getKeyValue(obj1, fieldName, false)
assert.Empty(t, val)
assert.Error(t, err)
var obj2 interface{} = json.Number("10")
val, err = getKeyValue(obj2, fieldName, false)
assert.Equal(t, val, "10")
assert.NoError(t, err)
val, err = getKeyValue(obj2, fieldName, true)
assert.Empty(t, val)
assert.Error(t, err)
}
func Test_JSONRowConsumerFlush(t *testing.T) {
var callTime int32
var totalCount int
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shard int) error {
callTime++
field, ok := fields[101]
assert.True(t, ok)
assert.Greater(t, field.RowNum(), 0)
totalCount += field.RowNum()
return nil
}
schema := &schemapb.CollectionSchema{
Name: "schema",
AutoID: true,
Fields: []*schemapb.FieldSchema{
{
FieldID: 101,
Name: "uid",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
},
},
}
var shardNum int32 = 4
var blockSize int64 = 1
consumer, err := NewJSONRowConsumer(schema, nil, shardNum, blockSize, flushFunc)
assert.NotNil(t, consumer)
assert.NoError(t, err)
// force flush
rowCountEachShard := 100
for i := 0; i < int(shardNum); i++ {
pkFieldData := consumer.segmentsData[i][101].(*storage.Int64FieldData)
for j := 0; j < rowCountEachShard; j++ {
pkFieldData.Data = append(pkFieldData.Data, int64(j))
}
}
err = consumer.flush(true)
assert.NoError(t, err)
assert.Equal(t, shardNum, callTime)
assert.Equal(t, rowCountEachShard*int(shardNum), totalCount)
assert.Equal(t, 0, len(consumer.IDRange())) // not auto-generated id, no id range
// execeed block size trigger flush
callTime = 0
totalCount = 0
for i := 0; i < int(shardNum); i++ {
consumer.segmentsData[i] = initSegmentData(schema)
if i%2 == 0 {
continue
}
pkFieldData := consumer.segmentsData[i][101].(*storage.Int64FieldData)
for j := 0; j < rowCountEachShard; j++ {
pkFieldData.Data = append(pkFieldData.Data, int64(j))
}
}
err = consumer.flush(true)
assert.NoError(t, err)
assert.Equal(t, shardNum/2, callTime)
assert.Equal(t, rowCountEachShard*int(shardNum)/2, totalCount)
assert.Equal(t, 0, len(consumer.IDRange())) // not auto-generated id, no id range
}
func Test_JSONRowConsumerHandle(t *testing.T) {
func Test_JSONRowConsumerNew(t *testing.T) {
ctx := context.Background()
idAllocator := newIDAllocator(ctx, t, errors.New("error"))
var callTime int32
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shard int) error {
callTime++
return errors.New("dummy error")
}
schema := &schemapb.CollectionSchema{
Name: "schema",
AutoID: true,
Fields: []*schemapb.FieldSchema{
{
FieldID: 101,
Name: "uid",
IsPrimaryKey: true,
AutoID: true,
DataType: schemapb.DataType_Int64,
},
},
}
var consumer *JSONRowConsumer
err := consumer.Handle(nil)
assert.Error(t, err)
t.Run("handle int64 pk", func(t *testing.T) {
consumer, err := NewJSONRowConsumer(schema, idAllocator, 1, 1, flushFunc)
assert.NotNil(t, consumer)
assert.NoError(t, err)
pkFieldData := consumer.segmentsData[0][101].(*storage.Int64FieldData)
for i := 0; i < 10; i++ {
pkFieldData.Data = append(pkFieldData.Data, int64(i))
}
// nil input will trigger flush
err = consumer.Handle(nil)
assert.Error(t, err)
assert.Equal(t, int32(1), callTime)
// optional flush
callTime = 0
rowCount := 100
pkFieldData = consumer.segmentsData[0][101].(*storage.Int64FieldData)
for j := 0; j < rowCount; j++ {
pkFieldData.Data = append(pkFieldData.Data, int64(j))
}
input := make([]map[storage.FieldID]interface{}, rowCount)
for j := 0; j < rowCount; j++ {
input[j] = make(map[int64]interface{})
input[j][101] = int64(j)
}
err = consumer.Handle(input)
assert.Error(t, err)
assert.Equal(t, int32(1), callTime)
// failed to auto-generate pk
consumer.blockSize = 1024 * 1024
err = consumer.Handle(input)
assert.Error(t, err)
// hash int64 pk
consumer.rowIDAllocator = newIDAllocator(ctx, t, nil)
err = consumer.Handle(input)
assert.NoError(t, err)
assert.Equal(t, int64(rowCount), consumer.rowCounter)
assert.Equal(t, 2, len(consumer.autoIDRange))
assert.Equal(t, int64(1), consumer.autoIDRange[0])
assert.Equal(t, int64(1+rowCount), consumer.autoIDRange[1])
// pk is auto-generated but IDAllocator is nil
consumer.rowIDAllocator = nil
err = consumer.Handle(input)
assert.Error(t, err)
// pk is not auto-generated, pk is not numeric value
input = make([]map[storage.FieldID]interface{}, 1)
input[0] = make(map[int64]interface{})
input[0][101] = "1"
schema.Fields[0].AutoID = false
consumer, err = NewJSONRowConsumer(schema, idAllocator, 1, 1, flushFunc)
assert.NotNil(t, consumer)
assert.NoError(t, err)
err = consumer.Handle(input)
assert.Error(t, err)
// pk is numeric value, but cannot parsed
input[0][101] = json.Number("A1")
err = consumer.Handle(input)
t.Run("nil schema", func(t *testing.T) {
consumer, err := NewJSONRowConsumer(ctx, nil, nil, 16, nil)
assert.Error(t, err)
assert.Nil(t, consumer)
})
t.Run("handle varchar pk", func(t *testing.T) {
schema = &schemapb.CollectionSchema{
t.Run("wrong schema", func(t *testing.T) {
schema := &schemapb.CollectionSchema{
Name: "schema",
AutoID: true,
Fields: []*schemapb.FieldSchema{
{
FieldID: 101,
Name: "uid",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
},
},
}
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
schema.Fields[0].DataType = schemapb.DataType_None
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
assert.Error(t, err)
assert.Nil(t, consumer)
})
t.Run("primary key is autoid but no IDAllocator", func(t *testing.T) {
schema := &schemapb.CollectionSchema{
Name: "schema",
AutoID: true,
Fields: []*schemapb.FieldSchema{
@ -279,67 +124,543 @@ func Test_JSONRowConsumerHandle(t *testing.T) {
Name: "uid",
IsPrimaryKey: true,
AutoID: true,
DataType: schemapb.DataType_VarChar,
TypeParams: []*commonpb.KeyValuePair{
{Key: common.MaxLengthKey, Value: "1024"},
},
DataType: schemapb.DataType_Int64,
},
},
}
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
assert.Error(t, err)
assert.Nil(t, consumer)
})
t.Run("succeed", func(t *testing.T) {
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
assert.NotNil(t, consumer)
assert.NoError(t, err)
})
}
func Test_JSONRowConsumerHandleIntPK(t *testing.T) {
ctx := context.Background()
t.Run("nil input", func(t *testing.T) {
var consumer *JSONRowConsumer
err := consumer.Handle(nil)
assert.Error(t, err)
})
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
FieldID: 101,
Name: "FieldInt64",
IsPrimaryKey: true,
AutoID: true,
DataType: schemapb.DataType_Int64,
},
{
FieldID: 102,
Name: "FieldVarchar",
DataType: schemapb.DataType_VarChar,
},
{
FieldID: 103,
Name: "FieldFloat",
DataType: schemapb.DataType_Float,
},
},
}
createConsumeFunc := func(shardNum int32, partitionIDs []int64, flushFunc ImportFlushFunc) *JSONRowConsumer {
collectionInfo, err := NewCollectionInfo(schema, shardNum, partitionIDs)
assert.NoError(t, err)
idAllocator := newIDAllocator(ctx, t, nil)
consumer, err := NewJSONRowConsumer(schema, idAllocator, 1, 1024*1024, flushFunc)
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, idAllocator, 1, flushFunc)
assert.NotNil(t, consumer)
assert.NoError(t, err)
rowCount := 100
input := make([]map[storage.FieldID]interface{}, rowCount)
for j := 0; j < rowCount; j++ {
input[j] = make(map[int64]interface{})
input[j][101] = "abc"
return consumer
}
t.Run("auto pk no partition key", func(t *testing.T) {
flushErrFunc := func(fields BlockData, shard int, partID int64) error {
return errors.New("dummy error")
}
// varchar pk cannot be auto-generated
// rows to input
intputRowCount := 100
input := make([]map[storage.FieldID]interface{}, intputRowCount)
for j := 0; j < intputRowCount; j++ {
input[j] = map[int64]interface{}{
102: "string",
103: json.Number("6.18"),
}
}
shardNum := int32(2)
partitionID := int64(1)
consumer := createConsumeFunc(shardNum, []int64{partitionID}, flushErrFunc)
consumer.rowIDAllocator = newIDAllocator(ctx, t, errors.New("error"))
waitFlushRowCount := 10
fieldsData := createFieldsData(schema, waitFlushRowCount)
consumer.shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
// nil input will trigger force flush, flushErrFunc returns error
err := consumer.Handle(nil)
assert.Error(t, err)
// optional flush, flushErrFunc returns error
err = consumer.Handle(input)
assert.Error(t, err)
// hash varchar pk
schema.Fields[0].AutoID = false
consumer, err = NewJSONRowConsumer(schema, idAllocator, 1, 1024*1024, flushFunc)
assert.NotNil(t, consumer)
// reset flushFunc
var callTime int32
var flushedRowCount int
consumer.callFlushFunc = func(fields BlockData, shard int, partID int64) error {
callTime++
assert.Less(t, int32(shard), shardNum)
assert.Equal(t, partitionID, partID)
assert.Greater(t, len(fields), 0)
for _, v := range fields {
assert.Greater(t, v.RowNum(), 0)
}
flushedRowCount += fields[102].RowNum()
return nil
}
// optional flush succeed, each shard has 10 rows, idErrAllocator returns error
err = consumer.Handle(input)
assert.Error(t, err)
assert.Equal(t, waitFlushRowCount*int(shardNum), flushedRowCount)
assert.Equal(t, shardNum, callTime)
// optional flush again, large blockSize, nothing flushed, idAllocator returns error
callTime = int32(0)
flushedRowCount = 0
consumer.shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
consumer.rowIDAllocator = nil
consumer.blockSize = 8 * 1024 * 1024
err = consumer.Handle(input)
assert.Error(t, err)
assert.Equal(t, 0, flushedRowCount)
assert.Equal(t, int32(0), callTime)
// idAllocator is ok, consume 100 rows, the previous shardsData(10 rows per shard) is flushed
callTime = int32(0)
flushedRowCount = 0
consumer.blockSize = 1
consumer.rowIDAllocator = newIDAllocator(ctx, t, nil)
err = consumer.Handle(input)
assert.NoError(t, err)
assert.Equal(t, waitFlushRowCount*int(shardNum), flushedRowCount)
assert.Equal(t, shardNum, callTime)
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
assert.Equal(t, 2, len(consumer.IDRange()))
assert.Equal(t, int64(1), consumer.IDRange()[0])
assert.Equal(t, int64(1+intputRowCount), consumer.IDRange()[1])
// call handle again, the 100 rows are flushed
callTime = int32(0)
flushedRowCount = 0
err = consumer.Handle(nil)
assert.NoError(t, err)
assert.Equal(t, intputRowCount, flushedRowCount)
assert.Equal(t, shardNum, callTime)
})
schema.Fields[0].AutoID = false
t.Run("manual pk no partition key", func(t *testing.T) {
shardNum := int32(1)
partitionID := int64(100)
var callTime int32
var flushedRowCount int
flushFunc := func(fields BlockData, shard int, partID int64) error {
callTime++
assert.Less(t, int32(shard), shardNum)
assert.Equal(t, partitionID, partID)
assert.Greater(t, len(fields), 0)
flushedRowCount += fields[102].RowNum()
return nil
}
consumer := createConsumeFunc(shardNum, []int64{partitionID}, flushFunc)
// failed to parse primary key
input := make([]map[storage.FieldID]interface{}, 1)
input[0] = map[int64]interface{}{
101: int64(99),
102: "string",
103: 11.11,
}
err := consumer.Handle(input)
assert.Error(t, err)
// failed to convert pk to int value
input[0] = map[int64]interface{}{
101: json.Number("a"),
102: "string",
103: 11.11,
}
err = consumer.Handle(input)
assert.Error(t, err)
// failed to hash to partition
input[0] = map[int64]interface{}{
101: json.Number("99"),
102: "string",
103: json.Number("4.56"),
}
consumer.collectionInfo.PartitionIDs = nil
err = consumer.Handle(input)
assert.Error(t, err)
consumer.collectionInfo.PartitionIDs = []int64{partitionID}
// failed to convert value
input[0] = map[int64]interface{}{
101: json.Number("99"),
102: "string",
103: json.Number("abc.56"),
}
err = consumer.Handle(input)
assert.Error(t, err)
consumer.shardsData = createShardsData(schema, nil, shardNum, []int64{partitionID}) // in-memory data is dirty, reset
// succeed, consume 1 row
input[0] = map[int64]interface{}{
101: json.Number("99"),
102: "string",
103: json.Number("4.56"),
}
err = consumer.Handle(input)
assert.NoError(t, err)
assert.Equal(t, int64(rowCount), consumer.RowCount())
assert.Equal(t, 0, len(consumer.autoIDRange))
assert.Equal(t, int64(1), consumer.RowCount())
assert.Equal(t, 0, len(consumer.IDRange()))
// pk is not string value
input = make([]map[storage.FieldID]interface{}, 1)
input[0] = make(map[int64]interface{})
input[0][101] = false
// call handle again, the 1 row is flushed
callTime = int32(0)
flushedRowCount = 0
err = consumer.Handle(nil)
assert.NoError(t, err)
assert.Equal(t, 1, flushedRowCount)
assert.Equal(t, shardNum, callTime)
})
schema.Fields[1].IsPartitionKey = true
t.Run("manual pk with partition key", func(t *testing.T) {
// 10 partitions
partitionIDs := make([]int64, 0)
for j := 0; j < 10; j++ {
partitionIDs = append(partitionIDs, int64(j))
}
shardNum := int32(2)
var flushedRowCount int
flushFunc := func(fields BlockData, shard int, partID int64) error {
assert.Less(t, int32(shard), shardNum)
assert.Contains(t, partitionIDs, partID)
assert.Greater(t, len(fields), 0)
flushedRowCount += fields[102].RowNum()
return nil
}
consumer := createConsumeFunc(shardNum, partitionIDs, flushFunc)
// rows to input
intputRowCount := 100
input := make([]map[storage.FieldID]interface{}, intputRowCount)
for j := 0; j < intputRowCount; j++ {
input[j] = map[int64]interface{}{
101: json.Number(strconv.Itoa(j)),
102: "partitionKey_" + strconv.Itoa(j),
103: json.Number("6.18"),
}
}
// 100 rows are consumed to different partitions
err := consumer.Handle(input)
assert.NoError(t, err)
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
// call handle again, 100 rows are flushed
flushedRowCount = 0
err = consumer.Handle(nil)
assert.NoError(t, err)
assert.Equal(t, intputRowCount, flushedRowCount)
})
}
func Test_JSONRowConsumerHandleVarcharPK(t *testing.T) {
ctx := context.Background()
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
FieldID: 101,
Name: "FieldVarchar",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_VarChar,
},
{
FieldID: 102,
Name: "FieldInt64",
DataType: schemapb.DataType_Int64,
},
{
FieldID: 103,
Name: "FieldFloat",
DataType: schemapb.DataType_Float,
},
},
}
createConsumeFunc := func(shardNum int32, partitionIDs []int64, flushFunc ImportFlushFunc) *JSONRowConsumer {
collectionInfo, err := NewCollectionInfo(schema, shardNum, partitionIDs)
assert.NoError(t, err)
idAllocator := newIDAllocator(ctx, t, nil)
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, idAllocator, 1, flushFunc)
assert.NotNil(t, consumer)
assert.NoError(t, err)
return consumer
}
t.Run("no partition key", func(t *testing.T) {
shardNum := int32(2)
partitionID := int64(1)
var callTime int32
var flushedRowCount int
flushFunc := func(fields BlockData, shard int, partID int64) error {
callTime++
assert.Less(t, int32(shard), shardNum)
assert.Equal(t, partitionID, partID)
assert.Greater(t, len(fields), 0)
for _, v := range fields {
assert.Greater(t, v.RowNum(), 0)
}
flushedRowCount += fields[102].RowNum()
return nil
}
consumer := createConsumeFunc(shardNum, []int64{partitionID}, flushFunc)
consumer.shardsData = createShardsData(schema, nil, shardNum, []int64{partitionID})
// string type primary key cannot be auto-generated
input := make([]map[storage.FieldID]interface{}, 1)
input[0] = map[int64]interface{}{
101: true,
102: json.Number("1"),
103: json.Number("1.56"),
}
consumer.collectionInfo.PrimaryKey.AutoID = true
err := consumer.Handle(input)
assert.Error(t, err)
consumer.collectionInfo.PrimaryKey.AutoID = false
// failed to parse primary key
err = consumer.Handle(input)
assert.Error(t, err)
// failed to hash to partition
input[0] = map[int64]interface{}{
101: "primaryKey_0",
102: json.Number("1"),
103: json.Number("1.56"),
}
consumer.collectionInfo.PartitionIDs = nil
err = consumer.Handle(input)
assert.Error(t, err)
consumer.collectionInfo.PartitionIDs = []int64{partitionID}
// rows to input
intputRowCount := 100
input = make([]map[storage.FieldID]interface{}, intputRowCount)
for j := 0; j < intputRowCount; j++ {
input[j] = map[int64]interface{}{
101: "primaryKey_" + strconv.Itoa(j),
102: json.Number(strconv.Itoa(j)),
103: json.Number("0.618"),
}
}
// rows are consumed
err = consumer.Handle(input)
assert.NoError(t, err)
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
assert.Equal(t, 0, len(consumer.IDRange()))
// call handle again, 100 rows are flushed
err = consumer.Handle(nil)
assert.NoError(t, err)
assert.Equal(t, intputRowCount, flushedRowCount)
assert.Equal(t, shardNum, callTime)
})
schema.Fields[1].IsPartitionKey = true
t.Run("has partition key", func(t *testing.T) {
// 10 partitions
partitionIDs := make([]int64, 0)
for j := 0; j < 10; j++ {
partitionIDs = append(partitionIDs, int64(j))
}
shardNum := int32(2)
var flushedRowCount int
flushFunc := func(fields BlockData, shard int, partID int64) error {
assert.Less(t, int32(shard), shardNum)
assert.Contains(t, partitionIDs, partID)
assert.Greater(t, len(fields), 0)
flushedRowCount += fields[102].RowNum()
return nil
}
consumer := createConsumeFunc(shardNum, partitionIDs, flushFunc)
// rows to input
intputRowCount := 100
input := make([]map[storage.FieldID]interface{}, intputRowCount)
for j := 0; j < intputRowCount; j++ {
input[j] = map[int64]interface{}{
101: "primaryKey_" + strconv.Itoa(j),
102: json.Number(strconv.Itoa(j)),
103: json.Number("0.618"),
}
}
// 100 rows are consumed to different partitions
err := consumer.Handle(input)
assert.NoError(t, err)
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
// call handle again, 100 rows are flushed
flushedRowCount = 0
err = consumer.Handle(nil)
assert.NoError(t, err)
assert.Equal(t, intputRowCount, flushedRowCount)
// string type primary key cannot be auto-generated
consumer.validators[101].autoID = true
err = consumer.Handle(input)
assert.Error(t, err)
})
}
func Test_GetPrimaryKey(t *testing.T) {
fieldName := "dummy"
var obj1 interface{} = "aa"
val, err := getPrimaryKey(obj1, fieldName, true)
assert.Equal(t, val, "aa")
func Test_JSONRowHashToPartition(t *testing.T) {
ctx := context.Background()
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
FieldID: 100,
Name: "ID",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
},
{
FieldID: 101,
Name: "FieldVarchar",
DataType: schemapb.DataType_VarChar,
},
{
FieldID: 102,
Name: "FieldInt64",
DataType: schemapb.DataType_Int64,
},
},
}
partitionID := int64(1)
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{partitionID})
assert.NoError(t, err)
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
assert.NoError(t, err)
assert.NotNil(t, consumer)
input := make(map[int64]interface{})
input[100] = int64(1)
input[101] = "abc"
input[102] = int64(100)
t.Run("no partition key", func(t *testing.T) {
partID, err := consumer.hashToPartition(input, 0)
assert.NoError(t, err)
assert.Equal(t, partitionID, partID)
})
t.Run("partition list is empty", func(t *testing.T) {
collectionInfo.PartitionIDs = []int64{}
partID, err := consumer.hashToPartition(input, 0)
assert.Error(t, err)
assert.Equal(t, int64(0), partID)
collectionInfo.PartitionIDs = []int64{partitionID}
})
schema.Fields[1].IsPartitionKey = true
err = collectionInfo.resetSchema(schema)
assert.NoError(t, err)
collectionInfo.PartitionIDs = []int64{1, 2, 3}
t.Run("varchar partition key", func(t *testing.T) {
input := make(map[int64]interface{})
input[100] = int64(1)
input[101] = true
input[102] = int64(100)
// getKeyValue failed
partID, err := consumer.hashToPartition(input, 0)
assert.Error(t, err)
assert.Equal(t, int64(0), partID)
// succeed
input[101] = "abc"
partID, err = consumer.hashToPartition(input, 0)
assert.NoError(t, err)
assert.Contains(t, collectionInfo.PartitionIDs, partID)
})
schema.Fields[1].IsPartitionKey = false
schema.Fields[2].IsPartitionKey = true
err = collectionInfo.resetSchema(schema)
assert.NoError(t, err)
val, err = getPrimaryKey(obj1, fieldName, false)
assert.Empty(t, val)
assert.Error(t, err)
t.Run("int64 partition key", func(t *testing.T) {
input := make(map[int64]interface{})
input[100] = int64(1)
input[101] = "abc"
input[102] = 100
var obj2 interface{} = json.Number("10")
val, err = getPrimaryKey(obj2, fieldName, false)
assert.Equal(t, val, "10")
assert.NoError(t, err)
// getKeyValue failed
partID, err := consumer.hashToPartition(input, 0)
assert.Error(t, err)
assert.Equal(t, int64(0), partID)
val, err = getPrimaryKey(obj2, fieldName, true)
assert.Empty(t, val)
assert.Error(t, err)
// parse int failed
input[102] = json.Number("d")
partID, err = consumer.hashToPartition(input, 0)
assert.Error(t, err)
assert.Equal(t, int64(0), partID)
// succeed
input[102] = json.Number("100")
partID, err = consumer.hashToPartition(input, 0)
assert.NoError(t, err)
assert.Contains(t, collectionInfo.PartitionIDs, partID)
})
}

View File

@ -27,7 +27,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/typeutil"
"go.uber.org/zap"
@ -45,42 +44,21 @@ type IOReader struct {
}
type JSONParser struct {
ctx context.Context // for canceling parse process
bufRowCount int // max rows in a buffer
name2FieldID map[string]storage.FieldID
ctx context.Context // for canceling parse process
collectionInfo *CollectionInfo // collection details including schema
bufRowCount int // max rows in a buffer
updateProgressFunc func(percent int64) // update working progress percent value
dynamicFieldID storage.FieldID // dynamic field id, set to -1 if no dynamic field
}
// NewJSONParser helper function to create a JSONParser
func NewJSONParser(ctx context.Context, collectionSchema *schemapb.CollectionSchema, updateProgressFunc func(percent int64)) *JSONParser {
name2FieldID := make(map[string]storage.FieldID)
dynamicFieldID := int64(-1)
for i := 0; i < len(collectionSchema.Fields); i++ {
schema := collectionSchema.Fields[i]
// RowIDField and TimeStampField is internal field, no need to parse
if schema.GetFieldID() == common.RowIDField || schema.GetFieldID() == common.TimeStampField {
continue
}
// if primary key field is auto-gernerated, no need to parse
if schema.GetAutoID() {
continue
}
name2FieldID[schema.GetName()] = schema.GetFieldID()
if schema.GetIsDynamic() && collectionSchema.GetEnableDynamicField() {
dynamicFieldID = schema.GetFieldID()
}
}
func NewJSONParser(ctx context.Context, collectionInfo *CollectionInfo, updateProgressFunc func(percent int64)) *JSONParser {
parser := &JSONParser{
ctx: ctx,
collectionInfo: collectionInfo,
bufRowCount: 1024,
name2FieldID: name2FieldID,
updateProgressFunc: updateProgressFunc,
dynamicFieldID: dynamicFieldID,
}
adjustBufSize(parser, collectionSchema)
adjustBufSize(parser, collectionInfo.Schema)
return parser
}
@ -112,43 +90,50 @@ func adjustBufSize(parser *JSONParser, collectionSchema *schemapb.CollectionSche
}
func (p *JSONParser) combineDynamicRow(dynamicValues map[string]interface{}, row map[storage.FieldID]interface{}) error {
if p.dynamicFieldID < 0 {
if p.collectionInfo.DynamicField == nil {
return nil
}
dynamicFieldID := p.collectionInfo.DynamicField.GetFieldID()
// combine the dynamic field value
// valid input:
// case 1: {"id": 1, "vector": [], "x": 8, "$meta": "{\"y\": 8}"}
// case 2: {"id": 1, "vector": [], "x": 8, "$meta": {}}
// case 1: {"id": 1, "vector": [], "x": 8, "$meta": "{\"y\": 8}"} ==>> {"id": 1, "vector": [], "$meta": "{\"y\": 8, \"x\": 8}"}
// case 2: {"id": 1, "vector": [], "x": 8, "$meta": {}} ==>> {"id": 1, "vector": [], "$meta": {\"x\": 8}}
// case 3: {"id": 1, "vector": [], "$meta": "{\"x\": 8}"}
// case 4: {"id": 1, "vector": [], "$meta": {"x": 8}}
// case 5: {"id": 1, "vector": [], "$meta": {}}
// case 6: {"id": 1, "vector": [], "x": 8}
// case 6: {"id": 1, "vector": [], "x": 8} ==>> {"id": 1, "vector": [], "$meta": "{\"x\": 8}"}
// case 7: {"id": 1, "vector": []}
obj, ok := row[p.dynamicFieldID]
obj, ok := row[dynamicFieldID]
if ok {
if len(dynamicValues) > 0 {
if value, is := obj.(string); is {
// case 1
mp := make(map[string]interface{})
json.Unmarshal([]byte(value), &mp)
err := json.Unmarshal([]byte(value), &mp)
if err != nil {
// invalid input
return errors.New("illegal value for dynamic field, not a JSON format string")
}
maps.Copy(dynamicValues, mp)
} else if mp, is := obj.(map[string]interface{}); is {
// case 2
maps.Copy(dynamicValues, mp)
} else {
// invalid input
return errors.New("illegal value for dynamic field")
return errors.New("illegal value for dynamic field, not a JSON object")
}
row[p.dynamicFieldID] = dynamicValues
row[dynamicFieldID] = dynamicValues
}
// else case 3/4/5
} else {
if len(dynamicValues) > 0 {
// case 6
row[p.dynamicFieldID] = dynamicValues
row[dynamicFieldID] = dynamicValues
} else {
// case 7
row[p.dynamicFieldID] = "{}"
row[dynamicFieldID] = "{}"
}
}
@ -158,36 +143,50 @@ func (p *JSONParser) combineDynamicRow(dynamicValues map[string]interface{}, row
func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{}, error) {
stringMap, ok := raw.(map[string]interface{})
if !ok {
log.Error("JSON parser: invalid JSON format, each row should be a key-value map")
log.Warn("JSON parser: invalid JSON format, each row should be a key-value map")
return nil, errors.New("invalid JSON format, each row should be a key-value map")
}
dynamicValues := make(map[string]interface{})
row := make(map[storage.FieldID]interface{})
// some fields redundant?
for k, v := range stringMap {
fieldID, ok := p.name2FieldID[k]
fieldID, ok := p.collectionInfo.Name2FieldID[k]
if (fieldID == p.collectionInfo.PrimaryKey.GetFieldID()) && p.collectionInfo.PrimaryKey.GetAutoID() {
// primary key is auto-id, no need to provide
log.Warn("JSON parser: the primary key is auto-generated, no need to provide", zap.String("fieldName", k))
return nil, fmt.Errorf("the primary key '%s' is auto-generated, no need to provide", k)
}
if ok {
row[fieldID] = v
} else if p.dynamicFieldID >= 0 {
} else if p.collectionInfo.DynamicField != nil {
// has dynamic field. put redundant pair to dynamicValues
dynamicValues[k] = v
} else {
// no dynamic field. if user provided redundant field, return error
log.Error("JSON parser: the field is not defined in collection schema", zap.String("fieldName", k))
log.Warn("JSON parser: the field is not defined in collection schema", zap.String("fieldName", k))
return nil, fmt.Errorf("the field '%s' is not defined in collection schema", k)
}
}
// some fields not provided?
if len(row) != len(p.name2FieldID) {
for k, v := range p.name2FieldID {
if v == p.dynamicFieldID {
// dyanmic field, allow user ignore this field
if len(row) != len(p.collectionInfo.Name2FieldID) {
for k, v := range p.collectionInfo.Name2FieldID {
if (p.collectionInfo.DynamicField != nil) && (v == p.collectionInfo.DynamicField.GetFieldID()) {
// ignore dyanmic field, user don't have to provide values for dynamic field
continue
}
if v == p.collectionInfo.PrimaryKey.GetFieldID() && p.collectionInfo.PrimaryKey.GetAutoID() {
// ignore auto-generaed primary key
continue
}
_, ok := row[v]
if !ok {
log.Error("JSON parser: a field value is missed", zap.String("fieldName", k))
// not auto-id primary key, no dynamic field, must provide value
log.Warn("JSON parser: a field value is missed", zap.String("fieldName", k))
return nil, fmt.Errorf("value of field '%s' is missed", k)
}
}
@ -195,12 +194,17 @@ func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{}
// combine the redundant pairs into dunamic field(if has)
err := p.combineDynamicRow(dynamicValues, row)
if err != nil {
log.Warn("JSON parser: failed to combine dynamic values", zap.Error(err))
return nil, err
}
return row, err
}
func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
if handler == nil || reader == nil {
log.Error("JSON parse handler is nil")
log.Warn("JSON parse handler is nil")
return errors.New("JSON parse handler is nil")
}
@ -225,11 +229,11 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
dec.UseNumber()
t, err := dec.Token()
if err != nil {
log.Error("JSON parser: failed to decode the JSON file", zap.Error(err))
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
return fmt.Errorf("failed to decode the JSON file, error: %w", err)
}
if t != json.Delim('{') {
log.Error("JSON parser: invalid JSON format, the content should be started with'{'")
log.Warn("JSON parser: invalid JSON format, the content should be started with'{'")
return errors.New("invalid JSON format, the content should be started with'{'")
}
@ -239,26 +243,26 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
// read the key
t, err := dec.Token()
if err != nil {
log.Error("JSON parser: failed to decode the JSON file", zap.Error(err))
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
return fmt.Errorf("failed to decode the JSON file, error: %w", err)
}
key := t.(string)
keyLower := strings.ToLower(key)
// the root key should be RowRootNode
if keyLower != RowRootNode {
log.Error("JSON parser: invalid JSON format, the root key is not found", zap.String("RowRootNode", RowRootNode), zap.String("key", key))
log.Warn("JSON parser: invalid JSON format, the root key is not found", zap.String("RowRootNode", RowRootNode), zap.String("key", key))
return fmt.Errorf("invalid JSON format, the root key should be '%s', but get '%s'", RowRootNode, key)
}
// started by '['
t, err = dec.Token()
if err != nil {
log.Error("JSON parser: failed to decode the JSON file", zap.Error(err))
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
return fmt.Errorf("failed to decode the JSON file, error: %w", err)
}
if t != json.Delim('[') {
log.Error("JSON parser: invalid JSON format, rows list should begin with '['")
log.Warn("JSON parser: invalid JSON format, rows list should begin with '['")
return errors.New("invalid JSON format, rows list should begin with '['")
}
@ -267,7 +271,7 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
for dec.More() {
var value interface{}
if err := dec.Decode(&value); err != nil {
log.Error("JSON parser: failed to parse row value", zap.Error(err))
log.Warn("JSON parser: failed to parse row value", zap.Error(err))
return fmt.Errorf("failed to parse row value, error: %w", err)
}
@ -282,7 +286,7 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
if len(buf) >= p.bufRowCount {
isEmpty = false
if err = handler.Handle(buf); err != nil {
log.Error("JSON parser: failed to convert row value to entity", zap.Error(err))
log.Warn("JSON parser: failed to convert row value to entity", zap.Error(err))
return fmt.Errorf("failed to convert row value to entity, error: %w", err)
}
@ -295,7 +299,7 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
if len(buf) > 0 {
isEmpty = false
if err = handler.Handle(buf); err != nil {
log.Error("JSON parser: failed to convert row value to entity", zap.Error(err))
log.Warn("JSON parser: failed to convert row value to entity", zap.Error(err))
return fmt.Errorf("failed to convert row value to entity, error: %w", err)
}
}
@ -303,18 +307,18 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
// end by ']'
t, err = dec.Token()
if err != nil {
log.Error("JSON parser: failed to decode the JSON file", zap.Error(err))
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
return fmt.Errorf("failed to decode the JSON file, error: %w", err)
}
if t != json.Delim(']') {
log.Error("JSON parser: invalid JSON format, rows list should end with a ']'")
log.Warn("JSON parser: invalid JSON format, rows list should end with a ']'")
return errors.New("invalid JSON format, rows list should end with a ']'")
}
// outside context might be canceled(service stop, or future enhancement for canceling import task)
if isCanceled(p.ctx) {
log.Error("JSON parser: import task was canceled")
log.Warn("JSON parser: import task was canceled")
return errors.New("import task was canceled")
}
@ -323,9 +327,10 @@ func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
break
}
// empty file is allowed, don't return error
if isEmpty {
log.Error("JSON parser: row count is 0")
return errors.New("row count is 0")
log.Info("JSON parser: row count is 0")
return nil
}
updateProgress()

View File

@ -58,7 +58,9 @@ func Test_AdjustBufSize(t *testing.T) {
// small row
schema := sampleSchema()
parser := NewJSONParser(ctx, schema, nil)
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
parser := NewJSONParser(ctx, collectionInfo, nil)
assert.NotNil(t, parser)
assert.Greater(t, parser.bufRowCount, 0)
@ -66,7 +68,7 @@ func Test_AdjustBufSize(t *testing.T) {
schema.Fields[9].TypeParams = []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "32768"},
}
parser = NewJSONParser(ctx, schema, nil)
parser = NewJSONParser(ctx, collectionInfo, nil)
assert.NotNil(t, parser)
assert.Greater(t, parser.bufRowCount, 0)
@ -77,9 +79,11 @@ func Test_AdjustBufSize(t *testing.T) {
AutoID: true,
Fields: []*schemapb.FieldSchema{},
}
parser = NewJSONParser(ctx, schema, nil)
parser = NewJSONParser(ctx, collectionInfo, nil)
assert.NotNil(t, parser)
assert.Greater(t, parser.bufRowCount, 0)
adjustBufSize(parser, schema)
}
func Test_JSONParserParseRows_IntPK(t *testing.T) {
@ -87,7 +91,9 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
defer cancel()
schema := sampleSchema()
parser := NewJSONParser(ctx, schema, nil)
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
parser := NewJSONParser(ctx, collectionInfo, nil)
assert.NotNil(t, parser)
// prepare test data
@ -188,6 +194,18 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
assert.InDelta(t, contenctRow.FieldFloatVector[k], float32(fval), 10e-6)
}
}
// empty content
reader = strings.NewReader(`{}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(2)}, consumer)
assert.NoError(t, err)
// row count is 0
reader = strings.NewReader(`{
"rows":[]
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
assert.NoError(t, err)
})
t.Run("error cases", func(t *testing.T) {
@ -195,11 +213,15 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(0)}, nil)
assert.Error(t, err)
// not a valid JSON format
reader = strings.NewReader(`{[]`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
assert.Error(t, err)
// not a row-based format
reader = strings.NewReader(`{
"dummy":[]
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
assert.Error(t, err)
@ -207,7 +229,6 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
reader = strings.NewReader(`{
"rows":
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(5)}, consumer)
assert.Error(t, err)
@ -215,7 +236,6 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
reader = strings.NewReader(`{
"rows": [}
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(6)}, consumer)
assert.Error(t, err)
@ -223,7 +243,6 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
reader = strings.NewReader(`{
"rows": {}
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(8)}, consumer)
assert.Error(t, err)
@ -231,32 +250,30 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
reader = strings.NewReader(`{
"rows": [[]]
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
assert.Error(t, err)
// typo
reader = strings.NewReader(`{
"rows": ["]
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
assert.Error(t, err)
// not valid json format
reader = strings.NewReader(`[]`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(2)}, consumer)
assert.Error(t, err)
// empty content
reader = strings.NewReader(`{}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(2)}, consumer)
assert.Error(t, err)
// empty content
// empty file
reader = strings.NewReader(``)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(0)}, consumer)
assert.Error(t, err)
// redundant field
reader = strings.NewReader(`{
"rows":[
{"dummy": 1, "FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]}
{"dummy": 1, "FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 10, "b": true}}
]
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
@ -265,7 +282,7 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
// field missed
reader = strings.NewReader(`{
"rows":[
{"FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]}
{"FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 10, "b": true}}
]
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
@ -274,9 +291,9 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
// handle() error
content := `{
"rows":[
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]},
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]},
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4]}
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 7, "b": true}},
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 8, "b": false}},
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 9, "b": true}}
]
}`
consumer.handleErr = errors.New("error")
@ -290,13 +307,6 @@ func Test_JSONParserParseRows_IntPK(t *testing.T) {
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
assert.Error(t, err)
// row count is 0
reader = strings.NewReader(`{
"rows":[]
}`)
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
assert.Error(t, err)
// canceled
consumer.handleErr = nil
cancel()
@ -311,10 +321,12 @@ func Test_JSONParserParseRows_StrPK(t *testing.T) {
defer cancel()
schema := strKeySchema()
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
updateProgress := func(percent int64) {
assert.Greater(t, percent, int64(0))
}
parser := NewJSONParser(ctx, schema, updateProgress)
parser := NewJSONParser(ctx, collectionInfo, updateProgress)
assert.NotNil(t, parser)
// prepare test data
@ -413,7 +425,9 @@ func Test_JSONParserCombineDynamicRow(t *testing.T) {
},
},
}
parser := NewJSONParser(ctx, schema, nil)
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
parser := NewJSONParser(ctx, collectionInfo, nil)
assert.NotNil(t, parser)
// valid input:
@ -425,87 +439,101 @@ func Test_JSONParserCombineDynamicRow(t *testing.T) {
// case 6: {"id": 1, "vector": [], "x": 8}
// case 7: {"id": 1, "vector": []}
// case 1
dynamicValues := map[string]interface{}{
"x": 8,
}
row := map[storage.FieldID]interface{}{
106: 1,
113: "{\"y\": 8}",
}
err := parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
assert.Contains(t, row[113], "y")
t.Run("values combined for dynamic field", func(t *testing.T) {
dynamicValues := map[string]interface{}{
"x": 8,
}
row := map[storage.FieldID]interface{}{
106: 1,
113: "{\"y\": 8}",
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
assert.Contains(t, row[113], "y")
})
// case 2
dynamicValues = map[string]interface{}{
"x": 8,
}
row = map[storage.FieldID]interface{}{
106: 1,
113: map[string]interface{}{},
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
t.Run("outside value for dynamic field", func(t *testing.T) {
dynamicValues := map[string]interface{}{
"x": 8,
}
row := map[storage.FieldID]interface{}{
106: 1,
113: map[string]interface{}{},
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
})
// case 3/4/5
dynamicValues = map[string]interface{}{}
row = map[storage.FieldID]interface{}{
106: 1,
113: "{\"x\": 8}",
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
t.Run("JSON format string/object for dynamic field", func(t *testing.T) {
dynamicValues := map[string]interface{}{}
row := map[storage.FieldID]interface{}{
106: 1,
113: "{\"x\": 8}",
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
})
// case 6
dynamicValues = map[string]interface{}{
"x": 8,
}
row = map[storage.FieldID]interface{}{
106: 1,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
t.Run("dynamic field is hidden", func(t *testing.T) {
dynamicValues := map[string]interface{}{
"x": 8,
}
row := map[storage.FieldID]interface{}{
106: 1,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
})
// case 7
dynamicValues = map[string]interface{}{}
row = map[storage.FieldID]interface{}{
106: 1,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Equal(t, "{}", row[113])
t.Run("no values for dynamic field", func(t *testing.T) {
dynamicValues := map[string]interface{}{}
row := map[storage.FieldID]interface{}{
106: 1,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
assert.Equal(t, "{}", row[113])
})
// invalid input
dynamicValues = map[string]interface{}{
"x": 8,
}
row = map[storage.FieldID]interface{}{
106: 1,
113: 5,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.Error(t, err)
t.Run("invalid input for dynamic field", func(t *testing.T) {
dynamicValues := map[string]interface{}{
"x": 8,
}
row := map[storage.FieldID]interface{}{
106: 1,
113: 5,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.Error(t, err)
// no dynamic field
parser.dynamicFieldID = -1
dynamicValues = map[string]interface{}{
"x": 8,
}
row = map[storage.FieldID]interface{}{
106: 1,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.NotContains(t, row, int64(113))
row = map[storage.FieldID]interface{}{
106: 1,
113: "abc",
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.Error(t, err)
})
t.Run("not allow dynamic values if no dynamic field", func(t *testing.T) {
parser.collectionInfo.DynamicField = nil
dynamicValues := map[string]interface{}{
"x": 8,
}
row := map[storage.FieldID]interface{}{
106: 1,
}
err = parser.combineDynamicRow(dynamicValues, row)
assert.NoError(t, err)
assert.NotContains(t, row, int64(113))
})
}
func Test_JSONParserVerifyRow(t *testing.T) {
@ -535,39 +563,91 @@ func Test_JSONParserVerifyRow(t *testing.T) {
},
},
}
parser := NewJSONParser(ctx, schema, nil)
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
parser := NewJSONParser(ctx, collectionInfo, nil)
assert.NotNil(t, parser)
assert.Equal(t, int64(113), parser.dynamicFieldID)
// dynamic field provided
raw := map[string]interface{}{
"FieldID": 100,
"FieldDynamic": "{\"x\": 8}",
"y": true,
}
row, err := parser.verifyRow(raw)
t.Run("input is not key-value map", func(t *testing.T) {
_, err = parser.verifyRow(nil)
assert.Error(t, err)
_, err = parser.verifyRow([]int{0})
assert.Error(t, err)
})
t.Run("not auto-id, dynamic field provided", func(t *testing.T) {
raw := map[string]interface{}{
"FieldID": 100,
"FieldDynamic": "{\"x\": 8}",
"y": true,
}
row, err := parser.verifyRow(raw)
assert.NoError(t, err)
assert.Contains(t, row, int64(106))
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
assert.Contains(t, row[113], "y")
})
t.Run("not auto-id, dynamic field not provided", func(t *testing.T) {
raw := map[string]interface{}{
"FieldID": 100,
}
row, err := parser.verifyRow(raw)
assert.NoError(t, err)
assert.Contains(t, row, int64(106))
assert.Contains(t, row, int64(113))
assert.Equal(t, "{}", row[113])
})
t.Run("not auto-id, invalid input dynamic field", func(t *testing.T) {
raw := map[string]interface{}{
"FieldID": 100,
"FieldDynamic": true,
"y": true,
}
_, err = parser.verifyRow(raw)
assert.Error(t, err)
})
schema.Fields[0].AutoID = true
err = collectionInfo.resetSchema(schema)
assert.NoError(t, err)
assert.Contains(t, row, int64(106))
assert.Contains(t, row, int64(113))
assert.Contains(t, row[113], "x")
assert.Contains(t, row[113], "y")
// dynamic field not provided
raw = map[string]interface{}{
"FieldID": 100,
}
row, err = parser.verifyRow(raw)
t.Run("no need to provide value for auto-id", func(t *testing.T) {
raw := map[string]interface{}{
"FieldID": 100,
"FieldDynamic": "{\"x\": 8}",
"y": true,
}
_, err := parser.verifyRow(raw)
assert.Error(t, err)
raw = map[string]interface{}{
"FieldDynamic": "{\"x\": 8}",
"y": true,
}
row, err := parser.verifyRow(raw)
assert.NoError(t, err)
assert.Contains(t, row, int64(113))
})
schema.Fields[1].IsDynamic = false
err = collectionInfo.resetSchema(schema)
assert.NoError(t, err)
assert.Contains(t, row, int64(106))
assert.Contains(t, row, int64(113))
assert.Equal(t, "{}", row[113])
// invalid input dynamic field
raw = map[string]interface{}{
"FieldID": 100,
"FieldDynamic": true,
"y": true,
}
_, err = parser.verifyRow(raw)
assert.Error(t, err)
t.Run("auto id, no dynamic field", func(t *testing.T) {
raw := map[string]interface{}{
"FieldDynamic": "{\"x\": 8}",
"y": true,
}
_, err := parser.verifyRow(raw)
assert.Error(t, err)
raw = map[string]interface{}{}
_, err = parser.verifyRow(raw)
assert.Error(t, err)
})
}

View File

@ -87,13 +87,13 @@ type NumpyAdapter struct {
func NewNumpyAdapter(reader io.Reader) (*NumpyAdapter, error) {
r, err := npyio.NewReader(reader)
if err != nil {
log.Error("Numpy adapter: failed to read numpy header", zap.Error(err))
log.Warn("Numpy adapter: failed to read numpy header", zap.Error(err))
return nil, err
}
dataType, err := convertNumpyType(r.Header.Descr.Type)
if err != nil {
log.Error("Numpy adapter: failed to detect data type", zap.Error(err))
log.Warn("Numpy adapter: failed to detect data type", zap.Error(err))
return nil, err
}
@ -139,7 +139,7 @@ func convertNumpyType(typeStr string) (schemapb.DataType, error) {
// Note: JSON field and VARCHAR field are using string type numpy
return schemapb.DataType_VarChar, nil
}
log.Error("Numpy adapter: the numpy file data type is not supported", zap.String("dtype", typeStr))
log.Warn("Numpy adapter: the numpy file data type is not supported", zap.String("dtype", typeStr))
return schemapb.DataType_None, fmt.Errorf("the numpy file dtype '%s' is not supported", typeStr)
}
}
@ -182,7 +182,7 @@ func stringLen(dtype string) (int, bool, error) {
return v, utf, nil
}
log.Error("Numpy adapter: the numpy file dtype is not varchar data type", zap.String("dtype", dtype))
log.Warn("Numpy adapter: the numpy file dtype is not varchar data type", zap.String("dtype", dtype))
return 0, false, fmt.Errorf("dtype '%s' of numpy file is not varchar data type", dtype)
}
@ -255,13 +255,13 @@ func (n *NumpyAdapter) checkCount(count int) int {
func (n *NumpyAdapter) ReadBool(count int) ([]bool, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read bool data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read bool data with a zero or nagative count")
return nil, errors.New("cannot read bool data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_Bool {
log.Error("Numpy adapter: numpy data is not bool type")
log.Warn("Numpy adapter: numpy data is not bool type")
return nil, errors.New("numpy data is not bool type")
}
@ -277,7 +277,7 @@ func (n *NumpyAdapter) ReadBool(count int) ([]bool, error) {
data := make([]bool, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read bool data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read bool data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf(" failed to read bool data with count %d, error: %w", readSize, err)
}
@ -289,7 +289,7 @@ func (n *NumpyAdapter) ReadBool(count int) ([]bool, error) {
func (n *NumpyAdapter) ReadUint8(count int) ([]uint8, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read uint8 data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read uint8 data with a zero or nagative count")
return nil, errors.New("cannot read uint8 data with a zero or nagative count")
}
@ -298,7 +298,7 @@ func (n *NumpyAdapter) ReadUint8(count int) ([]uint8, error) {
switch n.npyReader.Header.Descr.Type {
case "u1", "<u1", "|u1", "uint8":
default:
log.Error("Numpy adapter: numpy data is not uint8 type")
log.Warn("Numpy adapter: numpy data is not uint8 type")
return nil, errors.New("numpy data is not uint8 type")
}
@ -314,7 +314,7 @@ func (n *NumpyAdapter) ReadUint8(count int) ([]uint8, error) {
data := make([]uint8, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read uint8 data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read uint8 data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf("failed to read uint8 data with count %d, error: %w", readSize, err)
}
@ -326,13 +326,13 @@ func (n *NumpyAdapter) ReadUint8(count int) ([]uint8, error) {
func (n *NumpyAdapter) ReadInt8(count int) ([]int8, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read int8 data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read int8 data with a zero or nagative count")
return nil, errors.New("cannot read int8 data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_Int8 {
log.Error("Numpy adapter: numpy data is not int8 type")
log.Warn("Numpy adapter: numpy data is not int8 type")
return nil, errors.New("numpy data is not int8 type")
}
@ -348,7 +348,7 @@ func (n *NumpyAdapter) ReadInt8(count int) ([]int8, error) {
data := make([]int8, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read int8 data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read int8 data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf("failed to read int8 data with count %d, error: %w", readSize, err)
}
@ -360,13 +360,13 @@ func (n *NumpyAdapter) ReadInt8(count int) ([]int8, error) {
func (n *NumpyAdapter) ReadInt16(count int) ([]int16, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read int16 data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read int16 data with a zero or nagative count")
return nil, errors.New("cannot read int16 data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_Int16 {
log.Error("Numpy adapter: numpy data is not int16 type")
log.Warn("Numpy adapter: numpy data is not int16 type")
return nil, errors.New("numpy data is not int16 type")
}
@ -382,7 +382,7 @@ func (n *NumpyAdapter) ReadInt16(count int) ([]int16, error) {
data := make([]int16, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read int16 data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read int16 data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf("failed to read int16 data with count %d, error: %w", readSize, err)
}
@ -394,13 +394,13 @@ func (n *NumpyAdapter) ReadInt16(count int) ([]int16, error) {
func (n *NumpyAdapter) ReadInt32(count int) ([]int32, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read int32 data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read int32 data with a zero or nagative count")
return nil, errors.New("cannot read int32 data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_Int32 {
log.Error("Numpy adapter: numpy data is not int32 type")
log.Warn("Numpy adapter: numpy data is not int32 type")
return nil, errors.New("numpy data is not int32 type")
}
@ -416,7 +416,7 @@ func (n *NumpyAdapter) ReadInt32(count int) ([]int32, error) {
data := make([]int32, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read int32 data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read int32 data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf("failed to read int32 data with count %d, error: %w", readSize, err)
}
@ -428,13 +428,13 @@ func (n *NumpyAdapter) ReadInt32(count int) ([]int32, error) {
func (n *NumpyAdapter) ReadInt64(count int) ([]int64, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read int64 data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read int64 data with a zero or nagative count")
return nil, errors.New("cannot read int64 data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_Int64 {
log.Error("Numpy adapter: numpy data is not int64 type")
log.Warn("Numpy adapter: numpy data is not int64 type")
return nil, errors.New("numpy data is not int64 type")
}
@ -450,7 +450,7 @@ func (n *NumpyAdapter) ReadInt64(count int) ([]int64, error) {
data := make([]int64, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read int64 data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read int64 data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf("failed to read int64 data with count %d, error: %w", readSize, err)
}
@ -462,13 +462,13 @@ func (n *NumpyAdapter) ReadInt64(count int) ([]int64, error) {
func (n *NumpyAdapter) ReadFloat32(count int) ([]float32, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read float32 data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read float32 data with a zero or nagative count")
return nil, errors.New("cannot read float32 data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_Float {
log.Error("Numpy adapter: numpy data is not float32 type")
log.Warn("Numpy adapter: numpy data is not float32 type")
return nil, errors.New("numpy data is not float32 type")
}
@ -484,7 +484,7 @@ func (n *NumpyAdapter) ReadFloat32(count int) ([]float32, error) {
data := make([]float32, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read float32 data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read float32 data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf("failed to read float32 data with count %d, error: %w", readSize, err)
}
@ -496,13 +496,13 @@ func (n *NumpyAdapter) ReadFloat32(count int) ([]float32, error) {
func (n *NumpyAdapter) ReadFloat64(count int) ([]float64, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read float64 data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read float64 data with a zero or nagative count")
return nil, errors.New("cannot read float64 data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_Double {
log.Error("Numpy adapter: numpy data is not float64 type")
log.Warn("Numpy adapter: numpy data is not float64 type")
return nil, errors.New("numpy data is not float64 type")
}
@ -518,7 +518,7 @@ func (n *NumpyAdapter) ReadFloat64(count int) ([]float64, error) {
data := make([]float64, readSize)
err := binary.Read(n.reader, n.order, &data)
if err != nil {
log.Error("Numpy adapter: failed to read float64 data", zap.Int("count", count), zap.Error(err))
log.Warn("Numpy adapter: failed to read float64 data", zap.Int("count", count), zap.Error(err))
return nil, fmt.Errorf("failed to read float64 data with count %d, error: %w", readSize, err)
}
@ -530,20 +530,20 @@ func (n *NumpyAdapter) ReadFloat64(count int) ([]float64, error) {
func (n *NumpyAdapter) ReadString(count int) ([]string, error) {
if count <= 0 {
log.Error("Numpy adapter: cannot read varchar data with a zero or nagative count")
log.Warn("Numpy adapter: cannot read varchar data with a zero or nagative count")
return nil, errors.New("cannot read varchar data with a zero or nagative count")
}
// incorrect type
if n.dataType != schemapb.DataType_VarChar {
log.Error("Numpy adapter: numpy data is not varchar type")
log.Warn("Numpy adapter: numpy data is not varchar type")
return nil, errors.New("numpy data is not varchar type")
}
// varchar length, this is the max length, some item is shorter than this length, but they also occupy bytes of max length
maxLen, utf, err := stringLen(n.npyReader.Header.Descr.Type)
if err != nil || maxLen <= 0 {
log.Error("Numpy adapter: failed to get max length of varchar from numpy file header", zap.Int("maxLen", maxLen), zap.Error(err))
log.Warn("Numpy adapter: failed to get max length of varchar from numpy file header", zap.Int("maxLen", maxLen), zap.Error(err))
return nil, fmt.Errorf("failed to get max length %d of varchar from numpy file header, error: %w", maxLen, err)
}
// log.Info("Numpy adapter: get varchar max length from numpy file header", zap.Int("maxLen", maxLen), zap.Bool("utf", utf))
@ -557,7 +557,7 @@ func (n *NumpyAdapter) ReadString(count int) ([]string, error) {
}
if n.reader == nil {
log.Error("Numpy adapter: reader is nil")
log.Warn("Numpy adapter: reader is nil")
return nil, errors.New("numpy reader is nil")
}
@ -574,13 +574,13 @@ func (n *NumpyAdapter) ReadString(count int) ([]string, error) {
// for non-ascii characters, the unicode could be 1 ~ 4 bytes, each character occupys 4 bytes, too
raw, err := ioutil.ReadAll(io.LimitReader(n.reader, utf8.UTFMax*int64(maxLen)))
if err != nil {
log.Error("Numpy adapter: failed to read utf32 bytes from numpy file", zap.Int("i", i), zap.Error(err))
log.Warn("Numpy adapter: failed to read utf32 bytes from numpy file", zap.Int("i", i), zap.Error(err))
return nil, fmt.Errorf("failed to read utf32 bytes from numpy file, error: %w", err)
}
str, err := decodeUtf32(raw, n.order)
if err != nil {
log.Error("Numpy adapter: failed todecode utf32 bytes", zap.Int("i", i), zap.Error(err))
log.Warn("Numpy adapter: failed todecode utf32 bytes", zap.Int("i", i), zap.Error(err))
return nil, fmt.Errorf("failed to decode utf32 bytes, error: %w", err)
}
@ -590,7 +590,7 @@ func (n *NumpyAdapter) ReadString(count int) ([]string, error) {
// bytes.Index(buf, []byte{0}) tell us which position is the end of the string
buf, err := ioutil.ReadAll(io.LimitReader(n.reader, int64(maxLen)))
if err != nil {
log.Error("Numpy adapter: failed to read ascii bytes from numpy file", zap.Int("i", i), zap.Error(err))
log.Warn("Numpy adapter: failed to read ascii bytes from numpy file", zap.Int("i", i), zap.Error(err))
return nil, fmt.Errorf("failed to read ascii bytes from numpy file, error: %w", err)
}
n := bytes.Index(buf, []byte{0})
@ -610,7 +610,7 @@ func (n *NumpyAdapter) ReadString(count int) ([]string, error) {
func decodeUtf32(src []byte, order binary.ByteOrder) (string, error) {
if len(src)%4 != 0 {
log.Error("Numpy adapter: invalid utf32 bytes length, the byte array length should be multiple of 4", zap.Int("byteLen", len(src)))
log.Warn("Numpy adapter: invalid utf32 bytes length, the byte array length should be multiple of 4", zap.Int("byteLen", len(src)))
return "", fmt.Errorf("invalid utf32 bytes length %d, the byte array length should be multiple of 4", len(src))
}
@ -640,7 +640,7 @@ func decodeUtf32(src []byte, order binary.ByteOrder) (string, error) {
decoder := unicode.UTF16(uOrder, unicode.IgnoreBOM).NewDecoder()
res, err := decoder.Bytes(src[lowbytesPosition : lowbytesPosition+2])
if err != nil {
log.Error("Numpy adapter: failed to decode utf32 binary bytes", zap.Error(err))
log.Warn("Numpy adapter: failed to decode utf32 binary bytes", zap.Error(err))
return "", fmt.Errorf("failed to decode utf32 binary bytes, error: %w", err)
}
str += string(res)
@ -658,7 +658,7 @@ func decodeUtf32(src []byte, order binary.ByteOrder) (string, error) {
utf8Code := make([]byte, 4)
utf8.EncodeRune(utf8Code, r)
if r == utf8.RuneError {
log.Error("Numpy adapter: failed to convert 4 bytes unicode to utf8 rune", zap.Uint32("code", x))
log.Warn("Numpy adapter: failed to convert 4 bytes unicode to utf8 rune", zap.Uint32("code", x))
return "", fmt.Errorf("failed to convert 4 bytes unicode %d to utf8 rune", x)
}
str += string(utf8Code)

View File

@ -48,58 +48,55 @@ func closeReaders(columnReaders []*NumpyColumnReader) {
if reader.file != nil {
err := reader.file.Close()
if err != nil {
log.Error("Numper parser: failed to close numpy file", zap.String("fileName", reader.fieldName+NumpyFileExt))
log.Warn("Numper parser: failed to close numpy file", zap.String("fileName", reader.fieldName+NumpyFileExt))
}
}
}
}
type NumpyParser struct {
ctx context.Context // for canceling parse process
collectionSchema *schemapb.CollectionSchema // collection schema
rowIDAllocator *allocator.IDAllocator // autoid allocator
shardNum int32 // sharding number of the collection
blockSize int64 // maximum size of a read block(unit:byte)
chunkManager storage.ChunkManager // storage interfaces to browse/read the files
autoIDRange []int64 // auto-generated id range, for example: [1, 10, 20, 25] means id from 1 to 10 and 20 to 25
callFlushFunc ImportFlushFunc // call back function to flush segment
updateProgressFunc func(percent int64) // update working progress percent value
ctx context.Context // for canceling parse process
collectionInfo *CollectionInfo // collection details including schema
rowIDAllocator *allocator.IDAllocator // autoid allocator
blockSize int64 // maximum size of a read block(unit:byte)
chunkManager storage.ChunkManager // storage interfaces to browse/read the files
autoIDRange []int64 // auto-generated id range, for example: [1, 10, 20, 25] means id from 1 to 10 and 20 to 25
callFlushFunc ImportFlushFunc // call back function to flush segment
updateProgressFunc func(percent int64) // update working progress percent value
}
// NewNumpyParser is helper function to create a NumpyParser
func NewNumpyParser(ctx context.Context,
collectionSchema *schemapb.CollectionSchema,
collectionInfo *CollectionInfo,
idAlloc *allocator.IDAllocator,
shardNum int32,
blockSize int64,
chunkManager storage.ChunkManager,
flushFunc ImportFlushFunc,
updateProgressFunc func(percent int64)) (*NumpyParser, error) {
if collectionSchema == nil {
log.Error("Numper parser: collection schema is nil")
if collectionInfo == nil {
log.Warn("Numper parser: collection schema is nil")
return nil, errors.New("collection schema is nil")
}
if idAlloc == nil {
log.Error("Numper parser: id allocator is nil")
log.Warn("Numper parser: id allocator is nil")
return nil, errors.New("id allocator is nil")
}
if chunkManager == nil {
log.Error("Numper parser: chunk manager pointer is nil")
log.Warn("Numper parser: chunk manager pointer is nil")
return nil, errors.New("chunk manager pointer is nil")
}
if flushFunc == nil {
log.Error("Numper parser: flush function is nil")
log.Warn("Numper parser: flush function is nil")
return nil, errors.New("flush function is nil")
}
parser := &NumpyParser{
ctx: ctx,
collectionSchema: collectionSchema,
collectionInfo: collectionInfo,
rowIDAllocator: idAlloc,
shardNum: shardNum,
blockSize: blockSize,
chunkManager: chunkManager,
autoIDRange: make([]int64, 0),
@ -145,8 +142,8 @@ func (p *NumpyParser) Parse(filePaths []string) error {
func (p *NumpyParser) validateFileNames(filePaths []string) error {
dynamicFieldName := ""
requiredFieldNames := make(map[string]interface{})
for _, schema := range p.collectionSchema.Fields {
if schema.GetIsDynamic() && p.collectionSchema.GetEnableDynamicField() {
for _, schema := range p.collectionInfo.Schema.Fields {
if schema.GetIsDynamic() && p.collectionInfo.Schema.GetEnableDynamicField() {
dynamicFieldName = schema.GetName()
}
if schema.GetIsPrimaryKey() {
@ -165,7 +162,7 @@ func (p *NumpyParser) validateFileNames(filePaths []string) error {
fileNames[name] = nil
_, ok := requiredFieldNames[name]
if !ok {
log.Error("Numpy parser: the file has no corresponding field in collection", zap.String("fieldName", name))
log.Warn("Numpy parser: the file has no corresponding field in collection", zap.String("fieldName", name))
return fmt.Errorf("the file '%s' has no corresponding field in collection", filePath)
}
}
@ -178,7 +175,7 @@ func (p *NumpyParser) validateFileNames(filePaths []string) error {
}
_, ok := fileNames[name]
if !ok {
log.Error("Numpy parser: there is no file corresponding to field", zap.String("fieldName", name))
log.Warn("Numpy parser: there is no file corresponding to field", zap.String("fieldName", name))
return fmt.Errorf("there is no file corresponding to field '%s'", name)
}
}
@ -195,8 +192,8 @@ func (p *NumpyParser) createReaders(filePaths []string) ([]*NumpyColumnReader, e
// check existence of the target field
var schema *schemapb.FieldSchema
for i := 0; i < len(p.collectionSchema.Fields); i++ {
tmpSchema := p.collectionSchema.Fields[i]
for i := 0; i < len(p.collectionInfo.Schema.Fields); i++ {
tmpSchema := p.collectionInfo.Schema.Fields[i]
if tmpSchema.GetName() == fileName {
schema = tmpSchema
break
@ -204,24 +201,24 @@ func (p *NumpyParser) createReaders(filePaths []string) ([]*NumpyColumnReader, e
}
if schema == nil {
log.Error("Numpy parser: the field is not found in collection schema", zap.String("fileName", fileName))
log.Warn("Numpy parser: the field is not found in collection schema", zap.String("fileName", fileName))
return nil, fmt.Errorf("the field name '%s' is not found in collection schema", fileName)
}
file, err := p.chunkManager.Reader(p.ctx, filePath)
if err != nil {
log.Error("Numpy parser: failed to read the file", zap.String("filePath", filePath), zap.Error(err))
log.Warn("Numpy parser: failed to read the file", zap.String("filePath", filePath), zap.Error(err))
return nil, fmt.Errorf("failed to read the file '%s', error: %s", filePath, err.Error())
}
adapter, err := NewNumpyAdapter(file)
if err != nil {
log.Error("Numpy parser: failed to read the file header", zap.String("filePath", filePath), zap.Error(err))
log.Warn("Numpy parser: failed to read the file header", zap.String("filePath", filePath), zap.Error(err))
return nil, fmt.Errorf("failed to read the file header '%s', error: %s", filePath, err.Error())
}
if file == nil || adapter == nil {
log.Error("Numpy parser: failed to open file", zap.String("filePath", filePath))
log.Warn("Numpy parser: failed to open file", zap.String("filePath", filePath))
return nil, fmt.Errorf("failed to open file '%s'", filePath)
}
@ -250,7 +247,7 @@ func (p *NumpyParser) createReaders(filePaths []string) ([]*NumpyColumnReader, e
for i := 1; i < len(readers); i++ {
compareReader := readers[i]
if rowCount != compareReader.rowCount {
log.Error("Numpy parser: the row count of files are not equal",
log.Warn("Numpy parser: the row count of files are not equal",
zap.String("firstFile", firstReader.fieldName), zap.Int("firstRowCount", firstReader.rowCount),
zap.String("compareFile", compareReader.fieldName), zap.Int("compareRowCount", compareReader.rowCount))
return nil, fmt.Errorf("the row count(%d) of file '%s.npy' is not equal to row count(%d) of file '%s.npy'",
@ -265,7 +262,7 @@ func (p *NumpyParser) createReaders(filePaths []string) ([]*NumpyColumnReader, e
// validateHeader is to verify numpy file header, file header information should match field's schema
func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
if columnReader == nil || columnReader.reader == nil {
log.Error("Numpy parser: numpy reader is nil")
log.Warn("Numpy parser: numpy reader is nil")
return errors.New("numpy adapter is nil")
}
@ -273,7 +270,7 @@ func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
shape := columnReader.reader.GetShape()
// if user only save an element in a numpy file, the shape list will be empty
if len(shape) == 0 {
log.Error("Numpy parser: the content stored in numpy file is not valid numpy array",
log.Warn("Numpy parser: the content stored in numpy file is not valid numpy array",
zap.String("fieldName", columnReader.fieldName))
return fmt.Errorf("the content stored in numpy file is not valid numpy array for field '%s'", columnReader.fieldName)
}
@ -286,7 +283,7 @@ func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
// 1. for float vector, we support float32 and float64 numpy file because python float value is 64 bit
// 2. for float64 numpy file, the performance is worse than float32 numpy file
if elementType != schemapb.DataType_Float && elementType != schemapb.DataType_Double {
log.Error("Numpy parser: illegal data type of numpy file for float vector field", zap.Any("dataType", elementType),
log.Warn("Numpy parser: illegal data type of numpy file for float vector field", zap.Any("dataType", elementType),
zap.String("fieldName", columnReader.fieldName))
return fmt.Errorf("illegal data type %s of numpy file for float vector field '%s'", getTypeName(elementType),
columnReader.fieldName)
@ -294,21 +291,21 @@ func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
// vector field, the shape should be 2
if len(shape) != 2 {
log.Error("Numpy parser: illegal shape of numpy file for float vector field, shape should be 2", zap.Int("shape", len(shape)),
log.Warn("Numpy parser: illegal shape of numpy file for float vector field, shape should be 2", zap.Int("shape", len(shape)),
zap.String("fieldName", columnReader.fieldName))
return fmt.Errorf("illegal shape %d of numpy file for float vector field '%s', shape should be 2", shape,
columnReader.fieldName)
}
if shape[1] != columnReader.dimension {
log.Error("Numpy parser: illegal dimension of numpy file for float vector field", zap.String("fieldName", columnReader.fieldName),
log.Warn("Numpy parser: illegal dimension of numpy file for float vector field", zap.String("fieldName", columnReader.fieldName),
zap.Int("numpyDimension", shape[1]), zap.Int("fieldDimension", columnReader.dimension))
return fmt.Errorf("illegal dimension %d of numpy file for float vector field '%s', dimension should be %d",
shape[1], columnReader.fieldName, columnReader.dimension)
}
} else if schemapb.DataType_BinaryVector == columnReader.dataType {
if elementType != schemapb.DataType_BinaryVector {
log.Error("Numpy parser: illegal data type of numpy file for binary vector field", zap.Any("dataType", elementType),
log.Warn("Numpy parser: illegal data type of numpy file for binary vector field", zap.Any("dataType", elementType),
zap.String("fieldName", columnReader.fieldName))
return fmt.Errorf("illegal data type %s of numpy file for binary vector field '%s'", getTypeName(elementType),
columnReader.fieldName)
@ -316,14 +313,14 @@ func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
// vector field, the shape should be 2
if len(shape) != 2 {
log.Error("Numpy parser: illegal shape of numpy file for binary vector field, shape should be 2", zap.Int("shape", len(shape)),
log.Warn("Numpy parser: illegal shape of numpy file for binary vector field, shape should be 2", zap.Int("shape", len(shape)),
zap.String("fieldName", columnReader.fieldName))
return fmt.Errorf("illegal shape %d of numpy file for binary vector field '%s', shape should be 2", shape,
columnReader.fieldName)
}
if shape[1] != columnReader.dimension/8 {
log.Error("Numpy parser: illegal dimension of numpy file for float vector field", zap.String("fieldName", columnReader.fieldName),
log.Warn("Numpy parser: illegal dimension of numpy file for float vector field", zap.String("fieldName", columnReader.fieldName),
zap.Int("numpyDimension", shape[1]*8), zap.Int("fieldDimension", columnReader.dimension))
return fmt.Errorf("illegal dimension %d of numpy file for binary vector field '%s', dimension should be %d",
shape[1]*8, columnReader.fieldName, columnReader.dimension)
@ -333,7 +330,7 @@ func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
// legal input if columnReader.dataType is JSON and elementType is VARCHAR
if elementType != schemapb.DataType_VarChar && columnReader.dataType != schemapb.DataType_JSON {
if elementType != columnReader.dataType {
log.Error("Numpy parser: illegal data type of numpy file for scalar field", zap.Any("numpyDataType", elementType),
log.Warn("Numpy parser: illegal data type of numpy file for scalar field", zap.Any("numpyDataType", elementType),
zap.String("fieldName", columnReader.fieldName), zap.Any("fieldDataType", columnReader.dataType))
return fmt.Errorf("illegal data type %s of numpy file for scalar field '%s' with type %s",
getTypeName(elementType), columnReader.fieldName, getTypeName(columnReader.dataType))
@ -342,7 +339,7 @@ func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
// scalar field, the shape should be 1
if len(shape) != 1 {
log.Error("Numpy parser: illegal shape of numpy file for scalar field, shape should be 1", zap.Int("shape", len(shape)),
log.Warn("Numpy parser: illegal shape of numpy file for scalar field, shape should be 1", zap.Int("shape", len(shape)),
zap.String("fieldName", columnReader.fieldName))
return fmt.Errorf("illegal shape %d of numpy file for scalar field '%s', shape should be 1", shape, columnReader.fieldName)
}
@ -353,14 +350,14 @@ func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
// calcRowCountPerBlock calculates a proper value for a batch row count to read file
func (p *NumpyParser) calcRowCountPerBlock() (int64, error) {
sizePerRecord, err := typeutil.EstimateSizePerRecord(p.collectionSchema)
sizePerRecord, err := typeutil.EstimateSizePerRecord(p.collectionInfo.Schema)
if err != nil {
log.Error("Numpy parser: failed to estimate size of each row", zap.Error(err))
log.Warn("Numpy parser: failed to estimate size of each row", zap.Error(err))
return 0, fmt.Errorf("failed to estimate size of each row: %s", err.Error())
}
if sizePerRecord <= 0 {
log.Error("Numpy parser: failed to estimate size of each row, the collection schema might be empty")
log.Warn("Numpy parser: failed to estimate size of each row, the collection schema might be empty")
return 0, fmt.Errorf("failed to estimate size of each row: the collection schema might be empty")
}
@ -395,14 +392,14 @@ func (p *NumpyParser) consume(columnReaders []*NumpyColumnReader) error {
}
// prepare shards
shards := 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("Numper parser: failed to initialize FieldData list")
shards := 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("Numper parser: failed to initialize FieldData list")
return fmt.Errorf("failed to initialize FieldData list")
}
shards = append(shards, segmentData)
shards = append(shards, shardData)
}
tr := timerecord.NewTimeRecorder("consume performance")
defer tr.Elapse("end")
@ -410,7 +407,7 @@ func (p *NumpyParser) consume(columnReaders []*NumpyColumnReader) error {
totalRead := 0
for {
readRowCount := 0
segmentData := make(map[storage.FieldID]storage.FieldData)
segmentData := make(BlockData)
for _, reader := range columnReaders {
fieldData, err := p.readData(reader, int(rowCountPerBlock))
if err != nil {
@ -420,7 +417,7 @@ func (p *NumpyParser) consume(columnReaders []*NumpyColumnReader) error {
if readRowCount == 0 {
readRowCount = fieldData.RowNum()
} else if readRowCount != fieldData.RowNum() {
log.Error("Numpy parser: data block's row count mismatch", zap.Int("firstBlockRowCount", readRowCount),
log.Warn("Numpy parser: data block's row count mismatch", zap.Int("firstBlockRowCount", readRowCount),
zap.Int("thisBlockRowCount", fieldData.RowNum()), zap.Int64("rowCountPerBlock", rowCountPerBlock))
return fmt.Errorf("data block's row count mismatch: %d vs %d", readRowCount, fieldData.RowNum())
}
@ -442,7 +439,7 @@ func (p *NumpyParser) consume(columnReaders []*NumpyColumnReader) error {
}
tr.Record("splitFieldsData")
// when the estimated size is close to blockSize, save to binlog
err = tryFlushBlocks(p.ctx, shards, p.collectionSchema, p.callFlushFunc, p.blockSize, MaxTotalSizeInMemory, false)
err = tryFlushBlocks(p.ctx, shards, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, MaxTotalSizeInMemory, false)
if err != nil {
return err
}
@ -450,7 +447,7 @@ func (p *NumpyParser) consume(columnReaders []*NumpyColumnReader) error {
}
// force flush at the end
return tryFlushBlocks(p.ctx, shards, p.collectionSchema, p.callFlushFunc, p.blockSize, MaxTotalSizeInMemory, true)
return tryFlushBlocks(p.ctx, shards, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, MaxTotalSizeInMemory, true)
}
// readData method reads numpy data section into a storage.FieldData
@ -459,7 +456,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_Bool:
data, err := columnReader.reader.ReadBool(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read bool array", zap.Error(err))
log.Warn("Numpy parser: failed to read bool array", zap.Error(err))
return nil, fmt.Errorf("failed to read bool array: %s", err.Error())
}
@ -469,7 +466,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_Int8:
data, err := columnReader.reader.ReadInt8(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read int8 array", zap.Error(err))
log.Warn("Numpy parser: failed to read int8 array", zap.Error(err))
return nil, fmt.Errorf("failed to read int8 array: %s", err.Error())
}
@ -479,7 +476,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_Int16:
data, err := columnReader.reader.ReadInt16(rowCount)
if err != nil {
log.Error("Numpy parser: failed to int16 array", zap.Error(err))
log.Warn("Numpy parser: failed to int16 array", zap.Error(err))
return nil, fmt.Errorf("failed to read int16 array: %s", err.Error())
}
@ -489,7 +486,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_Int32:
data, err := columnReader.reader.ReadInt32(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read int32 array", zap.Error(err))
log.Warn("Numpy parser: failed to read int32 array", zap.Error(err))
return nil, fmt.Errorf("failed to read int32 array: %s", err.Error())
}
@ -499,7 +496,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_Int64:
data, err := columnReader.reader.ReadInt64(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read int64 array", zap.Error(err))
log.Warn("Numpy parser: failed to read int64 array", zap.Error(err))
return nil, fmt.Errorf("failed to read int64 array: %s", err.Error())
}
@ -509,13 +506,13 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_Float:
data, err := columnReader.reader.ReadFloat32(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read float array", zap.Error(err))
log.Warn("Numpy parser: failed to read float array", zap.Error(err))
return nil, fmt.Errorf("failed to read float array: %s", err.Error())
}
err = typeutil.VerifyFloats32(data)
if err != nil {
log.Error("Numpy parser: illegal value in float array", zap.Error(err))
log.Warn("Numpy parser: illegal value in float array", zap.Error(err))
return nil, fmt.Errorf("illegal value in float array: %s", err.Error())
}
@ -525,13 +522,13 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_Double:
data, err := columnReader.reader.ReadFloat64(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read double array", zap.Error(err))
log.Warn("Numpy parser: failed to read double array", zap.Error(err))
return nil, fmt.Errorf("failed to read double array: %s", err.Error())
}
err = typeutil.VerifyFloats64(data)
if err != nil {
log.Error("Numpy parser: illegal value in double array", zap.Error(err))
log.Warn("Numpy parser: illegal value in double array", zap.Error(err))
return nil, fmt.Errorf("illegal value in double array: %s", err.Error())
}
@ -541,7 +538,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_VarChar:
data, err := columnReader.reader.ReadString(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read varchar array", zap.Error(err))
log.Warn("Numpy parser: failed to read varchar array", zap.Error(err))
return nil, fmt.Errorf("failed to read varchar array: %s", err.Error())
}
@ -552,7 +549,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
// JSON field read data from string array numpy
data, err := columnReader.reader.ReadString(rowCount)
if err != nil {
log.Error("Numpy parser: failed to read json string array", zap.Error(err))
log.Warn("Numpy parser: failed to read json string array", zap.Error(err))
return nil, fmt.Errorf("failed to read json string array: %s", err.Error())
}
@ -561,7 +558,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
var dummy interface{}
err := json.Unmarshal([]byte(str), &dummy)
if err != nil {
log.Error("Numpy parser: illegal string value for JSON field",
log.Warn("Numpy parser: illegal string value for JSON field",
zap.String("value", str), zap.String("FieldName", columnReader.fieldName), zap.Error(err))
return nil, fmt.Errorf("failed to parse value '%v' for JSON field '%s', error: %w",
str, columnReader.fieldName, err)
@ -575,7 +572,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
case schemapb.DataType_BinaryVector:
data, err := columnReader.reader.ReadUint8(rowCount * (columnReader.dimension / 8))
if err != nil {
log.Error("Numpy parser: failed to read binary vector array", zap.Error(err))
log.Warn("Numpy parser: failed to read binary vector array", zap.Error(err))
return nil, fmt.Errorf("failed to read binary vector array: %s", err.Error())
}
@ -594,13 +591,13 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
if elementType == schemapb.DataType_Float {
data, err = columnReader.reader.ReadFloat32(rowCount * columnReader.dimension)
if err != nil {
log.Error("Numpy parser: failed to read float vector array", zap.Error(err))
log.Warn("Numpy parser: failed to read float vector array", zap.Error(err))
return nil, fmt.Errorf("failed to read float vector array: %s", err.Error())
}
err = typeutil.VerifyFloats32(data)
if err != nil {
log.Error("Numpy parser: illegal value in float vector array", zap.Error(err))
log.Warn("Numpy parser: illegal value in float vector array", zap.Error(err))
return nil, fmt.Errorf("illegal value in float vector array: %s", err.Error())
}
@ -608,14 +605,14 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
data = make([]float32, 0, columnReader.rowCount)
data64, err := columnReader.reader.ReadFloat64(rowCount * columnReader.dimension)
if err != nil {
log.Error("Numpy parser: failed to read float vector array", zap.Error(err))
log.Warn("Numpy parser: failed to read float vector array", zap.Error(err))
return nil, fmt.Errorf("failed to read float vector array: %s", err.Error())
}
for _, f64 := range data64 {
err = typeutil.VerifyFloat(f64)
if err != nil {
log.Error("Numpy parser: illegal value in float vector array", zap.Error(err))
log.Warn("Numpy parser: illegal value in float vector array", zap.Error(err))
return nil, fmt.Errorf("illegal value in float vector array: %s", err.Error())
}
@ -628,7 +625,7 @@ func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (s
Dim: columnReader.dimension,
}, nil
default:
log.Error("Numpy parser: unsupported data type of field", zap.Any("dataType", columnReader.dataType),
log.Warn("Numpy parser: unsupported data type of field", zap.Any("dataType", columnReader.dataType),
zap.String("fieldName", columnReader.fieldName))
return nil, fmt.Errorf("unsupported data type %s of field '%s'", getTypeName(columnReader.dataType),
columnReader.fieldName)
@ -711,11 +708,11 @@ func (p *NumpyParser) appendFunc(schema *schemapb.FieldSchema) func(src storage.
func (p *NumpyParser) prepareAppendFunctions() (map[string]func(src storage.FieldData, n int, target storage.FieldData) error, error) {
appendFunctions := make(map[string]func(src storage.FieldData, n int, target storage.FieldData) error)
for i := 0; i < len(p.collectionSchema.Fields); i++ {
schema := p.collectionSchema.Fields[i]
for i := 0; i < len(p.collectionInfo.Schema.Fields); i++ {
schema := p.collectionInfo.Schema.Fields[i]
appendFuncErr := p.appendFunc(schema)
if appendFuncErr == nil {
log.Error("Numpy parser: unsupported field data type")
log.Warn("Numpy parser: unsupported field data type")
return nil, fmt.Errorf("unsupported field data type: %d", schema.GetDataType())
}
appendFunctions[schema.GetName()] = appendFuncErr
@ -723,18 +720,12 @@ func (p *NumpyParser) prepareAppendFunctions() (map[string]func(src storage.Fiel
return appendFunctions, nil
}
// checkRowCount checks existence of each field, and returns the primary key schema
// check row count, all fields row count must be equal
func (p *NumpyParser) checkRowCount(fieldsData map[storage.FieldID]storage.FieldData) (int, *schemapb.FieldSchema, error) {
// checkRowCount check row count of each field, all fields row count must be equal
func (p *NumpyParser) checkRowCount(fieldsData BlockData) (int, error) {
rowCount := 0
rowCounter := make(map[string]int)
var primaryKey *schemapb.FieldSchema
for i := 0; i < len(p.collectionSchema.Fields); i++ {
schema := p.collectionSchema.Fields[i]
if schema.GetIsPrimaryKey() {
primaryKey = schema
}
for i := 0; i < len(p.collectionInfo.Schema.Fields); i++ {
schema := p.collectionInfo.Schema.Fields[i]
if !schema.GetAutoID() {
v, ok := fieldsData[schema.GetFieldID()]
if !ok {
@ -742,8 +733,8 @@ func (p *NumpyParser) checkRowCount(fieldsData map[storage.FieldID]storage.Field
// user might not provide numpy file for dynamic field, skip it, will auto-generate later
continue
}
log.Error("Numpy parser: field not provided", zap.String("fieldName", schema.GetName()))
return 0, nil, fmt.Errorf("field '%s' not provided", schema.GetName())
log.Warn("Numpy parser: field not provided", zap.String("fieldName", schema.GetName()))
return 0, fmt.Errorf("field '%s' not provided", schema.GetName())
}
rowCounter[schema.GetName()] = v.RowNum()
if v.RowNum() > rowCount {
@ -751,37 +742,32 @@ func (p *NumpyParser) checkRowCount(fieldsData map[storage.FieldID]storage.Field
}
}
}
if primaryKey == nil {
log.Error("Numpy parser: primary key field is not found")
return 0, nil, fmt.Errorf("primary key field is not found")
}
for name, count := range rowCounter {
if count != rowCount {
log.Error("Numpy parser: field row count is not equal to other fields row count", zap.String("fieldName", name),
log.Warn("Numpy parser: field row count is not equal to other fields row count", zap.String("fieldName", name),
zap.Int("rowCount", count), zap.Int("otherRowCount", rowCount))
return 0, nil, fmt.Errorf("field '%s' row count %d is not equal to other fields row count: %d", name, count, rowCount)
return 0, fmt.Errorf("field '%s' row count %d is not equal to other fields row count: %d", name, count, rowCount)
}
}
// log.Info("Numpy parser: try to split a block with row count", zap.Int("rowCount", rowCount))
return rowCount, primaryKey, nil
return rowCount, nil
}
// splitFieldsData is to split the in-memory data(parsed from column-based files) into shards
func (p *NumpyParser) splitFieldsData(fieldsData map[storage.FieldID]storage.FieldData, shards []map[storage.FieldID]storage.FieldData) error {
func (p *NumpyParser) splitFieldsData(fieldsData BlockData, shards []ShardData) error {
if len(fieldsData) == 0 {
log.Error("Numpy parser: fields data to split is empty")
log.Warn("Numpy parser: fields data to split is empty")
return fmt.Errorf("fields data to split is empty")
}
if len(shards) != int(p.shardNum) {
log.Error("Numpy parser: block count is not equal to collection shard number", zap.Int("shardsLen", len(shards)),
zap.Int32("shardNum", p.shardNum))
return fmt.Errorf("block count %d is not equal to collection shard number %d", len(shards), p.shardNum)
if len(shards) != int(p.collectionInfo.ShardNum) {
log.Warn("Numpy parser: block count is not equal to collection shard number", zap.Int("shardsLen", len(shards)),
zap.Int32("shardNum", p.collectionInfo.ShardNum))
return fmt.Errorf("block count %d is not equal to collection shard number %d", len(shards), p.collectionInfo.ShardNum)
}
rowCount, primaryKey, err := p.checkRowCount(fieldsData)
rowCount, err := p.checkRowCount(fieldsData)
if err != nil {
return err
}
@ -789,7 +775,7 @@ func (p *NumpyParser) splitFieldsData(fieldsData map[storage.FieldID]storage.Fie
// generate auto id for primary key and rowid field
rowIDBegin, rowIDEnd, err := p.rowIDAllocator.Alloc(uint32(rowCount))
if err != nil {
log.Error("Numpy parser: failed to alloc row ID", zap.Int("rowCount", rowCount), zap.Error(err))
log.Warn("Numpy parser: failed to alloc row ID", zap.Int("rowCount", rowCount), zap.Error(err))
return fmt.Errorf("failed to alloc %d rows ID, error: %w", rowCount, err)
}
@ -806,10 +792,11 @@ func (p *NumpyParser) splitFieldsData(fieldsData map[storage.FieldID]storage.Fie
}
// reset the primary keys, as we know, only int64 pk can be auto-generated
primaryKey := p.collectionInfo.PrimaryKey
if primaryKey.GetAutoID() {
log.Info("Numpy parser: generating auto-id", zap.Int("rowCount", rowCount), zap.Int64("rowIDBegin", rowIDBegin))
if primaryKey.GetDataType() != schemapb.DataType_Int64 {
log.Error("Numpy parser: primary key field is auto-generated but the field type is not int64")
log.Warn("Numpy parser: primary key field is auto-generated but the field type is not int64")
return fmt.Errorf("primary key field is auto-generated but the field type is not int64")
}
@ -827,7 +814,7 @@ func (p *NumpyParser) splitFieldsData(fieldsData map[storage.FieldID]storage.Fie
// if the primary key is not auto-gernerate and user doesn't provide, return error
primaryData, ok := fieldsData[primaryKey.GetFieldID()]
if !ok || primaryData.RowNum() <= 0 {
log.Error("Numpy parser: primary key field is not provided", zap.String("keyName", primaryKey.GetName()))
log.Warn("Numpy parser: primary key field is not provided", zap.String("keyName", primaryKey.GetName()))
return fmt.Errorf("primary key '%s' field data is not provided", primaryKey.GetName())
}
@ -839,29 +826,33 @@ func (p *NumpyParser) splitFieldsData(fieldsData map[storage.FieldID]storage.Fie
// split data into shards
for i := 0; i < rowCount; i++ {
// hash to a shard number
// hash to a shard number and partition
pk := primaryData.GetRow(i)
shard, err := pkToShard(pk, uint32(p.shardNum))
shard, err := pkToShard(pk, uint32(p.collectionInfo.ShardNum))
if err != nil {
return err
}
partitionID, err := p.hashToPartition(fieldsData, i)
if err != nil {
return err
}
// set rowID field
rowIDField := shards[shard][common.RowIDField].(*storage.Int64FieldData)
rowIDField := shards[shard][partitionID][common.RowIDField].(*storage.Int64FieldData)
rowIDField.Data = append(rowIDField.Data, rowIDFieldArr.GetRow(i).(int64))
// append row to shard
for k := 0; k < len(p.collectionSchema.Fields); k++ {
schema := p.collectionSchema.Fields[k]
for k := 0; k < len(p.collectionInfo.Schema.Fields); k++ {
schema := p.collectionInfo.Schema.Fields[k]
srcData := fieldsData[schema.GetFieldID()]
targetData := shards[shard][schema.GetFieldID()]
targetData := shards[shard][partitionID][schema.GetFieldID()]
if srcData == nil && schema.GetIsDynamic() {
// user might not provide numpy file for dynamic field, skip it, will auto-generate later
continue
}
if srcData == nil || targetData == nil {
log.Error("Numpy parser: cannot append data since source or target field data is nil",
log.Warn("Numpy parser: cannot append data since source or target field data is nil",
zap.String("FieldName", schema.GetName()),
zap.Bool("sourceNil", srcData == nil), zap.Bool("targetNil", targetData == nil))
return fmt.Errorf("cannot append data for field '%s', possibly no any fields corresponding to this numpy file, or a required numpy file is not provided",
@ -877,3 +868,25 @@ func (p *NumpyParser) splitFieldsData(fieldsData map[storage.FieldID]storage.Fie
return nil
}
// hashToPartition hash partition key to get an partition ID, return the first partition ID if no partition key exist
// CollectionInfo ensures only one partition ID in the PartitionIDs if no partition key exist
func (p *NumpyParser) hashToPartition(fieldsData BlockData, rowNumber int) (int64, error) {
if p.collectionInfo.PartitionKey == nil {
// no partition key, directly return the target partition id
if len(p.collectionInfo.PartitionIDs) != 1 {
return 0, fmt.Errorf("collection '%s' partition list is empty", p.collectionInfo.Schema.Name)
}
return p.collectionInfo.PartitionIDs[0], nil
}
partitionKeyID := p.collectionInfo.PartitionKey.GetFieldID()
fieldData := fieldsData[partitionKeyID]
value := fieldData.GetRow(rowNumber)
index, err := pkToShard(value, uint32(len(p.collectionInfo.PartitionIDs)))
if err != nil {
return 0, err
}
return p.collectionInfo.PartitionIDs[index], nil
}

View File

@ -20,6 +20,7 @@ import (
"context"
"math"
"os"
"path"
"testing"
"github.com/cockroachdb/errors"
@ -51,11 +52,13 @@ func createNumpyParser(t *testing.T) *NumpyParser {
cm := createLocalChunkManager(t)
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
parser, err := NewNumpyParser(ctx, schema, idAllocator, 2, 100, cm, flushFunc, nil)
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
parser, err := NewNumpyParser(ctx, collectionInfo, idAllocator, 100, cm, flushFunc, nil)
assert.NoError(t, err)
assert.NotNil(t, parser)
return parser
@ -71,33 +74,118 @@ func findSchema(schema *schemapb.CollectionSchema, dt schemapb.DataType) *schema
return nil
}
func createSampleNumpyFiles(t *testing.T, cm storage.ChunkManager) []string {
ctx := context.Background()
files := make([]string, 0)
filePath := path.Join(cm.RootPath(), "FieldBool.npy")
content, err := CreateNumpyData([]bool{true, false, true, true, true})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldInt8.npy")
content, err = CreateNumpyData([]int8{10, 11, 12, 13, 14})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldInt16.npy")
content, err = CreateNumpyData([]int16{100, 101, 102, 103, 104})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldInt32.npy")
content, err = CreateNumpyData([]int32{1000, 1001, 1002, 1003, 1004})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldInt64.npy")
content, err = CreateNumpyData([]int64{10000, 10001, 10002, 10003, 10004})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldFloat.npy")
content, err = CreateNumpyData([]float32{3.14, 3.15, 3.16, 3.17, 3.18})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldDouble.npy")
content, err = CreateNumpyData([]float64{5.1, 5.2, 5.3, 5.4, 5.5})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldString.npy")
content, err = CreateNumpyData([]string{"a", "bb", "ccc", "dd", "e"})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldJSON.npy")
content, err = CreateNumpyData([]string{"{\"x\": 10, \"y\": 5}", "{\"z\": 5}", "{}", "{}", "{\"x\": 3}"})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldBinaryVector.npy")
content, err = CreateNumpyData([][2]uint8{{1, 2}, {3, 4}, {5, 6}, {7, 8}, {9, 10}})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
filePath = path.Join(cm.RootPath(), "FieldFloatVector.npy")
content, err = CreateNumpyData([][4]float32{{1, 2, 3, 4}, {3, 4, 5, 6}, {5, 6, 7, 8}, {7, 8, 9, 10}, {9, 10, 11, 12}})
assert.NoError(t, err)
err = cm.Write(ctx, filePath, content)
assert.NoError(t, err)
files = append(files, filePath)
return files
}
func Test_NewNumpyParser(t *testing.T) {
ctx := context.Background()
parser, err := NewNumpyParser(ctx, nil, nil, 2, 100, nil, nil, nil)
parser, err := NewNumpyParser(ctx, nil, nil, 100, nil, nil, nil)
assert.Error(t, err)
assert.Nil(t, parser)
schema := sampleSchema()
parser, err = NewNumpyParser(ctx, schema, nil, 2, 100, nil, nil, nil)
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
assert.NoError(t, err)
parser, err = NewNumpyParser(ctx, collectionInfo, nil, 100, nil, nil, nil)
assert.Error(t, err)
assert.Nil(t, parser)
idAllocator := newIDAllocator(ctx, t, nil)
parser, err = NewNumpyParser(ctx, schema, idAllocator, 2, 100, nil, nil, nil)
parser, err = NewNumpyParser(ctx, collectionInfo, idAllocator, 100, nil, nil, nil)
assert.Error(t, err)
assert.Nil(t, parser)
cm := createLocalChunkManager(t)
parser, err = NewNumpyParser(ctx, schema, idAllocator, 2, 100, cm, nil, nil)
parser, err = NewNumpyParser(ctx, collectionInfo, idAllocator, 100, cm, nil, nil)
assert.Error(t, err)
assert.Nil(t, parser)
flushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
parser, err = NewNumpyParser(ctx, schema, idAllocator, 2, 100, cm, flushFunc, nil)
parser, err = NewNumpyParser(ctx, collectionInfo, idAllocator, 100, cm, flushFunc, nil)
assert.NoError(t, err)
assert.NotNil(t, parser)
}
@ -131,7 +219,7 @@ func Test_NumpyParserValidateFileNames(t *testing.T) {
assert.NoError(t, err)
// has dynamic field
parser.collectionSchema = &schemapb.CollectionSchema{
schema := &schemapb.CollectionSchema{
Name: "schema",
Description: "schema",
AutoID: true,
@ -152,6 +240,8 @@ func Test_NumpyParserValidateFileNames(t *testing.T) {
},
},
}
parser.collectionInfo.resetSchema(schema)
fileNames = []string{"FieldInt64.npy"}
err = parser.validateFileNames(fileNames)
assert.NoError(t, err)
@ -574,7 +664,7 @@ func Test_NumpyParserPrepareAppendFunctions(t *testing.T) {
assert.Equal(t, len(sampleSchema().Fields), len(appendFuncs))
// schema has unsupported data type
parser.collectionSchema = &schemapb.CollectionSchema{
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
@ -592,6 +682,7 @@ func Test_NumpyParserPrepareAppendFunctions(t *testing.T) {
},
},
}
parser.collectionInfo.resetSchema(schema)
appendFuncs, err = parser.prepareAppendFunctions()
assert.Error(t, err)
assert.Nil(t, appendFuncs)
@ -611,28 +702,25 @@ func Test_NumpyParserCheckRowCount(t *testing.T) {
defer closeReaders(readers)
// succeed
segmentData := make(map[storage.FieldID]storage.FieldData)
segmentData := make(BlockData)
for _, reader := range readers {
fieldData, err := parser.readData(reader, 100)
assert.NoError(t, err)
segmentData[reader.fieldID] = fieldData
}
rowCount, primaryKey, err := parser.checkRowCount(segmentData)
rowCount, err := parser.checkRowCount(segmentData)
assert.NoError(t, err)
assert.Equal(t, 5, rowCount)
assert.NotNil(t, primaryKey)
assert.Equal(t, "FieldInt64", primaryKey.GetName())
// field data missed
delete(segmentData, 102)
rowCount, primaryKey, err = parser.checkRowCount(segmentData)
rowCount, err = parser.checkRowCount(segmentData)
assert.Error(t, err)
assert.Zero(t, rowCount)
assert.Nil(t, primaryKey)
// primarykey missed
parser.collectionSchema = &schemapb.CollectionSchema{
// row count mismatch
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
@ -642,38 +730,30 @@ func Test_NumpyParserCheckRowCount(t *testing.T) {
AutoID: false,
DataType: schemapb.DataType_Int32,
},
{
FieldID: 106,
Name: "FieldInt64",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
},
},
}
segmentData[105] = &storage.Int32FieldData{
Data: []int32{1, 2, 3, 4},
}
rowCount, primaryKey, err = parser.checkRowCount(segmentData)
assert.Error(t, err)
assert.Zero(t, rowCount)
assert.Nil(t, primaryKey)
// row count mismatch
parser.collectionSchema.Fields = append(parser.collectionSchema.Fields, &schemapb.FieldSchema{
FieldID: 106,
Name: "FieldInt64",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
})
segmentData[106] = &storage.Int64FieldData{
Data: []int64{1, 2, 4},
}
rowCount, primaryKey, err = parser.checkRowCount(segmentData)
parser.collectionInfo.resetSchema(schema)
rowCount, err = parser.checkRowCount(segmentData)
assert.Error(t, err)
assert.Zero(t, rowCount)
assert.Nil(t, primaryKey)
// has dynamic field
parser.collectionSchema = &schemapb.CollectionSchema{
schema = &schemapb.CollectionSchema{
Name: "schema",
Description: "schema",
AutoID: true,
@ -698,10 +778,10 @@ func Test_NumpyParserCheckRowCount(t *testing.T) {
Data: []int64{1, 2, 4},
}
rowCount, primaryKey, err = parser.checkRowCount(segmentData)
parser.collectionInfo.resetSchema(schema)
rowCount, err = parser.checkRowCount(segmentData)
assert.NoError(t, err)
assert.Equal(t, 3, rowCount)
assert.NotNil(t, primaryKey)
}
func Test_NumpyParserSplitFieldsData(t *testing.T) {
@ -712,31 +792,36 @@ func Test_NumpyParserSplitFieldsData(t *testing.T) {
cm := createLocalChunkManager(t)
parser := createNumpyParser(t)
segmentData := make(map[storage.FieldID]storage.FieldData)
t.Run("segemnt data is empty", func(t *testing.T) {
err = parser.splitFieldsData(segmentData, nil)
err = parser.splitFieldsData(make(BlockData), nil)
assert.Error(t, err)
})
files := createSampleNumpyFiles(t, cm)
readers, err := parser.createReaders(files)
assert.NoError(t, err)
defer closeReaders(readers)
for _, reader := range readers {
fieldData, err := parser.readData(reader, 100)
genFieldsDataFunc := func() BlockData {
segmentData := make(BlockData)
files := createSampleNumpyFiles(t, cm)
readers, err := parser.createReaders(files)
assert.NoError(t, err)
segmentData[reader.fieldID] = fieldData
defer closeReaders(readers)
for _, reader := range readers {
fieldData, err := parser.readData(reader, 100)
assert.NoError(t, err)
segmentData[reader.fieldID] = fieldData
}
return segmentData
}
shards := make([]map[storage.FieldID]storage.FieldData, 0, parser.shardNum)
t.Run("shards number mismatch", func(t *testing.T) {
fieldsData := createFieldsData(sampleSchema(), 0)
shards := createShardsData(sampleSchema(), fieldsData, 1, []int64{1})
segmentData := genFieldsDataFunc()
err = parser.splitFieldsData(segmentData, shards)
assert.Error(t, err)
})
t.Run("checkRowCount returns error", func(t *testing.T) {
parser.collectionSchema = &schemapb.CollectionSchema{
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
@ -746,45 +831,64 @@ func Test_NumpyParserSplitFieldsData(t *testing.T) {
AutoID: false,
DataType: schemapb.DataType_Int32,
},
{
FieldID: 106,
Name: "FieldInt64",
IsPrimaryKey: true,
AutoID: false,
DataType: schemapb.DataType_Int64,
},
},
}
for i := 0; i < int(parser.shardNum); i++ {
shards = append(shards, initSegmentData(parser.collectionSchema))
segmentData := make(BlockData)
segmentData[105] = &storage.Int32FieldData{
Data: []int32{1, 2, 3, 4},
}
segmentData[106] = &storage.Int64FieldData{
Data: []int64{1, 2, 4},
}
parser.collectionInfo.resetSchema(schema)
parser.collectionInfo.ShardNum = 2
fieldsData := createFieldsData(schema, 0)
shards := createShardsData(schema, fieldsData, 2, []int64{1})
err = parser.splitFieldsData(segmentData, shards)
assert.Error(t, err)
parser.collectionSchema = sampleSchema()
})
t.Run("failed to alloc id", func(t *testing.T) {
ctx := context.Background()
parser.rowIDAllocator = newIDAllocator(ctx, t, errors.New("dummy error"))
parser.collectionInfo.resetSchema(sampleSchema())
fieldsData := createFieldsData(sampleSchema(), 0)
shards := createShardsData(sampleSchema(), fieldsData, 2, []int64{1})
segmentData := genFieldsDataFunc()
err = parser.splitFieldsData(segmentData, shards)
assert.Error(t, err)
parser.rowIDAllocator = newIDAllocator(ctx, t, nil)
})
t.Run("primary key auto-generated", func(t *testing.T) {
schema := findSchema(parser.collectionSchema, schemapb.DataType_Int64)
parser.collectionInfo.resetSchema(sampleSchema())
schema := findSchema(parser.collectionInfo.Schema, schemapb.DataType_Int64)
schema.AutoID = true
shards = make([]map[storage.FieldID]storage.FieldData, 0, parser.shardNum)
for i := 0; i < int(parser.shardNum); i++ {
segmentData := initSegmentData(parser.collectionSchema)
shards = append(shards, segmentData)
}
partitionID := int64(1)
fieldsData := createFieldsData(sampleSchema(), 0)
shards := createShardsData(sampleSchema(), fieldsData, 2, []int64{partitionID})
segmentData := genFieldsDataFunc()
err = parser.splitFieldsData(segmentData, shards)
assert.NoError(t, err)
assert.NotEmpty(t, parser.autoIDRange)
totalNum := 0
for i := 0; i < int(parser.shardNum); i++ {
totalNum += shards[i][106].RowNum()
for i := 0; i < int(parser.collectionInfo.ShardNum); i++ {
totalNum += shards[i][partitionID][106].RowNum()
}
assert.Equal(t, segmentData[106].RowNum(), totalNum)
// target field data is nil
shards[0][105] = nil
shards[0][partitionID][105] = nil
err = parser.splitFieldsData(segmentData, shards)
assert.Error(t, err)
@ -792,7 +896,7 @@ func Test_NumpyParserSplitFieldsData(t *testing.T) {
})
t.Run("has dynamic field", func(t *testing.T) {
parser.collectionSchema = &schemapb.CollectionSchema{
schema := &schemapb.CollectionSchema{
Name: "schema",
Description: "schema",
AutoID: true,
@ -813,12 +917,10 @@ func Test_NumpyParserSplitFieldsData(t *testing.T) {
},
},
}
shards = make([]map[storage.FieldID]storage.FieldData, 0, parser.shardNum)
for i := 0; i < int(parser.shardNum); i++ {
segmentData := initSegmentData(parser.collectionSchema)
shards = append(shards, segmentData)
}
segmentData = make(map[storage.FieldID]storage.FieldData)
parser.collectionInfo.resetSchema(schema)
fieldsData := createFieldsData(schema, 0)
shards := createShardsData(schema, fieldsData, 2, []int64{1})
segmentData := make(BlockData)
segmentData[101] = &storage.Int64FieldData{
Data: []int64{1, 2, 4},
}
@ -836,7 +938,7 @@ func Test_NumpyParserCalcRowCountPerBlock(t *testing.T) {
assert.Greater(t, rowCount, int64(0))
// failed to estimate row size
parser.collectionSchema = &schemapb.CollectionSchema{
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
@ -855,14 +957,16 @@ func Test_NumpyParserCalcRowCountPerBlock(t *testing.T) {
},
},
}
parser.collectionInfo.Schema = schema
rowCount, err = parser.calcRowCountPerBlock()
assert.Error(t, err)
assert.Zero(t, rowCount)
// no field
parser.collectionSchema = &schemapb.CollectionSchema{
schema = &schemapb.CollectionSchema{
Name: "schema",
}
parser.collectionInfo.Schema = schema
rowCount, err = parser.calcRowCountPerBlock()
assert.Error(t, err)
assert.Zero(t, rowCount)
@ -895,7 +999,7 @@ func Test_NumpyParserConsume(t *testing.T) {
assert.Error(t, err)
// invalid schema
parser.collectionSchema = &schemapb.CollectionSchema{
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
@ -913,6 +1017,7 @@ func Test_NumpyParserConsume(t *testing.T) {
},
},
}
parser.collectionInfo.resetSchema(schema)
err = parser.consume(readers)
assert.Error(t, err)
closeReaders(readers)
@ -933,20 +1038,21 @@ func Test_NumpyParserParse(t *testing.T) {
})
t.Run("file doesnt exist", func(t *testing.T) {
parser.collectionSchema = perfSchema(4)
parser.collectionInfo.resetSchema(perfSchema(4))
files := []string{"ID.npy", "Vector.npy"}
err = parser.Parse(files)
assert.Error(t, err)
parser.collectionSchema = sampleSchema()
})
parser.collectionInfo.resetSchema(sampleSchema())
t.Run("succeed", func(t *testing.T) {
cm := createLocalChunkManager(t)
files := createSampleNumpyFiles(t, cm)
totalRowCount := 0
parser.callFlushFunc = func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
assert.LessOrEqual(t, int32(shardID), parser.shardNum)
parser.callFlushFunc = func(fields BlockData, shardID int, partID int64) error {
assert.LessOrEqual(t, int32(shardID), parser.collectionInfo.ShardNum)
rowCount := 0
for _, fieldData := range fields {
if rowCount == 0 {
@ -1010,7 +1116,7 @@ func Test_NumpyParserParse_perf(t *testing.T) {
shardNum := int32(3)
totalRowCount := 0
callFlushFunc := func(fields map[storage.FieldID]storage.FieldData, shardID int) error {
callFlushFunc := func(fields BlockData, shardID int, partID int64) error {
assert.LessOrEqual(t, int32(shardID), shardNum)
rowCount := 0
for _, fieldData := range fields {
@ -1028,10 +1134,13 @@ func Test_NumpyParserParse_perf(t *testing.T) {
updateProgress := func(percent int64) {
assert.Greater(t, percent, int64(0))
}
parser, err := NewNumpyParser(ctx, perfSchema(dim), idAllocator, shardNum, 16*1024*1024, cm, callFlushFunc, updateProgress)
collectionInfo, err := NewCollectionInfo(perfSchema(dim), shardNum, []int64{1})
assert.NoError(t, err)
parser, err := NewNumpyParser(ctx, collectionInfo, idAllocator, 16*1024*1024, cm, callFlushFunc, updateProgress)
assert.NoError(t, err)
assert.NotNil(t, parser)
parser.collectionSchema = perfSchema(dim)
err = parser.Parse([]string{idFilePath, vecFilePath})
assert.NoError(t, err)
@ -1039,3 +1148,75 @@ func Test_NumpyParserParse_perf(t *testing.T) {
tr.Record("parse large numpy files")
}
func Test_NumpyParserHashToPartition(t *testing.T) {
ctx := context.Background()
schema := &schemapb.CollectionSchema{
Name: "schema",
Fields: []*schemapb.FieldSchema{
{
FieldID: 101,
Name: "FieldInt64",
IsPrimaryKey: true,
AutoID: true,
DataType: schemapb.DataType_Int64,
},
{
FieldID: 102,
Name: "FieldVarchar",
DataType: schemapb.DataType_VarChar,
},
{
FieldID: 103,
Name: "FieldFloat",
DataType: schemapb.DataType_Float,
},
},
}
idAllocator := newIDAllocator(ctx, t, nil)
cm := createLocalChunkManager(t)
flushFunc := func(fields BlockData, shardID int, partID int64) error {
return nil
}
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
assert.NoError(t, err)
parser, err := NewNumpyParser(ctx, collectionInfo, idAllocator, 100, cm, flushFunc, nil)
assert.NoError(t, err)
assert.NotNil(t, parser)
fieldsData := createFieldsData(schema, 5)
blockData := createBlockData(schema, fieldsData)
// no partition key, partition ID list greater than 1, return error
parser.collectionInfo.PartitionIDs = []int64{1, 2}
partID, err := parser.hashToPartition(blockData, 1)
assert.Error(t, err)
assert.Zero(t, partID)
// no partition key, return the only one partition ID
partitionID := int64(5)
parser.collectionInfo.PartitionIDs = []int64{partitionID}
partID, err = parser.hashToPartition(blockData, 1)
assert.NoError(t, err)
assert.Equal(t, partitionID, partID)
// has partition key
schema.Fields[1].IsPartitionKey = true
err = parser.collectionInfo.resetSchema(schema)
assert.NoError(t, err)
partitionIDs := []int64{3, 4, 5, 6}
partID, err = parser.hashToPartition(blockData, 1)
assert.NoError(t, err)
assert.Contains(t, partitionIDs, partID)
// has partition key, but value is invalid
blockData[102] = &storage.FloatFieldData{
Data: []float32{1, 2, 3, 4, 5},
}
partID, err = parser.hashToPartition(blockData, 1)
assert.Error(t, err)
assert.Zero(t, partID)
}

View File

@ -17,7 +17,10 @@
package typeutil
import (
"fmt"
"hash/crc32"
"strconv"
"strings"
"unsafe"
"github.com/cockroachdb/errors"
@ -126,3 +129,28 @@ func HashKey2Partitions(keys *schemapb.FieldData, partitionNames []string) ([]ui
return hashValues, nil
}
// this method returns a static sequence for partitions for partiton key mode
func RearrangePartitionsForPartitionKey(partitions map[string]int64) ([]string, []int64, error) {
// Make sure the order of the partition names got every time is the same
partitionNames := make([]string, len(partitions))
partitionIDs := make([]int64, len(partitions))
for partitionName, partitionID := range partitions {
splits := strings.Split(partitionName, "_")
if len(splits) < 2 {
return nil, nil, fmt.Errorf("bad default partion name in partition key mode: %s", partitionName)
}
index, err := strconv.ParseInt(splits[len(splits)-1], 10, 64)
if err != nil {
return nil, nil, err
}
if (index >= int64(len(partitions))) || (index < 0) {
return nil, nil, fmt.Errorf("illegal partition index in partition key mode: %s", partitionName)
}
partitionNames[index] = partitionName
partitionIDs[index] = partitionID
}
return partitionNames, partitionIDs, nil
}

View File

@ -94,3 +94,69 @@ func TestHashPK2Channels(t *testing.T) {
assert.Equal(t, 5, len(ret))
assert.Equal(t, ret[1], ret[2])
}
func TestRearrangePartitionsForPartitionKey(t *testing.T) {
// invalid partition name
partitions := map[string]int64{
"invalid": 1,
}
partitionNames, partitionIDs, err := RearrangePartitionsForPartitionKey(partitions)
assert.Error(t, err)
assert.Nil(t, partitionNames)
assert.Nil(t, partitionIDs)
// invalid partition index
partitions = map[string]int64{
"invalid_a": 1,
}
partitionNames, partitionIDs, err = RearrangePartitionsForPartitionKey(partitions)
assert.Error(t, err)
assert.Nil(t, partitionNames)
assert.Nil(t, partitionIDs)
partitions = map[string]int64{
"invalid_1": 1,
}
partitionNames, partitionIDs, err = RearrangePartitionsForPartitionKey(partitions)
assert.Error(t, err)
assert.Nil(t, partitionNames)
assert.Nil(t, partitionIDs)
// success cases
validateFunc := func(partitions map[string]int64) {
partitionNames, partitionIDs, err = RearrangePartitionsForPartitionKey(partitions)
assert.NoError(t, err)
assert.Equal(t, len(partitions), len(partitionNames))
assert.Equal(t, len(partitions), len(partitionIDs))
for i := 0; i < len(partitions); i++ {
assert.Contains(t, partitions, partitionNames[i])
assert.Equal(t, partitions[partitionNames[i]], partitionIDs[i])
if i > 0 {
assert.Greater(t, partitionIDs[i], partitionIDs[i-1])
}
}
}
validateFunc(map[string]int64{
"p_0": 1,
"p_1": 2,
"p_2": 3,
})
validateFunc(map[string]int64{
"p_2": 3,
"p_1": 2,
"p_0": 1,
})
validateFunc(map[string]int64{
"p_1": 2,
"p_2": 3,
"p_0": 1,
})
}