mirror of https://github.com/milvus-io/milvus.git
feat: Add import executor and manager for datanode (#29438)
This PR introduces novel importv2 roles for datanode: 1. Executor: To execute tasks, a import task will be divided into the following steps: read data -> hash data -> sync data; 2. Manager: To manage all the tasks; issue: https://github.com/milvus-io/milvus/issues/28521 --------- Signed-off-by: bigsheeper <yihao.dai@zilliz.com>pull/30425/head
parent
fc0d007bd1
commit
c5918290e6
|
@ -424,6 +424,10 @@ dataCoord:
|
|||
forceTrigger:
|
||||
minSize: 8 # The minmum size in MB to force trigger a LevelZero Compaction
|
||||
deltalogMinNum: 10 # the minimum number of deltalog files to force trigger a LevelZero Compaction
|
||||
import:
|
||||
filesPerPreImportTask: 2 # The maximum number of files allowed per pre-import task.
|
||||
taskRetention: 10800 # The retention period in seconds for tasks in the Completed or Failed state.
|
||||
inactiveTimeout: 1800 # The timeout duration in seconds for a task in the "InProgress" state if it remains inactive (with no progress updates).
|
||||
|
||||
enableGarbageCollection: true
|
||||
gc:
|
||||
|
@ -482,6 +486,8 @@ dataNode:
|
|||
# if this parameter <= 0, will set it as 1000
|
||||
# suggest to set it bigger on large collection numbers to avoid blocking
|
||||
updateChannelCheckpointMaxParallel: 1000
|
||||
import:
|
||||
maxConcurrentTaskNum: 16 # The maximum number of import/pre-import tasks allowed to run concurrently on a datanode.
|
||||
|
||||
# Configures the system log output.
|
||||
log:
|
||||
|
|
|
@ -37,6 +37,7 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
|
@ -92,6 +93,7 @@ type DataNode struct {
|
|||
|
||||
syncMgr syncmgr.SyncManager
|
||||
writeBufferManager writebuffer.BufferManager
|
||||
importManager *importv2.Manager
|
||||
|
||||
clearSignal chan string // vchannel name
|
||||
segmentCache *Cache
|
||||
|
@ -286,6 +288,8 @@ func (node *DataNode) Init() error {
|
|||
|
||||
node.writeBufferManager = writebuffer.NewManager(syncMgr)
|
||||
|
||||
node.importManager = importv2.NewManager(node.syncMgr, node.chunkManager)
|
||||
|
||||
node.channelCheckpointUpdater = newChannelCheckpointUpdater(node)
|
||||
|
||||
log.Info("init datanode done", zap.Int64("nodeID", paramtable.GetNodeID()), zap.String("Address", node.address))
|
||||
|
@ -379,6 +383,8 @@ func (node *DataNode) Start() error {
|
|||
|
||||
go node.compactionExecutor.start(node.ctx)
|
||||
|
||||
go node.importManager.Start()
|
||||
|
||||
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
|
||||
node.timeTickSender = newTimeTickSender(node.broker, node.session.ServerID,
|
||||
retry.Attempts(20), retry.Sleep(time.Millisecond*100))
|
||||
|
@ -453,6 +459,10 @@ func (node *DataNode) Stop() error {
|
|||
node.channelCheckpointUpdater.close()
|
||||
}
|
||||
|
||||
if node.importManager != nil {
|
||||
node.importManager.Close()
|
||||
}
|
||||
|
||||
node.stopWaiter.Wait()
|
||||
})
|
||||
return nil
|
||||
|
|
|
@ -0,0 +1,322 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/conc"
|
||||
"github.com/milvus-io/milvus/pkg/util/hardware"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
type Executor interface {
|
||||
Start()
|
||||
Slots() int64
|
||||
Close()
|
||||
}
|
||||
|
||||
type executor struct {
|
||||
manager TaskManager
|
||||
syncMgr syncmgr.SyncManager
|
||||
cm storage.ChunkManager
|
||||
|
||||
pool *conc.Pool[any]
|
||||
|
||||
closeOnce sync.Once
|
||||
closeChan chan struct{}
|
||||
}
|
||||
|
||||
func NewExecutor(manager TaskManager, syncMgr syncmgr.SyncManager, cm storage.ChunkManager) Executor {
|
||||
pool := conc.NewPool[any](
|
||||
hardware.GetCPUNum()*2,
|
||||
conc.WithPreAlloc(false),
|
||||
conc.WithDisablePurge(false),
|
||||
)
|
||||
return &executor{
|
||||
manager: manager,
|
||||
syncMgr: syncMgr,
|
||||
cm: cm,
|
||||
pool: pool,
|
||||
closeChan: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
func (e *executor) Start() {
|
||||
log.Info("start import executor")
|
||||
var (
|
||||
exeTicker = time.NewTicker(1 * time.Second)
|
||||
logTicker = time.NewTicker(10 * time.Minute)
|
||||
)
|
||||
defer exeTicker.Stop()
|
||||
defer logTicker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-e.closeChan:
|
||||
log.Info("import executor exited")
|
||||
return
|
||||
case <-exeTicker.C:
|
||||
tasks := e.manager.GetBy(WithStates(internalpb.ImportState_Pending))
|
||||
wg := &sync.WaitGroup{}
|
||||
for _, task := range tasks {
|
||||
wg.Add(1)
|
||||
go func(task Task) {
|
||||
defer wg.Done()
|
||||
switch task.GetType() {
|
||||
case PreImportTaskType:
|
||||
e.PreImport(task)
|
||||
case ImportTaskType:
|
||||
e.Import(task)
|
||||
}
|
||||
}(task)
|
||||
}
|
||||
wg.Wait()
|
||||
case <-logTicker.C:
|
||||
LogStats(e.manager)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (e *executor) Slots() int64 {
|
||||
tasks := e.manager.GetBy(WithStates(internalpb.ImportState_Pending, internalpb.ImportState_InProgress))
|
||||
return paramtable.Get().DataNodeCfg.MaxConcurrentImportTaskNum.GetAsInt64() - int64(len(tasks))
|
||||
}
|
||||
|
||||
func (e *executor) Close() {
|
||||
e.closeOnce.Do(func() {
|
||||
close(e.closeChan)
|
||||
})
|
||||
}
|
||||
|
||||
func WrapLogFields(task Task, fields ...zap.Field) []zap.Field {
|
||||
res := []zap.Field{
|
||||
zap.Int64("taskID", task.GetTaskID()),
|
||||
zap.Int64("jobID", task.GetJobID()),
|
||||
zap.Int64("collectionID", task.GetCollectionID()),
|
||||
zap.String("type", task.GetType().String()),
|
||||
}
|
||||
res = append(res, fields...)
|
||||
return res
|
||||
}
|
||||
|
||||
func (e *executor) handleErr(task Task, err error, msg string) {
|
||||
log.Warn(msg, WrapLogFields(task, zap.Error(err))...)
|
||||
e.manager.Update(task.GetTaskID(), UpdateState(internalpb.ImportState_Failed), UpdateReason(err.Error()))
|
||||
}
|
||||
|
||||
func (e *executor) PreImport(task Task) {
|
||||
bufferSize := paramtable.Get().DataNodeCfg.FlushInsertBufferSize.GetAsInt()
|
||||
log.Info("start to preimport", WrapLogFields(task,
|
||||
zap.Int("bufferSize", bufferSize),
|
||||
zap.Any("schema", task.GetSchema()))...)
|
||||
e.manager.Update(task.GetTaskID(), UpdateState(internalpb.ImportState_InProgress))
|
||||
files := lo.Map(task.(*PreImportTask).GetFileStats(),
|
||||
func(fileStat *datapb.ImportFileStats, _ int) *internalpb.ImportFile {
|
||||
return fileStat.GetImportFile()
|
||||
})
|
||||
|
||||
fn := func(i int, file *internalpb.ImportFile) error {
|
||||
reader, err := importutilv2.NewReader(task.GetCtx(), e.cm, task.GetSchema(), file, task.GetOptions(), bufferSize)
|
||||
if err != nil {
|
||||
e.handleErr(task, err, "new reader failed")
|
||||
return err
|
||||
}
|
||||
defer reader.Close()
|
||||
start := time.Now()
|
||||
err = e.readFileStat(reader, task, i)
|
||||
if err != nil {
|
||||
e.handleErr(task, err, "preimport failed")
|
||||
return err
|
||||
}
|
||||
log.Info("read file stat done", WrapLogFields(task, zap.Strings("files", file.GetPaths()),
|
||||
zap.Duration("dur", time.Since(start)))...)
|
||||
return nil
|
||||
}
|
||||
|
||||
futures := make([]*conc.Future[any], 0, len(files))
|
||||
for i, file := range files {
|
||||
i := i
|
||||
file := file
|
||||
f := e.pool.Submit(func() (any, error) {
|
||||
err := fn(i, file)
|
||||
return err, err
|
||||
})
|
||||
futures = append(futures, f)
|
||||
}
|
||||
err := conc.AwaitAll(futures...)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
e.manager.Update(task.GetTaskID(), UpdateState(internalpb.ImportState_Completed))
|
||||
log.Info("executor preimport done",
|
||||
WrapLogFields(task, zap.Any("fileStats", task.(*PreImportTask).GetFileStats()))...)
|
||||
}
|
||||
|
||||
func (e *executor) readFileStat(reader importutilv2.Reader, task Task, fileIdx int) error {
|
||||
totalRows := 0
|
||||
hashedRows := make(map[string]*datapb.PartitionRows)
|
||||
for {
|
||||
data, err := reader.Read()
|
||||
if err != nil {
|
||||
if errors.Is(err, io.EOF) {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
err = CheckRowsEqual(task.GetSchema(), data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
rowsCount, err := GetRowsStats(task, data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
MergeHashedRowsCount(rowsCount, hashedRows)
|
||||
rows := data.GetRowNum()
|
||||
totalRows += rows
|
||||
log.Info("reading file stat...", WrapLogFields(task, zap.Int("readRows", rows))...)
|
||||
}
|
||||
|
||||
stat := &datapb.ImportFileStats{
|
||||
TotalRows: int64(totalRows),
|
||||
HashedRows: hashedRows,
|
||||
}
|
||||
e.manager.Update(task.GetTaskID(), UpdateFileStat(fileIdx, stat))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *executor) Import(task Task) {
|
||||
bufferSize := paramtable.Get().DataNodeCfg.FlushInsertBufferSize.GetAsInt()
|
||||
log.Info("start to import", WrapLogFields(task,
|
||||
zap.Int("bufferSize", bufferSize),
|
||||
zap.Any("schema", task.GetSchema()))...)
|
||||
e.manager.Update(task.GetTaskID(), UpdateState(internalpb.ImportState_InProgress))
|
||||
|
||||
req := task.(*ImportTask).req
|
||||
|
||||
fn := func(file *internalpb.ImportFile) error {
|
||||
reader, err := importutilv2.NewReader(task.GetCtx(), e.cm, task.GetSchema(), file, task.GetOptions(), bufferSize)
|
||||
if err != nil {
|
||||
e.handleErr(task, err, fmt.Sprintf("new reader failed, file: %s", file.String()))
|
||||
return err
|
||||
}
|
||||
defer reader.Close()
|
||||
start := time.Now()
|
||||
err = e.importFile(reader, task)
|
||||
if err != nil {
|
||||
e.handleErr(task, err, fmt.Sprintf("do import failed, file: %s", file.String()))
|
||||
return err
|
||||
}
|
||||
log.Info("import file done", WrapLogFields(task, zap.Strings("files", file.GetPaths()),
|
||||
zap.Duration("dur", time.Since(start)))...)
|
||||
return nil
|
||||
}
|
||||
|
||||
futures := make([]*conc.Future[any], 0, len(req.GetFiles()))
|
||||
for _, file := range req.GetFiles() {
|
||||
file := file
|
||||
f := e.pool.Submit(func() (any, error) {
|
||||
err := fn(file)
|
||||
return err, err
|
||||
})
|
||||
futures = append(futures, f)
|
||||
}
|
||||
err := conc.AwaitAll(futures...)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
e.manager.Update(task.GetTaskID(), UpdateState(internalpb.ImportState_Completed))
|
||||
log.Info("import done", WrapLogFields(task)...)
|
||||
}
|
||||
|
||||
func (e *executor) importFile(reader importutilv2.Reader, task Task) error {
|
||||
iTask := task.(*ImportTask)
|
||||
futures := make([]*conc.Future[error], 0)
|
||||
syncTasks := make([]syncmgr.Task, 0)
|
||||
for {
|
||||
data, err := reader.Read()
|
||||
if err != nil {
|
||||
if errors.Is(err, io.EOF) {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
err = AppendSystemFieldsData(iTask, data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
hashedData, err := HashData(iTask, data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
fs, sts, err := e.Sync(iTask, hashedData)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
futures = append(futures, fs...)
|
||||
syncTasks = append(syncTasks, sts...)
|
||||
}
|
||||
err := conc.AwaitAll(futures...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, syncTask := range syncTasks {
|
||||
segmentInfo, err := NewImportSegmentInfo(syncTask, iTask)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
e.manager.Update(task.GetTaskID(), UpdateSegmentInfo(segmentInfo))
|
||||
log.Info("sync import data done", WrapLogFields(task, zap.Any("segmentInfo", segmentInfo))...)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (e *executor) Sync(task *ImportTask, hashedData HashedData) ([]*conc.Future[error], []syncmgr.Task, error) {
|
||||
log.Info("start to sync import data", WrapLogFields(task)...)
|
||||
futures := make([]*conc.Future[error], 0)
|
||||
syncTasks := make([]syncmgr.Task, 0)
|
||||
for channelIdx, datas := range hashedData {
|
||||
channel := task.vchannels[channelIdx]
|
||||
for partitionIdx, data := range datas {
|
||||
partitionID := task.partitions[partitionIdx]
|
||||
segmentID := PickSegment(task, channel, partitionID, data.GetRowNum())
|
||||
syncTask, err := NewSyncTask(task.GetCtx(), task, segmentID, partitionID, channel, data)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
future := e.syncMgr.SyncData(task.GetCtx(), syncTask)
|
||||
futures = append(futures, future)
|
||||
syncTasks = append(syncTasks, syncTask)
|
||||
}
|
||||
}
|
||||
return futures, syncTasks, nil
|
||||
}
|
|
@ -0,0 +1,536 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
rand2 "crypto/rand"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"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/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/mocks"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/conc"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type sampleRow struct {
|
||||
FieldString string `json:"pk,omitempty"`
|
||||
FieldInt64 int64 `json:"int64,omitempty"`
|
||||
FieldFloatVector []float32 `json:"vec,omitempty"`
|
||||
}
|
||||
|
||||
type sampleContent struct {
|
||||
Rows []sampleRow `json:"rows,omitempty"`
|
||||
}
|
||||
|
||||
type mockReader struct {
|
||||
io.Reader
|
||||
io.Closer
|
||||
io.ReaderAt
|
||||
io.Seeker
|
||||
}
|
||||
|
||||
type ExecutorSuite struct {
|
||||
suite.Suite
|
||||
|
||||
numRows int
|
||||
schema *schemapb.CollectionSchema
|
||||
|
||||
cm storage.ChunkManager
|
||||
reader *importutilv2.MockReader
|
||||
syncMgr *syncmgr.MockSyncManager
|
||||
manager TaskManager
|
||||
executor *executor
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) SetupSuite() {
|
||||
paramtable.Init()
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) SetupTest() {
|
||||
s.numRows = 100
|
||||
s.schema = &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 100,
|
||||
Name: "pk",
|
||||
IsPrimaryKey: true,
|
||||
DataType: schemapb.DataType_VarChar,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.MaxLengthKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "vec",
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DimKey,
|
||||
Value: "4",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
FieldID: 102,
|
||||
Name: "int64",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
s.manager = NewTaskManager()
|
||||
s.syncMgr = syncmgr.NewMockSyncManager(s.T())
|
||||
s.executor = NewExecutor(s.manager, s.syncMgr, nil).(*executor)
|
||||
}
|
||||
|
||||
func createInsertData(t *testing.T, schema *schemapb.CollectionSchema, rowCount int) *storage.InsertData {
|
||||
insertData, err := storage.NewInsertData(schema)
|
||||
assert.NoError(t, err)
|
||||
for _, field := range schema.GetFields() {
|
||||
switch field.GetDataType() {
|
||||
case schemapb.DataType_Bool:
|
||||
boolData := make([]bool, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
boolData = append(boolData, i%3 != 0)
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.BoolFieldData{Data: boolData}
|
||||
case schemapb.DataType_Float:
|
||||
floatData := make([]float32, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
floatData = append(floatData, float32(i/2))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.FloatFieldData{Data: floatData}
|
||||
case schemapb.DataType_Double:
|
||||
doubleData := make([]float64, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
doubleData = append(doubleData, float64(i/5))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.DoubleFieldData{Data: doubleData}
|
||||
case schemapb.DataType_Int8:
|
||||
int8Data := make([]int8, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
int8Data = append(int8Data, int8(i%256))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.Int8FieldData{Data: int8Data}
|
||||
case schemapb.DataType_Int16:
|
||||
int16Data := make([]int16, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
int16Data = append(int16Data, int16(i%65536))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.Int16FieldData{Data: int16Data}
|
||||
case schemapb.DataType_Int32:
|
||||
int32Data := make([]int32, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
int32Data = append(int32Data, int32(i%1000))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.Int32FieldData{Data: int32Data}
|
||||
case schemapb.DataType_Int64:
|
||||
int64Data := make([]int64, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
int64Data = append(int64Data, int64(i))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.Int64FieldData{Data: int64Data}
|
||||
case schemapb.DataType_BinaryVector:
|
||||
dim, err := typeutil.GetDim(field)
|
||||
assert.NoError(t, err)
|
||||
binVecData := make([]byte, 0)
|
||||
total := rowCount * int(dim) / 8
|
||||
for i := 0; i < total; i++ {
|
||||
binVecData = append(binVecData, byte(i%256))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.BinaryVectorFieldData{Data: binVecData, Dim: int(dim)}
|
||||
case schemapb.DataType_FloatVector:
|
||||
dim, err := typeutil.GetDim(field)
|
||||
assert.NoError(t, err)
|
||||
floatVecData := make([]float32, 0)
|
||||
total := rowCount * int(dim)
|
||||
for i := 0; i < total; i++ {
|
||||
floatVecData = append(floatVecData, rand.Float32())
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.FloatVectorFieldData{Data: floatVecData, Dim: int(dim)}
|
||||
case schemapb.DataType_Float16Vector:
|
||||
dim, err := typeutil.GetDim(field)
|
||||
assert.NoError(t, err)
|
||||
total := int64(rowCount) * dim * 2
|
||||
float16VecData := make([]byte, total)
|
||||
_, err = rand2.Read(float16VecData)
|
||||
assert.NoError(t, err)
|
||||
insertData.Data[field.GetFieldID()] = &storage.Float16VectorFieldData{Data: float16VecData, Dim: int(dim)}
|
||||
case schemapb.DataType_String, schemapb.DataType_VarChar:
|
||||
varcharData := make([]string, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
varcharData = append(varcharData, strconv.Itoa(i))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.StringFieldData{Data: varcharData}
|
||||
case schemapb.DataType_JSON:
|
||||
jsonData := make([][]byte, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
jsonData = append(jsonData, []byte(fmt.Sprintf("{\"y\": %d}", i)))
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.JSONFieldData{Data: jsonData}
|
||||
case schemapb.DataType_Array:
|
||||
arrayData := make([]*schemapb.ScalarField, 0)
|
||||
for i := 0; i < rowCount; i++ {
|
||||
arrayData = append(arrayData, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: []int32{int32(i), int32(i + 1), int32(i + 2)},
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
insertData.Data[field.GetFieldID()] = &storage.ArrayFieldData{Data: arrayData}
|
||||
default:
|
||||
panic(fmt.Sprintf("unexpected data type: %s", field.GetDataType().String()))
|
||||
}
|
||||
}
|
||||
return insertData
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) TestExecutor_Slots() {
|
||||
preimportReq := &datapb.PreImportRequest{
|
||||
JobID: 1,
|
||||
TaskID: 2,
|
||||
CollectionID: 3,
|
||||
PartitionIDs: []int64{4},
|
||||
Vchannels: []string{"ch-0"},
|
||||
Schema: s.schema,
|
||||
ImportFiles: []*internalpb.ImportFile{{Paths: []string{"dummy.json"}}},
|
||||
}
|
||||
preimportTask := NewPreImportTask(preimportReq)
|
||||
s.manager.Add(preimportTask)
|
||||
|
||||
slots := s.executor.Slots()
|
||||
s.Equal(paramtable.Get().DataNodeCfg.MaxConcurrentImportTaskNum.GetAsInt64()-1, slots)
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) TestExecutor_Start_Preimport() {
|
||||
content := &sampleContent{
|
||||
Rows: make([]sampleRow, 0),
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
row := sampleRow{
|
||||
FieldString: "No." + strconv.FormatInt(int64(i), 10),
|
||||
FieldInt64: int64(99999999999999999 + i),
|
||||
FieldFloatVector: []float32{float32(i) + 0.1, float32(i) + 0.2, float32(i) + 0.3, float32(i) + 0.4},
|
||||
}
|
||||
content.Rows = append(content.Rows, row)
|
||||
}
|
||||
bytes, err := json.Marshal(content)
|
||||
s.NoError(err)
|
||||
|
||||
cm := mocks.NewChunkManager(s.T())
|
||||
ioReader := strings.NewReader(string(bytes))
|
||||
cm.EXPECT().Reader(mock.Anything, mock.Anything).Return(&mockReader{Reader: ioReader}, nil)
|
||||
s.executor.cm = cm
|
||||
|
||||
preimportReq := &datapb.PreImportRequest{
|
||||
JobID: 1,
|
||||
TaskID: 2,
|
||||
CollectionID: 3,
|
||||
PartitionIDs: []int64{4},
|
||||
Vchannels: []string{"ch-0"},
|
||||
Schema: s.schema,
|
||||
ImportFiles: []*internalpb.ImportFile{{Paths: []string{"dummy.json"}}},
|
||||
}
|
||||
preimportTask := NewPreImportTask(preimportReq)
|
||||
s.manager.Add(preimportTask)
|
||||
|
||||
go s.executor.Start()
|
||||
defer s.executor.Close()
|
||||
s.Eventually(func() bool {
|
||||
return s.manager.Get(preimportTask.GetTaskID()).GetState() == internalpb.ImportState_Completed
|
||||
}, 10*time.Second, 100*time.Millisecond)
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) TestExecutor_Start_Preimport_Failed() {
|
||||
content := &sampleContent{
|
||||
Rows: make([]sampleRow, 0),
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
var row sampleRow
|
||||
if i == 0 { // make rows not consistent
|
||||
row = sampleRow{
|
||||
FieldString: "No." + strconv.FormatInt(int64(i), 10),
|
||||
FieldFloatVector: []float32{float32(i) + 0.1, float32(i) + 0.2, float32(i) + 0.3, float32(i) + 0.4},
|
||||
}
|
||||
} else {
|
||||
row = sampleRow{
|
||||
FieldString: "No." + strconv.FormatInt(int64(i), 10),
|
||||
FieldInt64: int64(99999999999999999 + i),
|
||||
FieldFloatVector: []float32{float32(i) + 0.1, float32(i) + 0.2, float32(i) + 0.3, float32(i) + 0.4},
|
||||
}
|
||||
}
|
||||
content.Rows = append(content.Rows, row)
|
||||
}
|
||||
bytes, err := json.Marshal(content)
|
||||
s.NoError(err)
|
||||
|
||||
cm := mocks.NewChunkManager(s.T())
|
||||
type mockReader struct {
|
||||
io.Reader
|
||||
io.Closer
|
||||
io.ReaderAt
|
||||
io.Seeker
|
||||
}
|
||||
ioReader := strings.NewReader(string(bytes))
|
||||
cm.EXPECT().Reader(mock.Anything, mock.Anything).Return(&mockReader{Reader: ioReader}, nil)
|
||||
s.executor.cm = cm
|
||||
|
||||
preimportReq := &datapb.PreImportRequest{
|
||||
JobID: 1,
|
||||
TaskID: 2,
|
||||
CollectionID: 3,
|
||||
PartitionIDs: []int64{4},
|
||||
Vchannels: []string{"ch-0"},
|
||||
Schema: s.schema,
|
||||
ImportFiles: []*internalpb.ImportFile{{Paths: []string{"dummy.json"}}},
|
||||
}
|
||||
preimportTask := NewPreImportTask(preimportReq)
|
||||
s.manager.Add(preimportTask)
|
||||
|
||||
go s.executor.Start()
|
||||
defer s.executor.Close()
|
||||
s.Eventually(func() bool {
|
||||
return s.manager.Get(preimportTask.GetTaskID()).GetState() == internalpb.ImportState_Failed
|
||||
}, 10*time.Second, 100*time.Millisecond)
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) TestExecutor_Start_Import() {
|
||||
content := &sampleContent{
|
||||
Rows: make([]sampleRow, 0),
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
row := sampleRow{
|
||||
FieldString: "No." + strconv.FormatInt(int64(i), 10),
|
||||
FieldInt64: int64(99999999999999999 + i),
|
||||
FieldFloatVector: []float32{float32(i) + 0.1, float32(i) + 0.2, float32(i) + 0.3, float32(i) + 0.4},
|
||||
}
|
||||
content.Rows = append(content.Rows, row)
|
||||
}
|
||||
bytes, err := json.Marshal(content)
|
||||
s.NoError(err)
|
||||
|
||||
cm := mocks.NewChunkManager(s.T())
|
||||
ioReader := strings.NewReader(string(bytes))
|
||||
cm.EXPECT().Reader(mock.Anything, mock.Anything).Return(&mockReader{Reader: ioReader}, nil)
|
||||
s.executor.cm = cm
|
||||
|
||||
s.syncMgr.EXPECT().SyncData(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, task syncmgr.Task) *conc.Future[error] {
|
||||
future := conc.Go(func() (error, error) {
|
||||
return nil, nil
|
||||
})
|
||||
return future
|
||||
})
|
||||
importReq := &datapb.ImportRequest{
|
||||
JobID: 10,
|
||||
TaskID: 11,
|
||||
CollectionID: 12,
|
||||
Schema: s.schema,
|
||||
Files: []*internalpb.ImportFile{
|
||||
{
|
||||
Paths: []string{"dummy.json"},
|
||||
},
|
||||
},
|
||||
Ts: 1000,
|
||||
AutoIDRange: &datapb.AutoIDRange{
|
||||
Begin: 0,
|
||||
End: int64(s.numRows),
|
||||
},
|
||||
RequestSegments: []*datapb.ImportRequestSegment{
|
||||
{
|
||||
SegmentID: 13,
|
||||
PartitionID: 14,
|
||||
Vchannel: "v0",
|
||||
},
|
||||
},
|
||||
}
|
||||
importTask := NewImportTask(importReq)
|
||||
s.manager.Add(importTask)
|
||||
|
||||
go s.executor.Start()
|
||||
defer s.executor.Close()
|
||||
s.Eventually(func() bool {
|
||||
return s.manager.Get(importTask.GetTaskID()).GetState() == internalpb.ImportState_Completed
|
||||
}, 10*time.Second, 100*time.Millisecond)
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) TestExecutor_Start_Import_Failed() {
|
||||
content := &sampleContent{
|
||||
Rows: make([]sampleRow, 0),
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
row := sampleRow{
|
||||
FieldString: "No." + strconv.FormatInt(int64(i), 10),
|
||||
FieldInt64: int64(99999999999999999 + i),
|
||||
FieldFloatVector: []float32{float32(i) + 0.1, float32(i) + 0.2, float32(i) + 0.3, float32(i) + 0.4},
|
||||
}
|
||||
content.Rows = append(content.Rows, row)
|
||||
}
|
||||
bytes, err := json.Marshal(content)
|
||||
s.NoError(err)
|
||||
|
||||
cm := mocks.NewChunkManager(s.T())
|
||||
ioReader := strings.NewReader(string(bytes))
|
||||
cm.EXPECT().Reader(mock.Anything, mock.Anything).Return(&mockReader{Reader: ioReader}, nil)
|
||||
s.executor.cm = cm
|
||||
|
||||
s.syncMgr.EXPECT().SyncData(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, task syncmgr.Task) *conc.Future[error] {
|
||||
future := conc.Go(func() (error, error) {
|
||||
return errors.New("mock err"), errors.New("mock err")
|
||||
})
|
||||
return future
|
||||
})
|
||||
importReq := &datapb.ImportRequest{
|
||||
JobID: 10,
|
||||
TaskID: 11,
|
||||
CollectionID: 12,
|
||||
Schema: s.schema,
|
||||
Files: []*internalpb.ImportFile{
|
||||
{
|
||||
Paths: []string{"dummy.json"},
|
||||
},
|
||||
},
|
||||
Ts: 1000,
|
||||
AutoIDRange: &datapb.AutoIDRange{
|
||||
Begin: 0,
|
||||
End: int64(s.numRows),
|
||||
},
|
||||
RequestSegments: []*datapb.ImportRequestSegment{
|
||||
{
|
||||
SegmentID: 13,
|
||||
PartitionID: 14,
|
||||
Vchannel: "v0",
|
||||
},
|
||||
},
|
||||
}
|
||||
importTask := NewImportTask(importReq)
|
||||
s.manager.Add(importTask)
|
||||
|
||||
go s.executor.Start()
|
||||
defer s.executor.Close()
|
||||
s.Eventually(func() bool {
|
||||
return s.manager.Get(importTask.GetTaskID()).GetState() == internalpb.ImportState_Failed
|
||||
}, 10*time.Second, 100*time.Millisecond)
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) TestExecutor_ReadFileStat() {
|
||||
importFile := &internalpb.ImportFile{
|
||||
Paths: []string{"dummy.json"},
|
||||
}
|
||||
var once sync.Once
|
||||
data := createInsertData(s.T(), s.schema, s.numRows)
|
||||
s.reader = importutilv2.NewMockReader(s.T())
|
||||
s.reader.EXPECT().Read().RunAndReturn(func() (*storage.InsertData, error) {
|
||||
var res *storage.InsertData
|
||||
once.Do(func() {
|
||||
res = data
|
||||
})
|
||||
if res != nil {
|
||||
return res, nil
|
||||
}
|
||||
return nil, io.EOF
|
||||
})
|
||||
preimportReq := &datapb.PreImportRequest{
|
||||
JobID: 1,
|
||||
TaskID: 2,
|
||||
CollectionID: 3,
|
||||
PartitionIDs: []int64{4},
|
||||
Vchannels: []string{"ch-0"},
|
||||
Schema: s.schema,
|
||||
ImportFiles: []*internalpb.ImportFile{importFile},
|
||||
}
|
||||
preimportTask := NewPreImportTask(preimportReq)
|
||||
s.manager.Add(preimportTask)
|
||||
err := s.executor.readFileStat(s.reader, preimportTask, 0)
|
||||
s.NoError(err)
|
||||
}
|
||||
|
||||
func (s *ExecutorSuite) TestExecutor_ImportFile() {
|
||||
s.syncMgr.EXPECT().SyncData(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, task syncmgr.Task) *conc.Future[error] {
|
||||
future := conc.Go(func() (error, error) {
|
||||
return nil, nil
|
||||
})
|
||||
return future
|
||||
})
|
||||
var once sync.Once
|
||||
data := createInsertData(s.T(), s.schema, s.numRows)
|
||||
s.reader = importutilv2.NewMockReader(s.T())
|
||||
s.reader.EXPECT().Read().RunAndReturn(func() (*storage.InsertData, error) {
|
||||
var res *storage.InsertData
|
||||
once.Do(func() {
|
||||
res = data
|
||||
})
|
||||
if res != nil {
|
||||
return res, nil
|
||||
}
|
||||
return nil, io.EOF
|
||||
})
|
||||
importReq := &datapb.ImportRequest{
|
||||
JobID: 10,
|
||||
TaskID: 11,
|
||||
CollectionID: 12,
|
||||
Schema: s.schema,
|
||||
Files: []*internalpb.ImportFile{
|
||||
{
|
||||
Paths: []string{"dummy.json"},
|
||||
},
|
||||
},
|
||||
Ts: 1000,
|
||||
AutoIDRange: &datapb.AutoIDRange{
|
||||
Begin: 0,
|
||||
End: int64(s.numRows),
|
||||
},
|
||||
RequestSegments: []*datapb.ImportRequestSegment{
|
||||
{
|
||||
SegmentID: 13,
|
||||
PartitionID: 14,
|
||||
Vchannel: "v0",
|
||||
},
|
||||
},
|
||||
}
|
||||
importTask := NewImportTask(importReq)
|
||||
s.manager.Add(importTask)
|
||||
err := s.executor.importFile(s.reader, importTask)
|
||||
s.NoError(err)
|
||||
}
|
||||
|
||||
func TestExecutor(t *testing.T) {
|
||||
suite.Run(t, new(ExecutorSuite))
|
||||
}
|
|
@ -0,0 +1,201 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"github.com/samber/lo"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type HashedData [][]*storage.InsertData // [vchannelIndex][partitionIndex]*storage.InsertData
|
||||
|
||||
func newHashedData(schema *schemapb.CollectionSchema, channelNum, partitionNum int) (HashedData, error) {
|
||||
var err error
|
||||
res := make(HashedData, channelNum)
|
||||
for i := 0; i < channelNum; i++ {
|
||||
res[i] = make([]*storage.InsertData, partitionNum)
|
||||
for j := 0; j < partitionNum; j++ {
|
||||
res[i][j], err = storage.NewInsertData(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
return res, nil
|
||||
}
|
||||
|
||||
func HashData(task Task, rows *storage.InsertData) (HashedData, error) {
|
||||
var (
|
||||
schema = typeutil.AppendSystemFields(task.GetSchema())
|
||||
channelNum = len(task.GetVchannels())
|
||||
partitionNum = len(task.GetPartitionIDs())
|
||||
)
|
||||
|
||||
pkField, err := typeutil.GetPrimaryFieldSchema(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
partKeyField, _ := typeutil.GetPartitionKeyFieldSchema(schema)
|
||||
|
||||
f1 := hashByVChannel(int64(channelNum), pkField)
|
||||
f2 := hashByPartition(int64(partitionNum), partKeyField)
|
||||
|
||||
res, err := newHashedData(schema, channelNum, partitionNum)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for i := 0; i < rows.GetRowNum(); i++ {
|
||||
row := rows.GetRow(i)
|
||||
p1, p2 := f1(row), f2(row)
|
||||
err = res[p1][p2].Append(row)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return res, nil
|
||||
}
|
||||
|
||||
func GetRowsStats(task Task, rows *storage.InsertData) (map[string]*datapb.PartitionRows, error) {
|
||||
var (
|
||||
schema = task.GetSchema()
|
||||
channelNum = len(task.GetVchannels())
|
||||
partitionNum = len(task.GetPartitionIDs())
|
||||
)
|
||||
|
||||
pkField, err := typeutil.GetPrimaryFieldSchema(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
partKeyField, _ := typeutil.GetPartitionKeyFieldSchema(schema)
|
||||
|
||||
hashRowsCount := make([][]int, channelNum)
|
||||
for i := 0; i < channelNum; i++ {
|
||||
hashRowsCount[i] = make([]int, partitionNum)
|
||||
}
|
||||
|
||||
rowNum := GetInsertDataRowCount(rows, schema)
|
||||
if pkField.GetAutoID() {
|
||||
id := int64(0)
|
||||
num := int64(channelNum)
|
||||
fn1 := hashByID()
|
||||
fn2 := hashByPartition(int64(partitionNum), partKeyField)
|
||||
rows.Data = lo.PickBy(rows.Data, func(fieldID int64, _ storage.FieldData) bool {
|
||||
return fieldID != pkField.GetFieldID()
|
||||
})
|
||||
for i := 0; i < rowNum; i++ {
|
||||
p1, p2 := fn1(id, num), fn2(rows.GetRow(i))
|
||||
hashRowsCount[p1][p2]++
|
||||
id++
|
||||
}
|
||||
} else {
|
||||
f1 := hashByVChannel(int64(channelNum), pkField)
|
||||
f2 := hashByPartition(int64(partitionNum), partKeyField)
|
||||
for i := 0; i < rowNum; i++ {
|
||||
row := rows.GetRow(i)
|
||||
p1, p2 := f1(row), f2(row)
|
||||
hashRowsCount[p1][p2]++
|
||||
}
|
||||
}
|
||||
|
||||
res := make(map[string]*datapb.PartitionRows)
|
||||
for _, channel := range task.GetVchannels() {
|
||||
res[channel] = &datapb.PartitionRows{
|
||||
PartitionRows: make(map[int64]int64),
|
||||
}
|
||||
}
|
||||
for i, partitionRows := range hashRowsCount {
|
||||
channel := task.GetVchannels()[i]
|
||||
for j, n := range partitionRows {
|
||||
partition := task.GetPartitionIDs()[j]
|
||||
res[channel].PartitionRows[partition] = int64(n)
|
||||
}
|
||||
}
|
||||
return res, nil
|
||||
}
|
||||
|
||||
func hashByVChannel(channelNum int64, pkField *schemapb.FieldSchema) func(row map[int64]interface{}) int64 {
|
||||
if channelNum == 1 || pkField == nil {
|
||||
return func(_ map[int64]interface{}) int64 {
|
||||
return 0
|
||||
}
|
||||
}
|
||||
switch pkField.GetDataType() {
|
||||
case schemapb.DataType_Int64:
|
||||
return func(row map[int64]interface{}) int64 {
|
||||
pk := row[pkField.GetFieldID()]
|
||||
hash, _ := typeutil.Hash32Int64(pk.(int64))
|
||||
return int64(hash) % channelNum
|
||||
}
|
||||
case schemapb.DataType_VarChar:
|
||||
return func(row map[int64]interface{}) int64 {
|
||||
pk := row[pkField.GetFieldID()]
|
||||
hash := typeutil.HashString2Uint32(pk.(string))
|
||||
return int64(hash) % channelNum
|
||||
}
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func hashByPartition(partitionNum int64, partField *schemapb.FieldSchema) func(row map[int64]interface{}) int64 {
|
||||
if partitionNum == 1 {
|
||||
return func(_ map[int64]interface{}) int64 {
|
||||
return 0
|
||||
}
|
||||
}
|
||||
switch partField.GetDataType() {
|
||||
case schemapb.DataType_Int64:
|
||||
return func(row map[int64]interface{}) int64 {
|
||||
data := row[partField.GetFieldID()]
|
||||
hash, _ := typeutil.Hash32Int64(data.(int64))
|
||||
return int64(hash) % partitionNum
|
||||
}
|
||||
case schemapb.DataType_VarChar:
|
||||
return func(row map[int64]interface{}) int64 {
|
||||
data := row[partField.GetFieldID()]
|
||||
hash := typeutil.HashString2Uint32(data.(string))
|
||||
return int64(hash) % partitionNum
|
||||
}
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func hashByID() func(id int64, shardNum int64) int64 {
|
||||
return func(id int64, shardNum int64) int64 {
|
||||
hash, _ := typeutil.Hash32Int64(id)
|
||||
return int64(hash) % shardNum
|
||||
}
|
||||
}
|
||||
|
||||
func MergeHashedRowsCount(src, dst map[string]*datapb.PartitionRows) {
|
||||
for channel, partitionRows := range src {
|
||||
for partitionID, rowCount := range partitionRows.GetPartitionRows() {
|
||||
if dst[channel] == nil {
|
||||
dst[channel] = &datapb.PartitionRows{
|
||||
PartitionRows: make(map[int64]int64),
|
||||
}
|
||||
}
|
||||
dst[channel].PartitionRows[partitionID] += rowCount
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
)
|
||||
|
||||
type Manager struct {
|
||||
TaskManager
|
||||
Executor
|
||||
}
|
||||
|
||||
func NewManager(syncMgr syncmgr.SyncManager, cm storage.ChunkManager) *Manager {
|
||||
tm := NewTaskManager()
|
||||
e := NewExecutor(tm, syncMgr, cm)
|
||||
return &Manager{
|
||||
TaskManager: tm,
|
||||
Executor: e,
|
||||
}
|
||||
}
|
|
@ -0,0 +1,302 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/samber/lo"
|
||||
|
||||
"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/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type TaskType int
|
||||
|
||||
const (
|
||||
PreImportTaskType TaskType = 0
|
||||
ImportTaskType TaskType = 1
|
||||
)
|
||||
|
||||
var ImportTaskTypeName = map[TaskType]string{
|
||||
0: "PreImportTask",
|
||||
1: "ImportTask",
|
||||
}
|
||||
|
||||
func (t TaskType) String() string {
|
||||
return ImportTaskTypeName[t]
|
||||
}
|
||||
|
||||
type TaskFilter func(task Task) bool
|
||||
|
||||
func WithStates(states ...internalpb.ImportState) TaskFilter {
|
||||
return func(task Task) bool {
|
||||
for _, state := range states {
|
||||
if task.GetState() == state {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
func WithType(taskType TaskType) TaskFilter {
|
||||
return func(task Task) bool {
|
||||
return task.GetType() == taskType
|
||||
}
|
||||
}
|
||||
|
||||
type UpdateAction func(task Task)
|
||||
|
||||
func UpdateState(state internalpb.ImportState) UpdateAction {
|
||||
return func(t Task) {
|
||||
switch t.GetType() {
|
||||
case PreImportTaskType:
|
||||
t.(*PreImportTask).PreImportTask.State = state
|
||||
case ImportTaskType:
|
||||
t.(*ImportTask).ImportTaskV2.State = state
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func UpdateReason(reason string) UpdateAction {
|
||||
return func(t Task) {
|
||||
switch t.GetType() {
|
||||
case PreImportTaskType:
|
||||
t.(*PreImportTask).PreImportTask.Reason = reason
|
||||
case ImportTaskType:
|
||||
t.(*ImportTask).ImportTaskV2.Reason = reason
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func UpdateFileStat(idx int, fileStat *datapb.ImportFileStats) UpdateAction {
|
||||
return func(task Task) {
|
||||
if it, ok := task.(*PreImportTask); ok {
|
||||
it.PreImportTask.FileStats[idx].TotalRows = fileStat.GetTotalRows()
|
||||
it.PreImportTask.FileStats[idx].HashedRows = fileStat.GetHashedRows()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func UpdateSegmentInfo(info *datapb.ImportSegmentInfo) UpdateAction {
|
||||
mergeFn := func(current []*datapb.FieldBinlog, new []*datapb.FieldBinlog) []*datapb.FieldBinlog {
|
||||
for _, binlog := range new {
|
||||
fieldBinlogs, ok := lo.Find(current, func(log *datapb.FieldBinlog) bool {
|
||||
return log.GetFieldID() == binlog.GetFieldID()
|
||||
})
|
||||
if !ok || fieldBinlogs == nil {
|
||||
current = append(current, binlog)
|
||||
} else {
|
||||
fieldBinlogs.Binlogs = append(fieldBinlogs.Binlogs, binlog.Binlogs...)
|
||||
}
|
||||
}
|
||||
return current
|
||||
}
|
||||
return func(task Task) {
|
||||
if it, ok := task.(*ImportTask); ok {
|
||||
segment := info.GetSegmentID()
|
||||
if _, ok = it.segmentsInfo[segment]; ok {
|
||||
it.segmentsInfo[segment].ImportedRows = info.GetImportedRows()
|
||||
it.segmentsInfo[segment].Binlogs = mergeFn(it.segmentsInfo[segment].Binlogs, info.GetBinlogs())
|
||||
it.segmentsInfo[segment].Statslogs = mergeFn(it.segmentsInfo[segment].Statslogs, info.GetStatslogs())
|
||||
return
|
||||
}
|
||||
it.segmentsInfo[segment] = info
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type Task interface {
|
||||
GetJobID() int64
|
||||
GetTaskID() int64
|
||||
GetCollectionID() int64
|
||||
GetPartitionIDs() []int64
|
||||
GetVchannels() []string
|
||||
GetType() TaskType
|
||||
GetState() internalpb.ImportState
|
||||
GetReason() string
|
||||
GetSchema() *schemapb.CollectionSchema
|
||||
GetCtx() context.Context
|
||||
GetOptions() []*commonpb.KeyValuePair
|
||||
Cancel()
|
||||
Clone() Task
|
||||
}
|
||||
|
||||
type PreImportTask struct {
|
||||
*datapb.PreImportTask
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
schema *schemapb.CollectionSchema
|
||||
}
|
||||
|
||||
func NewPreImportTask(req *datapb.PreImportRequest) Task {
|
||||
fileStats := lo.Map(req.GetImportFiles(), func(file *internalpb.ImportFile, _ int) *datapb.ImportFileStats {
|
||||
return &datapb.ImportFileStats{
|
||||
ImportFile: file,
|
||||
}
|
||||
})
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
return &PreImportTask{
|
||||
PreImportTask: &datapb.PreImportTask{
|
||||
JobID: req.GetJobID(),
|
||||
TaskID: req.GetTaskID(),
|
||||
CollectionID: req.GetCollectionID(),
|
||||
PartitionIDs: req.GetPartitionIDs(),
|
||||
Vchannels: req.GetVchannels(),
|
||||
State: internalpb.ImportState_Pending,
|
||||
FileStats: fileStats,
|
||||
Options: req.GetOptions(),
|
||||
},
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
schema: req.GetSchema(),
|
||||
}
|
||||
}
|
||||
|
||||
func (p *PreImportTask) GetType() TaskType {
|
||||
return PreImportTaskType
|
||||
}
|
||||
|
||||
func (p *PreImportTask) GetSchema() *schemapb.CollectionSchema {
|
||||
return p.schema
|
||||
}
|
||||
|
||||
func (p *PreImportTask) GetCtx() context.Context {
|
||||
return p.ctx
|
||||
}
|
||||
|
||||
func (p *PreImportTask) Cancel() {
|
||||
p.cancel()
|
||||
}
|
||||
|
||||
func (p *PreImportTask) Clone() Task {
|
||||
ctx, cancel := context.WithCancel(p.GetCtx())
|
||||
return &PreImportTask{
|
||||
PreImportTask: proto.Clone(p.PreImportTask).(*datapb.PreImportTask),
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
schema: p.GetSchema(),
|
||||
}
|
||||
}
|
||||
|
||||
type ImportTask struct {
|
||||
*datapb.ImportTaskV2
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
schema *schemapb.CollectionSchema
|
||||
segmentsInfo map[int64]*datapb.ImportSegmentInfo
|
||||
req *datapb.ImportRequest
|
||||
vchannels []string
|
||||
partitions []int64
|
||||
metaCaches map[string]metacache.MetaCache
|
||||
}
|
||||
|
||||
func NewImportTask(req *datapb.ImportRequest) Task {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
task := &ImportTask{
|
||||
ImportTaskV2: &datapb.ImportTaskV2{
|
||||
JobID: req.GetJobID(),
|
||||
TaskID: req.GetTaskID(),
|
||||
CollectionID: req.GetCollectionID(),
|
||||
State: internalpb.ImportState_Pending,
|
||||
Options: req.GetOptions(),
|
||||
},
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
schema: req.GetSchema(),
|
||||
segmentsInfo: make(map[int64]*datapb.ImportSegmentInfo),
|
||||
req: req,
|
||||
}
|
||||
task.Init(req)
|
||||
return task
|
||||
}
|
||||
|
||||
func (t *ImportTask) Init(req *datapb.ImportRequest) {
|
||||
metaCaches := make(map[string]metacache.MetaCache)
|
||||
channels := make(map[string]struct{})
|
||||
partitions := make(map[int64]struct{})
|
||||
for _, info := range req.GetRequestSegments() {
|
||||
channels[info.GetVchannel()] = struct{}{}
|
||||
partitions[info.GetPartitionID()] = struct{}{}
|
||||
}
|
||||
schema := typeutil.AppendSystemFields(req.GetSchema())
|
||||
for _, channel := range lo.Keys(channels) {
|
||||
info := &datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{
|
||||
CollectionID: req.GetCollectionID(),
|
||||
ChannelName: channel,
|
||||
},
|
||||
Schema: schema,
|
||||
}
|
||||
metaCache := metacache.NewMetaCache(info, func(segment *datapb.SegmentInfo) *metacache.BloomFilterSet {
|
||||
return metacache.NewBloomFilterSet()
|
||||
})
|
||||
metaCaches[channel] = metaCache
|
||||
}
|
||||
t.vchannels = lo.Keys(channels)
|
||||
t.partitions = lo.Keys(partitions)
|
||||
t.metaCaches = metaCaches
|
||||
}
|
||||
|
||||
func (t *ImportTask) GetType() TaskType {
|
||||
return ImportTaskType
|
||||
}
|
||||
|
||||
func (t *ImportTask) GetPartitionIDs() []int64 {
|
||||
return t.partitions
|
||||
}
|
||||
|
||||
func (t *ImportTask) GetVchannels() []string {
|
||||
return t.vchannels
|
||||
}
|
||||
|
||||
func (t *ImportTask) GetSchema() *schemapb.CollectionSchema {
|
||||
return t.schema
|
||||
}
|
||||
|
||||
func (t *ImportTask) GetCtx() context.Context {
|
||||
return t.ctx
|
||||
}
|
||||
|
||||
func (t *ImportTask) Cancel() {
|
||||
t.cancel()
|
||||
}
|
||||
|
||||
func (t *ImportTask) GetSegmentsInfo() []*datapb.ImportSegmentInfo {
|
||||
return lo.Values(t.segmentsInfo)
|
||||
}
|
||||
|
||||
func (t *ImportTask) Clone() Task {
|
||||
ctx, cancel := context.WithCancel(t.GetCtx())
|
||||
return &ImportTask{
|
||||
ImportTaskV2: proto.Clone(t.ImportTaskV2).(*datapb.ImportTaskV2),
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
schema: t.GetSchema(),
|
||||
segmentsInfo: t.segmentsInfo,
|
||||
req: t.req,
|
||||
vchannels: t.GetVchannels(),
|
||||
partitions: t.GetPartitionIDs(),
|
||||
metaCaches: t.metaCaches,
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"sync"
|
||||
)
|
||||
|
||||
type TaskManager interface {
|
||||
Add(task Task)
|
||||
Update(taskID int64, actions ...UpdateAction)
|
||||
Get(taskID int64) Task
|
||||
GetBy(filters ...TaskFilter) []Task
|
||||
Remove(taskID int64)
|
||||
}
|
||||
|
||||
type taskManager struct {
|
||||
mu sync.RWMutex // guards tasks
|
||||
tasks map[int64]Task
|
||||
}
|
||||
|
||||
func NewTaskManager() TaskManager {
|
||||
return &taskManager{
|
||||
tasks: make(map[int64]Task),
|
||||
}
|
||||
}
|
||||
|
||||
func (m *taskManager) Add(task Task) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
m.tasks[task.GetTaskID()] = task
|
||||
}
|
||||
|
||||
func (m *taskManager) Update(taskID int64, actions ...UpdateAction) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
if _, ok := m.tasks[taskID]; ok {
|
||||
updatedTask := m.tasks[taskID].Clone()
|
||||
for _, action := range actions {
|
||||
action(updatedTask)
|
||||
}
|
||||
m.tasks[taskID] = updatedTask
|
||||
}
|
||||
}
|
||||
|
||||
func (m *taskManager) Get(taskID int64) Task {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
return m.tasks[taskID]
|
||||
}
|
||||
|
||||
func (m *taskManager) GetBy(filters ...TaskFilter) []Task {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
ret := make([]Task, 0)
|
||||
OUTER:
|
||||
for _, task := range m.tasks {
|
||||
for _, f := range filters {
|
||||
if !f(task) {
|
||||
continue OUTER
|
||||
}
|
||||
}
|
||||
ret = append(ret, task)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func (m *taskManager) Remove(taskID int64) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
if task, ok := m.tasks[taskID]; ok {
|
||||
task.Cancel()
|
||||
}
|
||||
delete(m.tasks, taskID)
|
||||
}
|
|
@ -0,0 +1,79 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
func TestImportManager(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
manager := NewTaskManager()
|
||||
task1 := &ImportTask{
|
||||
ImportTaskV2: &datapb.ImportTaskV2{
|
||||
JobID: 1,
|
||||
TaskID: 2,
|
||||
CollectionID: 3,
|
||||
SegmentIDs: []int64{5, 6},
|
||||
NodeID: 7,
|
||||
State: internalpb.ImportState_Pending,
|
||||
},
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
}
|
||||
manager.Add(task1)
|
||||
manager.Add(task1)
|
||||
res := manager.Get(task1.GetTaskID())
|
||||
assert.Equal(t, task1, res)
|
||||
|
||||
task2 := &ImportTask{
|
||||
ImportTaskV2: &datapb.ImportTaskV2{
|
||||
JobID: 1,
|
||||
TaskID: 8,
|
||||
CollectionID: 3,
|
||||
SegmentIDs: []int64{5, 6},
|
||||
NodeID: 7,
|
||||
State: internalpb.ImportState_Completed,
|
||||
},
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
}
|
||||
manager.Add(task2)
|
||||
|
||||
tasks := manager.GetBy()
|
||||
assert.Equal(t, 2, len(tasks))
|
||||
tasks = manager.GetBy(WithStates(internalpb.ImportState_Completed))
|
||||
assert.Equal(t, 1, len(tasks))
|
||||
assert.Equal(t, task2.GetTaskID(), tasks[0].GetTaskID())
|
||||
|
||||
manager.Update(task1.GetTaskID(), UpdateState(internalpb.ImportState_Failed))
|
||||
task := manager.Get(task1.GetTaskID())
|
||||
assert.Equal(t, internalpb.ImportState_Failed, task.GetState())
|
||||
|
||||
manager.Remove(task1.GetTaskID())
|
||||
tasks = manager.GetBy()
|
||||
assert.Equal(t, 1, len(tasks))
|
||||
manager.Remove(10)
|
||||
tasks = manager.GetBy()
|
||||
assert.Equal(t, 1, len(tasks))
|
||||
}
|
|
@ -0,0 +1,213 @@
|
|||
// 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 importv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/samber/lo"
|
||||
"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/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"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/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
func WrapNoTaskError(taskID int64, taskType TaskType) error {
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("cannot find %s with id %d", taskType.String(), taskID))
|
||||
}
|
||||
|
||||
func NewSyncTask(ctx context.Context, task *ImportTask, segmentID, partitionID int64, vchannel string, insertData *storage.InsertData) (syncmgr.Task, error) {
|
||||
if params.Params.CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
return nil, merr.WrapErrImportFailed("storage v2 is not supported") // TODO: dyh, resolve storage v2
|
||||
}
|
||||
|
||||
metaCache := task.metaCaches[vchannel]
|
||||
if _, ok := metaCache.GetSegmentByID(segmentID); !ok {
|
||||
metaCache.AddSegment(&datapb.SegmentInfo{
|
||||
ID: segmentID,
|
||||
State: commonpb.SegmentState_Importing,
|
||||
CollectionID: task.GetCollectionID(),
|
||||
PartitionID: partitionID,
|
||||
InsertChannel: vchannel,
|
||||
}, func(info *datapb.SegmentInfo) *metacache.BloomFilterSet {
|
||||
bfs := metacache.NewBloomFilterSet()
|
||||
return bfs
|
||||
}, metacache.UpdateImporting(true))
|
||||
}
|
||||
|
||||
var serializer syncmgr.Serializer
|
||||
var err error
|
||||
serializer, err = syncmgr.NewStorageSerializer(
|
||||
metaCache,
|
||||
nil,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
syncPack := &syncmgr.SyncPack{}
|
||||
syncPack.WithInsertData(insertData).
|
||||
WithCollectionID(task.GetCollectionID()).
|
||||
WithPartitionID(partitionID).
|
||||
WithChannelName(vchannel).
|
||||
WithSegmentID(segmentID).
|
||||
WithTimeRange(task.req.GetTs(), task.req.GetTs()).
|
||||
WithBatchSize(int64(insertData.GetRowNum()))
|
||||
|
||||
return serializer.EncodeBuffer(ctx, syncPack)
|
||||
}
|
||||
|
||||
func NewImportSegmentInfo(syncTask syncmgr.Task, task *ImportTask) (*datapb.ImportSegmentInfo, error) {
|
||||
segmentID := syncTask.SegmentID()
|
||||
insertBinlogs, statsBinlog, _ := syncTask.(*syncmgr.SyncTask).Binlogs()
|
||||
metaCache := task.metaCaches[syncTask.ChannelName()]
|
||||
segment, ok := metaCache.GetSegmentByID(segmentID)
|
||||
if !ok {
|
||||
return nil, merr.WrapErrSegmentNotFound(segmentID, "import failed")
|
||||
}
|
||||
return &datapb.ImportSegmentInfo{
|
||||
SegmentID: segmentID,
|
||||
ImportedRows: segment.FlushedRows(),
|
||||
Binlogs: lo.Values(insertBinlogs),
|
||||
Statslogs: lo.Values(statsBinlog),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func PickSegment(task *ImportTask, vchannel string, partitionID int64, rows int) int64 {
|
||||
candidates := lo.Filter(task.req.GetRequestSegments(), func(info *datapb.ImportRequestSegment, _ int) bool {
|
||||
return info.GetVchannel() == vchannel && info.GetPartitionID() == partitionID
|
||||
})
|
||||
|
||||
importedSegments := lo.KeyBy(task.GetSegmentsInfo(), func(segment *datapb.ImportSegmentInfo) int64 {
|
||||
return segment.GetSegmentID()
|
||||
})
|
||||
|
||||
for _, candidate := range candidates {
|
||||
var importedRows int64 = 0
|
||||
if segment, ok := importedSegments[candidate.GetSegmentID()]; ok {
|
||||
importedRows = segment.GetImportedRows()
|
||||
}
|
||||
if importedRows+int64(rows) <= candidate.GetMaxRows() {
|
||||
return candidate.GetSegmentID()
|
||||
}
|
||||
}
|
||||
segmentID := lo.MinBy(task.GetSegmentsInfo(), func(s1, s2 *datapb.ImportSegmentInfo) bool {
|
||||
return s1.GetImportedRows() < s2.GetImportedRows()
|
||||
}).GetSegmentID()
|
||||
log.Warn("failed to pick an appropriate segment, opt for the smallest one instead",
|
||||
WrapLogFields(task, zap.Int64("segmentID", segmentID), zap.Int64("maxRows", candidates[0].GetMaxRows()),
|
||||
zap.Int("rows", rows), zap.Int64("importedRows", importedSegments[segmentID].GetImportedRows()))...)
|
||||
return segmentID
|
||||
}
|
||||
|
||||
func CheckRowsEqual(schema *schemapb.CollectionSchema, data *storage.InsertData) error {
|
||||
if len(data.Data) == 0 {
|
||||
return nil
|
||||
}
|
||||
idToField := lo.KeyBy(schema.GetFields(), func(field *schemapb.FieldSchema) int64 {
|
||||
return field.GetFieldID()
|
||||
})
|
||||
|
||||
var field int64
|
||||
var rows int
|
||||
for fieldID, d := range data.Data {
|
||||
if idToField[fieldID].GetIsPrimaryKey() && idToField[fieldID].GetAutoID() {
|
||||
continue
|
||||
}
|
||||
field, rows = fieldID, d.RowNum()
|
||||
break
|
||||
}
|
||||
for fieldID, d := range data.Data {
|
||||
if idToField[fieldID].GetIsPrimaryKey() && idToField[fieldID].GetAutoID() {
|
||||
continue
|
||||
}
|
||||
if d.RowNum() != rows {
|
||||
return merr.WrapErrImportFailed(
|
||||
fmt.Sprintf("imported rows are not aligned, field '%s' with '%d' rows, field '%s' with '%d' rows",
|
||||
idToField[field].GetName(), rows, idToField[fieldID].GetName(), d.RowNum()))
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func AppendSystemFieldsData(task *ImportTask, data *storage.InsertData) error {
|
||||
idRange := task.req.GetAutoIDRange()
|
||||
pkField, err := typeutil.GetPrimaryFieldSchema(task.GetSchema())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
rowNum := GetInsertDataRowCount(data, task.GetSchema())
|
||||
ids := make([]int64, rowNum)
|
||||
for i := 0; i < rowNum; i++ {
|
||||
ids[i] = idRange.GetBegin() + int64(i)
|
||||
}
|
||||
idRange.Begin += int64(rowNum)
|
||||
if pkField.GetAutoID() {
|
||||
data.Data[pkField.GetFieldID()] = &storage.Int64FieldData{Data: ids}
|
||||
}
|
||||
data.Data[common.RowIDField] = &storage.Int64FieldData{Data: ids}
|
||||
tss := make([]int64, rowNum)
|
||||
ts := int64(task.req.GetTs())
|
||||
for i := 0; i < rowNum; i++ {
|
||||
tss[i] = ts
|
||||
}
|
||||
data.Data[common.TimeStampField] = &storage.Int64FieldData{Data: tss}
|
||||
return nil
|
||||
}
|
||||
|
||||
func GetInsertDataRowCount(data *storage.InsertData, schema *schemapb.CollectionSchema) int {
|
||||
fields := lo.KeyBy(schema.GetFields(), func(field *schemapb.FieldSchema) int64 {
|
||||
return field.GetFieldID()
|
||||
})
|
||||
for fieldID, fd := range data.Data {
|
||||
if fields[fieldID].GetIsDynamic() {
|
||||
continue
|
||||
}
|
||||
if fd.RowNum() != 0 {
|
||||
return fd.RowNum()
|
||||
}
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func LogStats(manager TaskManager) {
|
||||
logFunc := func(tasks []Task, taskType TaskType) {
|
||||
byState := lo.GroupBy(tasks, func(t Task) internalpb.ImportState {
|
||||
return t.GetState()
|
||||
})
|
||||
log.Info("import task stats", zap.String("type", taskType.String()),
|
||||
zap.Int("pending", len(byState[internalpb.ImportState_Pending])),
|
||||
zap.Int("inProgress", len(byState[internalpb.ImportState_InProgress])),
|
||||
zap.Int("completed", len(byState[internalpb.ImportState_Completed])),
|
||||
zap.Int("failed", len(byState[internalpb.ImportState_Failed])))
|
||||
}
|
||||
tasks := manager.GetBy(WithType(PreImportTaskType))
|
||||
logFunc(tasks, PreImportTaskType)
|
||||
tasks = manager.GetBy(WithType(ImportTaskType))
|
||||
logFunc(tasks, ImportTaskType)
|
||||
}
|
|
@ -33,6 +33,7 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
|
@ -951,21 +952,113 @@ func logDupFlush(cID, segID int64) {
|
|||
}
|
||||
|
||||
func (node *DataNode) PreImport(ctx context.Context, req *datapb.PreImportRequest) (*commonpb.Status, error) {
|
||||
return nil, merr.ErrServiceUnimplemented
|
||||
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
||||
zap.Int64("jobID", req.GetJobID()),
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int64s("partitionIDs", req.GetPartitionIDs()),
|
||||
zap.Strings("vchannels", req.GetVchannels()),
|
||||
zap.Any("files", req.GetImportFiles()))
|
||||
|
||||
log.Info("datanode receive preimport request")
|
||||
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
task := importv2.NewPreImportTask(req)
|
||||
node.importManager.Add(task)
|
||||
|
||||
log.Info("datanode added preimport task")
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (node *DataNode) ImportV2(ctx context.Context, req *datapb.ImportRequest) (*commonpb.Status, error) {
|
||||
return nil, merr.ErrServiceUnimplemented
|
||||
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
||||
zap.Int64("jobID", req.GetJobID()),
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Any("segments", req.GetRequestSegments()),
|
||||
zap.Any("files", req.GetFiles()))
|
||||
|
||||
log.Info("datanode receive import request")
|
||||
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
task := importv2.NewImportTask(req)
|
||||
node.importManager.Add(task)
|
||||
|
||||
log.Info("datanode added import task")
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (node *DataNode) QueryPreImport(ctx context.Context, req *datapb.QueryPreImportRequest) (*datapb.QueryPreImportResponse, error) {
|
||||
return nil, merr.ErrServiceUnimplemented
|
||||
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
||||
zap.Int64("jobID", req.GetJobID()))
|
||||
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
return &datapb.QueryPreImportResponse{Status: merr.Status(err)}, nil
|
||||
}
|
||||
status := merr.Success()
|
||||
task := node.importManager.Get(req.GetTaskID())
|
||||
if task == nil || task.GetType() != importv2.PreImportTaskType {
|
||||
status = merr.Status(importv2.WrapNoTaskError(req.GetTaskID(), importv2.PreImportTaskType))
|
||||
}
|
||||
log.RatedInfo(10, "datanode query preimport done", zap.String("state", task.GetState().String()),
|
||||
zap.String("reason", task.GetReason()))
|
||||
return &datapb.QueryPreImportResponse{
|
||||
Status: status,
|
||||
TaskID: task.GetTaskID(),
|
||||
State: task.GetState(),
|
||||
Reason: task.GetReason(),
|
||||
FileStats: task.(*importv2.PreImportTask).GetFileStats(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (node *DataNode) QueryImport(ctx context.Context, req *datapb.QueryImportRequest) (*datapb.QueryImportResponse, error) {
|
||||
return nil, merr.ErrServiceUnimplemented
|
||||
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
||||
zap.Int64("jobID", req.GetJobID()))
|
||||
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
return &datapb.QueryImportResponse{Status: merr.Status(err)}, nil
|
||||
}
|
||||
|
||||
status := merr.Success()
|
||||
|
||||
// query slot
|
||||
if req.GetQuerySlot() {
|
||||
return &datapb.QueryImportResponse{
|
||||
Status: status,
|
||||
Slots: node.importManager.Slots(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// query import
|
||||
task := node.importManager.Get(req.GetTaskID())
|
||||
if task == nil || task.GetType() != importv2.ImportTaskType {
|
||||
status = merr.Status(importv2.WrapNoTaskError(req.GetTaskID(), importv2.ImportTaskType))
|
||||
}
|
||||
log.RatedInfo(10, "datanode query import done", zap.String("state", task.GetState().String()),
|
||||
zap.String("reason", task.GetReason()))
|
||||
return &datapb.QueryImportResponse{
|
||||
Status: status,
|
||||
TaskID: task.GetTaskID(),
|
||||
State: task.GetState(),
|
||||
Reason: task.GetReason(),
|
||||
ImportSegmentsInfo: task.(*importv2.ImportTask).GetSegmentsInfo(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (node *DataNode) DropImport(ctx context.Context, req *datapb.DropImportRequest) (*commonpb.Status, error) {
|
||||
return nil, merr.ErrServiceUnimplemented
|
||||
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
||||
zap.Int64("jobID", req.GetJobID()))
|
||||
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
node.importManager.Remove(req.GetTaskID())
|
||||
|
||||
log.Info("datanode drop import done")
|
||||
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
|
|
@ -192,7 +192,7 @@ func (t *SyncTask) Run() (err error) {
|
|||
|
||||
t.metacache.UpdateSegments(metacache.MergeSegmentAction(actions...), metacache.WithSegmentIDs(t.segment.SegmentID()))
|
||||
|
||||
log.Info("task done")
|
||||
log.Info("task done", zap.Float64("flushedSize", totalSize))
|
||||
|
||||
if !t.isFlush {
|
||||
metrics.DataNodeAutoFlushBufferCount.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.SuccessLabel, t.level.String()).Inc()
|
||||
|
@ -339,3 +339,7 @@ func (t *SyncTask) StartPosition() *msgpb.MsgPosition {
|
|||
func (t *SyncTask) ChannelName() string {
|
||||
return t.channelName
|
||||
}
|
||||
|
||||
func (t *SyncTask) Binlogs() (map[int64]*datapb.FieldBinlog, map[int64]*datapb.FieldBinlog, *datapb.FieldBinlog) {
|
||||
return t.insertBinlogs, t.statsBinlogs, t.deltaBinlog
|
||||
}
|
||||
|
|
|
@ -752,33 +752,16 @@ message ChannelOperationProgressResponse {
|
|||
int32 progress = 4;
|
||||
}
|
||||
|
||||
enum ImportState {
|
||||
None = 0;
|
||||
Pending = 1;
|
||||
InProgress = 2;
|
||||
Failed = 3;
|
||||
Completed = 4;
|
||||
}
|
||||
|
||||
message ColumnBasedFile {
|
||||
repeated string files = 1;
|
||||
}
|
||||
|
||||
message ImportFile {
|
||||
oneof file {
|
||||
string row_based_file = 1;
|
||||
ColumnBasedFile column_based_file = 2;
|
||||
}
|
||||
}
|
||||
|
||||
message PreImportRequest {
|
||||
string clusterID = 1;
|
||||
int64 requestID = 2;
|
||||
int64 jobID = 2;
|
||||
int64 taskID = 3;
|
||||
int64 collectionID = 4;
|
||||
int64 partitionID = 5;
|
||||
schema.CollectionSchema schema = 6;
|
||||
repeated ImportFile import_files = 7;
|
||||
repeated int64 partitionIDs = 5;
|
||||
repeated string vchannels = 6;
|
||||
schema.CollectionSchema schema = 7;
|
||||
repeated internal.ImportFile import_files = 8;
|
||||
repeated common.KeyValuePair options = 9;
|
||||
}
|
||||
|
||||
message autoIDRange {
|
||||
|
@ -786,35 +769,47 @@ message autoIDRange {
|
|||
int64 end = 2;
|
||||
}
|
||||
|
||||
message ImportRequestSegment {
|
||||
int64 segmentID = 1;
|
||||
int64 partitionID = 2;
|
||||
string vchannel = 3;
|
||||
int64 max_rows = 4;
|
||||
}
|
||||
|
||||
message ImportRequest {
|
||||
string clusterID = 1;
|
||||
int64 requestID = 2;
|
||||
int64 jobID = 2;
|
||||
int64 taskID = 3;
|
||||
int64 collectionID = 4;
|
||||
int64 partitionID = 5;
|
||||
schema.CollectionSchema schema = 6;
|
||||
repeated ImportFile import_files = 7;
|
||||
map<int64, autoIDRange> autoID_ranges = 8;
|
||||
map<int64, string> segment_channels = 9;
|
||||
schema.CollectionSchema schema = 5;
|
||||
repeated internal.ImportFile files = 6;
|
||||
repeated common.KeyValuePair options = 7;
|
||||
uint64 ts = 8;
|
||||
autoIDRange autoID_range = 9;
|
||||
repeated ImportRequestSegment request_segments = 10;
|
||||
}
|
||||
|
||||
message QueryPreImportRequest {
|
||||
string clusterID = 1;
|
||||
int64 requestID = 2;
|
||||
int64 jobID = 2;
|
||||
int64 taskID = 3;
|
||||
}
|
||||
|
||||
message PartitionRows {
|
||||
map<int64, int64> partition_rows = 1;
|
||||
}
|
||||
|
||||
message ImportFileStats {
|
||||
ImportFile import_file = 1;
|
||||
internal.ImportFile import_file = 1;
|
||||
int64 file_size = 2;
|
||||
int64 total_rows = 3;
|
||||
map<string, int64> channel_rows = 4;
|
||||
map<string, PartitionRows> hashed_rows = 4;
|
||||
}
|
||||
|
||||
message QueryPreImportResponse {
|
||||
common.Status status = 1;
|
||||
int64 taskID = 2;
|
||||
ImportState state = 3;
|
||||
internal.ImportState state = 3;
|
||||
string reason = 4;
|
||||
int64 slots = 5;
|
||||
repeated ImportFileStats file_stats = 6;
|
||||
|
@ -822,23 +817,22 @@ message QueryPreImportResponse {
|
|||
|
||||
message QueryImportRequest {
|
||||
string clusterID = 1;
|
||||
int64 requestID = 2;
|
||||
int64 jobID = 2;
|
||||
int64 taskID = 3;
|
||||
bool querySlot = 4;
|
||||
}
|
||||
|
||||
message ImportSegmentInfo {
|
||||
int64 segmentID = 1;
|
||||
int64 total_rows = 2;
|
||||
int64 imported_rows = 3;
|
||||
repeated FieldBinlog binlogs = 4;
|
||||
repeated FieldBinlog statslogs = 5;
|
||||
repeated index.IndexFilePathInfo index_infos = 6;
|
||||
int64 imported_rows = 2;
|
||||
repeated FieldBinlog binlogs = 3;
|
||||
repeated FieldBinlog statslogs = 4;
|
||||
}
|
||||
|
||||
message QueryImportResponse {
|
||||
common.Status status = 1;
|
||||
int64 taskID = 2;
|
||||
ImportState state = 3;
|
||||
internal.ImportState state = 3;
|
||||
string reason = 4;
|
||||
int64 slots = 5;
|
||||
repeated ImportSegmentInfo import_segments_info = 6;
|
||||
|
@ -846,31 +840,35 @@ message QueryImportResponse {
|
|||
|
||||
message DropImportRequest {
|
||||
string clusterID = 1;
|
||||
int64 requestID = 2;
|
||||
int64 jobID = 2;
|
||||
int64 taskID = 3;
|
||||
}
|
||||
|
||||
message PreImportTask {
|
||||
int64 requestID = 1;
|
||||
int64 jobID = 1;
|
||||
int64 taskID = 2;
|
||||
int64 collectionID = 3;
|
||||
int64 partitionID = 4;
|
||||
int64 nodeID = 5;
|
||||
ImportState state = 6;
|
||||
string reason = 7;
|
||||
repeated ImportFileStats file_stats = 8;
|
||||
repeated int64 partitionIDs = 4;
|
||||
repeated string vchannels = 5;
|
||||
int64 nodeID = 6;
|
||||
internal.ImportState state = 7;
|
||||
string reason = 8;
|
||||
uint64 timeout_ts = 9;
|
||||
repeated ImportFileStats file_stats = 10;
|
||||
repeated common.KeyValuePair options = 11;
|
||||
}
|
||||
|
||||
message ImportTaskV2 {
|
||||
int64 requestID = 1;
|
||||
int64 jobID = 1;
|
||||
int64 taskID = 2;
|
||||
int64 collectionID = 3;
|
||||
int64 partitionID = 4;
|
||||
repeated int64 segmentIDs = 5;
|
||||
int64 nodeID = 6;
|
||||
ImportState state = 7;
|
||||
string reason = 8;
|
||||
repeated ImportFile files = 9;
|
||||
repeated int64 segmentIDs = 4;
|
||||
int64 nodeID = 5;
|
||||
internal.ImportState state = 6;
|
||||
string reason = 7;
|
||||
uint64 timeout_ts = 8;
|
||||
repeated ImportFileStats file_stats = 9;
|
||||
repeated common.KeyValuePair options = 10;
|
||||
}
|
||||
|
||||
enum GcCommand {
|
||||
|
|
|
@ -249,3 +249,63 @@ message Rate {
|
|||
RateType rt = 1;
|
||||
double r = 2;
|
||||
}
|
||||
|
||||
enum ImportState {
|
||||
None = 0;
|
||||
Pending = 1;
|
||||
InProgress = 2;
|
||||
Failed = 3;
|
||||
Completed = 4;
|
||||
}
|
||||
|
||||
message ImportFile {
|
||||
// A singular row-based file or multiple column-based files.
|
||||
repeated string paths = 1;
|
||||
}
|
||||
|
||||
message ImportRequestInternal {
|
||||
int64 collectionID = 1;
|
||||
repeated int64 partitionIDs = 2;
|
||||
repeated string channel_names = 3;
|
||||
schema.CollectionSchema schema = 4;
|
||||
repeated internal.ImportFile files = 5;
|
||||
repeated common.KeyValuePair options = 6;
|
||||
}
|
||||
|
||||
message ImportRequest {
|
||||
string db_name = 1;
|
||||
string collection_name = 2;
|
||||
string partition_name = 3;
|
||||
repeated ImportFile files = 4;
|
||||
repeated common.KeyValuePair options = 5;
|
||||
}
|
||||
|
||||
message ImportResponse {
|
||||
common.Status status = 1;
|
||||
string jobID = 2;
|
||||
}
|
||||
|
||||
message GetImportProgressRequest {
|
||||
string db_name = 1;
|
||||
string jobID = 2;
|
||||
}
|
||||
|
||||
message GetImportProgressResponse {
|
||||
common.Status status = 1;
|
||||
ImportState state = 2;
|
||||
string reason = 3;
|
||||
int64 progress = 4;
|
||||
}
|
||||
|
||||
message ListImportsRequest {
|
||||
string db_name = 1;
|
||||
string collection_name = 2;
|
||||
}
|
||||
|
||||
message ListImportsResponse {
|
||||
common.Status status = 1;
|
||||
repeated string jobIDs = 2;
|
||||
repeated ImportState states = 3;
|
||||
repeated string reasons = 4;
|
||||
repeated int64 progresses = 5;
|
||||
}
|
||||
|
|
|
@ -75,7 +75,7 @@ func (iData *InsertData) IsEmpty() bool {
|
|||
}
|
||||
|
||||
func (i *InsertData) GetRowNum() int {
|
||||
if i.Data == nil || len(i.Data) == 0 {
|
||||
if i == nil || i.Data == nil || len(i.Data) == 0 {
|
||||
return 0
|
||||
}
|
||||
var rowNum int
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
package binlog
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
)
|
||||
|
@ -26,8 +28,8 @@ type fieldReader struct {
|
|||
fieldSchema *schemapb.FieldSchema
|
||||
}
|
||||
|
||||
func newFieldReader(cm storage.ChunkManager, fieldSchema *schemapb.FieldSchema, path string) (*fieldReader, error) {
|
||||
reader, err := newBinlogReader(cm, path)
|
||||
func newFieldReader(ctx context.Context, cm storage.ChunkManager, fieldSchema *schemapb.FieldSchema, path string) (*fieldReader, error) {
|
||||
reader, err := newBinlogReader(ctx, cm, path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -37,7 +39,7 @@ func newFieldReader(cm storage.ChunkManager, fieldSchema *schemapb.FieldSchema,
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (r *fieldReader) Next(_ int64) (storage.FieldData, error) {
|
||||
func (r *fieldReader) Next() (storage.FieldData, error) {
|
||||
fieldData, err := storage.NewFieldData(r.fieldSchema.GetDataType(), r.fieldSchema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
|
|
@ -19,6 +19,8 @@ package binlog
|
|||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
|
@ -28,6 +30,7 @@ import (
|
|||
)
|
||||
|
||||
type reader struct {
|
||||
ctx context.Context
|
||||
cm storage.ChunkManager
|
||||
schema *schemapb.CollectionSchema
|
||||
|
||||
|
@ -38,13 +41,15 @@ type reader struct {
|
|||
filters []Filter
|
||||
}
|
||||
|
||||
func NewReader(cm storage.ChunkManager,
|
||||
func NewReader(ctx context.Context,
|
||||
cm storage.ChunkManager,
|
||||
schema *schemapb.CollectionSchema,
|
||||
paths []string,
|
||||
tsStart,
|
||||
tsEnd uint64,
|
||||
) (*reader, error) {
|
||||
r := &reader{
|
||||
ctx: ctx,
|
||||
cm: cm,
|
||||
schema: schema,
|
||||
}
|
||||
|
@ -62,7 +67,7 @@ func (r *reader) init(paths []string, tsStart, tsEnd uint64) error {
|
|||
if len(paths) < 1 {
|
||||
return merr.WrapErrImportFailed("no insert binlogs to import")
|
||||
}
|
||||
insertLogs, err := listInsertLogs(r.cm, paths[0])
|
||||
insertLogs, err := listInsertLogs(r.ctx, r.cm, paths[0])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -98,7 +103,7 @@ func (r *reader) init(paths []string, tsStart, tsEnd uint64) error {
|
|||
func (r *reader) readDelete(deltaLogs []string, tsStart, tsEnd uint64) (*storage.DeleteData, error) {
|
||||
deleteData := storage.NewDeleteData(nil, nil)
|
||||
for _, path := range deltaLogs {
|
||||
reader, err := newBinlogReader(r.cm, path)
|
||||
reader, err := newBinlogReader(r.ctx, r.cm, path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -128,7 +133,10 @@ func (r *reader) Read() (*storage.InsertData, error) {
|
|||
return nil, err
|
||||
}
|
||||
if r.readIdx == len(r.insertLogs[0]) {
|
||||
return nil, nil
|
||||
// In the binlog import scenario, all data may be filtered out
|
||||
// due to time range or deletions. Therefore, we use io.EOF as
|
||||
// the indicator of the read end, instead of InsertData with 0 rows.
|
||||
return nil, io.EOF
|
||||
}
|
||||
for fieldID, binlogs := range r.insertLogs {
|
||||
field := typeutil.GetField(r.schema, fieldID)
|
||||
|
@ -136,11 +144,11 @@ func (r *reader) Read() (*storage.InsertData, error) {
|
|||
return nil, merr.WrapErrFieldNotFound(fieldID)
|
||||
}
|
||||
path := binlogs[r.readIdx]
|
||||
fr, err := newFieldReader(r.cm, field, path)
|
||||
fr, err := newFieldReader(r.ctx, r.cm, field, path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
fieldData, err := fr.Next(-1)
|
||||
fieldData, err := fr.Next()
|
||||
if err != nil {
|
||||
fr.Close()
|
||||
return nil, err
|
||||
|
@ -185,7 +193,7 @@ OUTER:
|
|||
row := insertData.GetRow(i)
|
||||
err = result.Append(row)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to append row, err=%s", err.Error()))
|
||||
}
|
||||
}
|
||||
return result, nil
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package binlog
|
||||
|
||||
import (
|
||||
"context"
|
||||
rand2 "crypto/rand"
|
||||
"fmt"
|
||||
"math"
|
||||
|
@ -327,7 +328,7 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
},
|
||||
}
|
||||
cm := mocks.NewChunkManager(suite.T())
|
||||
typeutil.AppendSystemFields(schema)
|
||||
schema = typeutil.AppendSystemFields(schema)
|
||||
|
||||
originalInsertData := createInsertData(suite.T(), schema, suite.numRows)
|
||||
insertLogs := lo.Flatten(lo.Values(insertBinlogs))
|
||||
|
@ -336,6 +337,7 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
cm.EXPECT().ListWithPrefix(mock.Anything, deltaPrefix, mock.Anything).Return(deltaLogs, nil, nil)
|
||||
for fieldID, paths := range insertBinlogs {
|
||||
field := typeutil.GetField(schema, fieldID)
|
||||
suite.NotNil(field)
|
||||
buf0 := createBinlogBuf(suite.T(), field, originalInsertData.Data[fieldID])
|
||||
cm.EXPECT().Read(mock.Anything, paths[0]).Return(buf0, nil)
|
||||
}
|
||||
|
@ -347,7 +349,7 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
}
|
||||
}
|
||||
|
||||
reader, err := NewReader(cm, schema, []string{insertPrefix, deltaPrefix}, suite.tsStart, suite.tsEnd)
|
||||
reader, err := NewReader(context.Background(), cm, schema, []string{insertPrefix, deltaPrefix}, suite.tsStart, suite.tsEnd)
|
||||
suite.NoError(err)
|
||||
insertData, err := reader.Read()
|
||||
suite.NoError(err)
|
||||
|
|
|
@ -52,8 +52,8 @@ func readData(reader *storage.BinlogReader, et storage.EventTypeCode) ([]any, er
|
|||
return rowsSet, nil
|
||||
}
|
||||
|
||||
func newBinlogReader(cm storage.ChunkManager, path string) (*storage.BinlogReader, error) {
|
||||
bytes, err := cm.Read(context.TODO(), path) // TODO: dyh, resolve context, and checks if the error is a retryable error
|
||||
func newBinlogReader(ctx context.Context, cm storage.ChunkManager, path string) (*storage.BinlogReader, error) {
|
||||
bytes, err := cm.Read(ctx, path) // TODO: dyh, checks if the error is a retryable error
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to open binlog %s", path))
|
||||
}
|
||||
|
@ -65,8 +65,8 @@ func newBinlogReader(cm storage.ChunkManager, path string) (*storage.BinlogReade
|
|||
return reader, nil
|
||||
}
|
||||
|
||||
func listInsertLogs(cm storage.ChunkManager, insertPrefix string) (map[int64][]string, error) {
|
||||
insertLogPaths, _, err := cm.ListWithPrefix(context.Background(), insertPrefix, true)
|
||||
func listInsertLogs(ctx context.Context, cm storage.ChunkManager, insertPrefix string) (map[int64][]string, error) {
|
||||
insertLogPaths, _, err := cm.ListWithPrefix(ctx, insertPrefix, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -38,18 +39,24 @@ type reader struct {
|
|||
schema *schemapb.CollectionSchema
|
||||
|
||||
bufferSize int
|
||||
count int64
|
||||
isOldFormat bool
|
||||
|
||||
parser RowParser
|
||||
}
|
||||
|
||||
func NewReader(r io.Reader, schema *schemapb.CollectionSchema, bufferSize int) (*reader, error) {
|
||||
var err error
|
||||
count, err := estimateReadCountPerBatch(bufferSize, schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
reader := &reader{
|
||||
dec: json.NewDecoder(r),
|
||||
schema: schema,
|
||||
bufferSize: bufferSize,
|
||||
count: count,
|
||||
}
|
||||
var err error
|
||||
reader.parser, err = NewRowParser(schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -69,7 +76,7 @@ func (j *reader) Init() error {
|
|||
j.dec.UseNumber()
|
||||
t, err := j.dec.Token()
|
||||
if err != nil {
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode JSON, error: %v", err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("init failed, failed to decode JSON, error: %v", err))
|
||||
}
|
||||
if t != json.Delim('{') && t != json.Delim('[') {
|
||||
return merr.WrapErrImportFailed("invalid JSON format, the content should be started with '{' or '['")
|
||||
|
@ -84,7 +91,7 @@ func (j *reader) Read() (*storage.InsertData, error) {
|
|||
return nil, err
|
||||
}
|
||||
if !j.dec.More() {
|
||||
return nil, nil
|
||||
return nil, io.EOF
|
||||
}
|
||||
if j.isOldFormat {
|
||||
// read the key
|
||||
|
@ -108,7 +115,9 @@ func (j *reader) Read() (*storage.InsertData, error) {
|
|||
if t != json.Delim('[') {
|
||||
return nil, merr.WrapErrImportFailed("invalid JSON format, rows list should begin with '['")
|
||||
}
|
||||
j.isOldFormat = false
|
||||
}
|
||||
var cnt int64 = 0
|
||||
for j.dec.More() {
|
||||
var value any
|
||||
if err = j.dec.Decode(&value); err != nil {
|
||||
|
@ -120,10 +129,14 @@ func (j *reader) Read() (*storage.InsertData, error) {
|
|||
}
|
||||
err = insertData.Append(row)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to append row, err=%s", err.Error()))
|
||||
}
|
||||
if insertData.GetMemorySize() >= j.bufferSize {
|
||||
break
|
||||
cnt++
|
||||
if cnt >= j.count {
|
||||
cnt = 0
|
||||
if insertData.GetMemorySize() >= j.bufferSize {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -141,3 +154,14 @@ func (j *reader) Read() (*storage.InsertData, error) {
|
|||
}
|
||||
|
||||
func (j *reader) Close() {}
|
||||
|
||||
func estimateReadCountPerBatch(bufferSize int, schema *schemapb.CollectionSchema) (int64, error) {
|
||||
sizePerRecord, err := typeutil.EstimateMaxSizePerRecord(schema)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if 1000*sizePerRecord <= bufferSize {
|
||||
return 1000, nil
|
||||
}
|
||||
return int64(bufferSize) / int64(sizePerRecord), nil
|
||||
}
|
||||
|
|
|
@ -170,6 +170,12 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
Name: "pk",
|
||||
IsPrimaryKey: true,
|
||||
DataType: suite.pkDataType,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.MaxLengthKey,
|
||||
Value: "128",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
|
@ -187,6 +193,12 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
Name: dt.String(),
|
||||
DataType: dt,
|
||||
ElementType: schemapb.DataType_Int32,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.MaxLengthKey,
|
||||
Value: "128",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
|
|
@ -159,7 +159,7 @@ func (r *rowParser) combineDynamicRow(dynamicValues map[string]any, row Row) err
|
|||
row[dynamicFieldID] = data
|
||||
} else {
|
||||
// case 3
|
||||
row[dynamicFieldID] = "{}"
|
||||
row[dynamicFieldID] = []byte("{}")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -0,0 +1,120 @@
|
|||
// Code generated by mockery v2.30.1. DO NOT EDIT.
|
||||
|
||||
package importutilv2
|
||||
|
||||
import (
|
||||
storage "github.com/milvus-io/milvus/internal/storage"
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
)
|
||||
|
||||
// MockReader is an autogenerated mock type for the Reader type
|
||||
type MockReader struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
type MockReader_Expecter struct {
|
||||
mock *mock.Mock
|
||||
}
|
||||
|
||||
func (_m *MockReader) EXPECT() *MockReader_Expecter {
|
||||
return &MockReader_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// Close provides a mock function with given fields:
|
||||
func (_m *MockReader) Close() {
|
||||
_m.Called()
|
||||
}
|
||||
|
||||
// MockReader_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
|
||||
type MockReader_Close_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Close is a helper method to define mock.On call
|
||||
func (_e *MockReader_Expecter) Close() *MockReader_Close_Call {
|
||||
return &MockReader_Close_Call{Call: _e.mock.On("Close")}
|
||||
}
|
||||
|
||||
func (_c *MockReader_Close_Call) Run(run func()) *MockReader_Close_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReader_Close_Call) Return() *MockReader_Close_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReader_Close_Call) RunAndReturn(run func()) *MockReader_Close_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Read provides a mock function with given fields:
|
||||
func (_m *MockReader) Read() (*storage.InsertData, error) {
|
||||
ret := _m.Called()
|
||||
|
||||
var r0 *storage.InsertData
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func() (*storage.InsertData, error)); ok {
|
||||
return rf()
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func() *storage.InsertData); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*storage.InsertData)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func() error); ok {
|
||||
r1 = rf()
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockReader_Read_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Read'
|
||||
type MockReader_Read_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Read is a helper method to define mock.On call
|
||||
func (_e *MockReader_Expecter) Read() *MockReader_Read_Call {
|
||||
return &MockReader_Read_Call{Call: _e.mock.On("Read")}
|
||||
}
|
||||
|
||||
func (_c *MockReader_Read_Call) Run(run func()) *MockReader_Read_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run()
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReader_Read_Call) Return(_a0 *storage.InsertData, _a1 error) *MockReader_Read_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReader_Read_Call) RunAndReturn(run func() (*storage.InsertData, error)) *MockReader_Read_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// NewMockReader creates a new instance of MockReader. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
|
||||
// The first argument is typically a *testing.T value.
|
||||
func NewMockReader(t interface {
|
||||
mock.TestingT
|
||||
Cleanup(func())
|
||||
}) *MockReader {
|
||||
mock := &MockReader{}
|
||||
mock.Mock.Test(t)
|
||||
|
||||
t.Cleanup(func() { mock.AssertExpectations(t) })
|
||||
|
||||
return mock
|
||||
}
|
|
@ -17,12 +17,17 @@
|
|||
package numpy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
|
||||
"github.com/samber/lo"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
type Reader struct {
|
||||
|
@ -31,7 +36,7 @@ type Reader struct {
|
|||
frs map[int64]*FieldReader // fieldID -> FieldReader
|
||||
}
|
||||
|
||||
func NewReader(schema *schemapb.CollectionSchema, readers map[int64]io.Reader, bufferSize int) (*Reader, error) {
|
||||
func NewReader(ctx context.Context, schema *schemapb.CollectionSchema, paths []string, cm storage.ChunkManager, bufferSize int) (*Reader, error) {
|
||||
fields := lo.KeyBy(schema.GetFields(), func(field *schemapb.FieldSchema) int64 {
|
||||
return field.GetFieldID()
|
||||
})
|
||||
|
@ -40,6 +45,10 @@ func NewReader(schema *schemapb.CollectionSchema, readers map[int64]io.Reader, b
|
|||
return nil, err
|
||||
}
|
||||
crs := make(map[int64]*FieldReader)
|
||||
readers, err := CreateReaders(ctx, paths, cm, schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for fieldID, r := range readers {
|
||||
cr, err := NewFieldReader(r, fields[fieldID])
|
||||
if err != nil {
|
||||
|
@ -60,18 +69,23 @@ func (r *Reader) Read() (*storage.InsertData, error) {
|
|||
return nil, err
|
||||
}
|
||||
for fieldID, cr := range r.frs {
|
||||
data, err := cr.Next(r.count)
|
||||
var data any
|
||||
data, err = cr.Next(r.count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if data == nil {
|
||||
return nil, nil
|
||||
return nil, io.EOF
|
||||
}
|
||||
err = insertData.Data[fieldID].AppendRows(data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
err = fillDynamicData(insertData, r.schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return insertData, nil
|
||||
}
|
||||
|
||||
|
@ -80,3 +94,28 @@ func (r *Reader) Close() {
|
|||
cr.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func CreateReaders(ctx context.Context, paths []string, cm storage.ChunkManager, schema *schemapb.CollectionSchema) (map[int64]io.Reader, error) {
|
||||
readers := make(map[int64]io.Reader)
|
||||
nameToPath := lo.SliceToMap(paths, func(path string) (string, string) {
|
||||
nameWithExt := filepath.Base(path)
|
||||
name := strings.TrimSuffix(nameWithExt, filepath.Ext(nameWithExt))
|
||||
return name, path
|
||||
})
|
||||
for _, field := range schema.GetFields() {
|
||||
if field.GetIsPrimaryKey() && field.GetAutoID() {
|
||||
continue
|
||||
}
|
||||
if _, ok := nameToPath[field.GetName()]; !ok {
|
||||
return nil, merr.WrapErrImportFailed(
|
||||
fmt.Sprintf("no file for field: %s, files: %v", field.GetName(), lo.Values(nameToPath)))
|
||||
}
|
||||
reader, err := cm.Reader(ctx, nameToPath[field.GetName()])
|
||||
if err != nil {
|
||||
return nil, merr.WrapErrImportFailed(
|
||||
fmt.Sprintf("failed to read the file '%s', error: %s", nameToPath[field.GetName()], err.Error()))
|
||||
}
|
||||
readers[field.GetFieldID()] = reader
|
||||
}
|
||||
return readers, nil
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ package numpy
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
rand2 "crypto/rand"
|
||||
"fmt"
|
||||
"io"
|
||||
|
@ -30,11 +31,13 @@ import (
|
|||
"github.com/samber/lo"
|
||||
"github.com/sbinet/npyio"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"golang.org/x/exp/slices"
|
||||
|
||||
"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/mocks"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
|
@ -218,8 +221,18 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
fieldIDToField := lo.KeyBy(schema.GetFields(), func(field *schemapb.FieldSchema) int64 {
|
||||
return field.GetFieldID()
|
||||
})
|
||||
files := make(map[int64]string)
|
||||
for _, field := range schema.GetFields() {
|
||||
files[field.GetFieldID()] = fmt.Sprintf("%s.npy", field.GetName())
|
||||
}
|
||||
|
||||
readers := make(map[int64]io.Reader)
|
||||
cm := mocks.NewChunkManager(suite.T())
|
||||
type mockReader struct {
|
||||
io.Reader
|
||||
io.Closer
|
||||
io.ReaderAt
|
||||
io.Seeker
|
||||
}
|
||||
for fieldID, fieldData := range insertData.Data {
|
||||
dataType := fieldIDToField[fieldID].GetDataType()
|
||||
if dataType == schemapb.DataType_JSON {
|
||||
|
@ -230,7 +243,9 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
}
|
||||
reader, err := CreateReader(jsonStrs)
|
||||
suite.NoError(err)
|
||||
readers[fieldID] = reader
|
||||
cm.EXPECT().Reader(mock.Anything, files[fieldID]).Return(&mockReader{
|
||||
Reader: reader,
|
||||
}, nil)
|
||||
} else if dataType == schemapb.DataType_FloatVector {
|
||||
chunked := lo.Chunk(insertData.Data[fieldID].GetRows().([]float32), dim)
|
||||
chunkedRows := make([][dim]float32, len(chunked))
|
||||
|
@ -239,7 +254,9 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
}
|
||||
reader, err := CreateReader(chunkedRows)
|
||||
suite.NoError(err)
|
||||
readers[fieldID] = reader
|
||||
cm.EXPECT().Reader(mock.Anything, files[fieldID]).Return(&mockReader{
|
||||
Reader: reader,
|
||||
}, nil)
|
||||
} else if dataType == schemapb.DataType_BinaryVector {
|
||||
chunked := lo.Chunk(insertData.Data[fieldID].GetRows().([]byte), dim/8)
|
||||
chunkedRows := make([][dim / 8]byte, len(chunked))
|
||||
|
@ -248,15 +265,19 @@ func (suite *ReaderSuite) run(dt schemapb.DataType) {
|
|||
}
|
||||
reader, err := CreateReader(chunkedRows)
|
||||
suite.NoError(err)
|
||||
readers[fieldID] = reader
|
||||
cm.EXPECT().Reader(mock.Anything, files[fieldID]).Return(&mockReader{
|
||||
Reader: reader,
|
||||
}, nil)
|
||||
} else {
|
||||
reader, err := CreateReader(insertData.Data[fieldID].GetRows())
|
||||
suite.NoError(err)
|
||||
readers[fieldID] = reader
|
||||
cm.EXPECT().Reader(mock.Anything, files[fieldID]).Return(&mockReader{
|
||||
Reader: reader,
|
||||
}, nil)
|
||||
}
|
||||
}
|
||||
|
||||
reader, err := NewReader(schema, readers, math.MaxInt)
|
||||
reader, err := NewReader(context.Background(), schema, lo.Values(files), cm, math.MaxInt)
|
||||
suite.NoError(err)
|
||||
|
||||
checkFn := func(actualInsertData *storage.InsertData, offsetBegin, expectRows int) {
|
||||
|
|
|
@ -24,11 +24,13 @@ import (
|
|||
"strconv"
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/samber/lo"
|
||||
"github.com/sbinet/npyio"
|
||||
"github.com/sbinet/npyio/npy"
|
||||
"golang.org/x/text/encoding/unicode"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
@ -233,10 +235,44 @@ func validateHeader(npyReader *npy.Reader, field *schemapb.FieldSchema, dim int)
|
|||
}
|
||||
|
||||
func calcRowCount(bufferSize int, schema *schemapb.CollectionSchema) (int64, error) {
|
||||
sizePerRecord, err := typeutil.EstimateSizePerRecord(schema)
|
||||
sizePerRecord, err := typeutil.EstimateMaxSizePerRecord(schema)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
rowCount := int64(bufferSize) / int64(sizePerRecord)
|
||||
return rowCount, nil
|
||||
}
|
||||
|
||||
func fillDynamicData(data *storage.InsertData, schema *schemapb.CollectionSchema) error {
|
||||
if !schema.GetEnableDynamicField() {
|
||||
return nil
|
||||
}
|
||||
dynamicField := typeutil.GetDynamicField(schema)
|
||||
if dynamicField == nil {
|
||||
return nil
|
||||
}
|
||||
rowNum := GetInsertDataRowNum(data, schema)
|
||||
dynamicData := data.Data[dynamicField.GetFieldID()]
|
||||
jsonFD := dynamicData.(*storage.JSONFieldData)
|
||||
bs := []byte("{}")
|
||||
for i := 0; i < rowNum-dynamicData.RowNum(); i++ {
|
||||
jsonFD.Data = append(jsonFD.Data, bs)
|
||||
}
|
||||
data.Data[dynamicField.GetFieldID()] = dynamicData
|
||||
return nil
|
||||
}
|
||||
|
||||
func GetInsertDataRowNum(data *storage.InsertData, schema *schemapb.CollectionSchema) int {
|
||||
fields := lo.KeyBy(schema.GetFields(), func(field *schemapb.FieldSchema) int64 {
|
||||
return field.GetFieldID()
|
||||
})
|
||||
for fieldID, fd := range data.Data {
|
||||
if fields[fieldID].GetIsDynamic() {
|
||||
continue
|
||||
}
|
||||
if fd.RowNum() != 0 {
|
||||
return fd.RowNum()
|
||||
}
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
|
|
@ -40,8 +40,8 @@ type FieldReader struct {
|
|||
field *schemapb.FieldSchema
|
||||
}
|
||||
|
||||
func NewFieldReader(reader *pqarrow.FileReader, columnIndex int, field *schemapb.FieldSchema) (*FieldReader, error) {
|
||||
columnReader, err := reader.GetColumn(context.Background(), columnIndex) // TODO: dyh, resolve context
|
||||
func NewFieldReader(ctx context.Context, reader *pqarrow.FileReader, columnIndex int, field *schemapb.FieldSchema) (*FieldReader, error) {
|
||||
columnReader, err := reader.GetColumn(ctx, columnIndex)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -17,7 +17,9 @@
|
|||
package parquet
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
|
||||
"github.com/apache/arrow/go/v12/arrow/memory"
|
||||
"github.com/apache/arrow/go/v12/parquet"
|
||||
|
@ -35,16 +37,15 @@ type Reader struct {
|
|||
reader *file.Reader
|
||||
|
||||
bufferSize int
|
||||
count int64
|
||||
|
||||
schema *schemapb.CollectionSchema
|
||||
frs map[int64]*FieldReader // fieldID -> FieldReader
|
||||
}
|
||||
|
||||
func NewReader(schema *schemapb.CollectionSchema, cmReader storage.FileReader, bufferSize int) (*Reader, error) {
|
||||
const pqBufSize = 32 * 1024 * 1024 // TODO: dyh, make if configurable
|
||||
size := calcBufferSize(pqBufSize, schema)
|
||||
func NewReader(ctx context.Context, schema *schemapb.CollectionSchema, cmReader storage.FileReader, bufferSize int) (*Reader, error) {
|
||||
reader, err := file.NewParquetReader(cmReader, file.WithReadProps(&parquet.ReaderProperties{
|
||||
BufferSize: int64(size),
|
||||
BufferSize: int64(bufferSize),
|
||||
BufferedStreamEnabled: true,
|
||||
}))
|
||||
if err != nil {
|
||||
|
@ -58,13 +59,18 @@ func NewReader(schema *schemapb.CollectionSchema, cmReader storage.FileReader, b
|
|||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("new parquet file reader failed, err=%v", err))
|
||||
}
|
||||
|
||||
crs, err := CreateFieldReaders(fileReader, schema)
|
||||
crs, err := CreateFieldReaders(ctx, fileReader, schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
count, err := estimateReadCountPerBatch(bufferSize, schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Reader{
|
||||
reader: reader,
|
||||
bufferSize: bufferSize,
|
||||
count: count,
|
||||
schema: schema,
|
||||
frs: crs,
|
||||
}, nil
|
||||
|
@ -78,7 +84,7 @@ func (r *Reader) Read() (*storage.InsertData, error) {
|
|||
OUTER:
|
||||
for {
|
||||
for fieldID, cr := range r.frs {
|
||||
data, err := cr.Next(1)
|
||||
data, err := cr.Next(r.count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -96,7 +102,7 @@ OUTER:
|
|||
}
|
||||
for fieldID := range r.frs {
|
||||
if insertData.Data[fieldID].RowNum() == 0 {
|
||||
return nil, nil
|
||||
return nil, io.EOF
|
||||
}
|
||||
}
|
||||
return insertData, nil
|
||||
|
|
|
@ -20,7 +20,6 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"math/rand"
|
||||
"os"
|
||||
"testing"
|
||||
|
@ -357,6 +356,12 @@ func (s *ReaderSuite) run(dt schemapb.DataType) {
|
|||
Name: "pk",
|
||||
IsPrimaryKey: true,
|
||||
DataType: s.pkDataType,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "max_length",
|
||||
Value: "256",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
|
@ -374,6 +379,12 @@ func (s *ReaderSuite) run(dt schemapb.DataType) {
|
|||
Name: dt.String(),
|
||||
DataType: dt,
|
||||
ElementType: schemapb.DataType_Int32,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "max_length",
|
||||
Value: "256",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
@ -391,7 +402,7 @@ func (s *ReaderSuite) run(dt schemapb.DataType) {
|
|||
assert.NoError(s.T(), err)
|
||||
cmReader, err := cm.Reader(ctx, filePath)
|
||||
assert.NoError(s.T(), err)
|
||||
reader, err := NewReader(schema, cmReader, math.MaxInt)
|
||||
reader, err := NewReader(ctx, schema, cmReader, 64*1024*1024)
|
||||
s.NoError(err)
|
||||
|
||||
checkFn := func(actualInsertData *storage.InsertData, offsetBegin, expectRows int) {
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package parquet
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/apache/arrow/go/v12/arrow"
|
||||
|
@ -41,7 +42,7 @@ func calcBufferSize(blockSize int, schema *schemapb.CollectionSchema) int {
|
|||
return blockSize / len(schema.GetFields())
|
||||
}
|
||||
|
||||
func CreateFieldReaders(fileReader *pqarrow.FileReader, schema *schemapb.CollectionSchema) (map[int64]*FieldReader, error) {
|
||||
func CreateFieldReaders(ctx context.Context, fileReader *pqarrow.FileReader, schema *schemapb.CollectionSchema) (map[int64]*FieldReader, error) {
|
||||
nameToField := lo.KeyBy(schema.GetFields(), func(field *schemapb.FieldSchema) string {
|
||||
return field.GetName()
|
||||
})
|
||||
|
@ -78,7 +79,7 @@ func CreateFieldReaders(fileReader *pqarrow.FileReader, schema *schemapb.Collect
|
|||
return nil, WrapTypeErr(dataType.String(), pqField.Type.Name(), field)
|
||||
}
|
||||
|
||||
cr, err := NewFieldReader(fileReader, i, field)
|
||||
cr, err := NewFieldReader(ctx, fileReader, i, field)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -173,3 +174,14 @@ func isConvertible(src, dst schemapb.DataType, isList bool) bool {
|
|||
return false
|
||||
}
|
||||
}
|
||||
|
||||
func estimateReadCountPerBatch(bufferSize int, schema *schemapb.CollectionSchema) (int64, error) {
|
||||
sizePerRecord, err := typeutil.EstimateMaxSizePerRecord(schema)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if 1000*sizePerRecord <= bufferSize {
|
||||
return 1000, nil
|
||||
}
|
||||
return int64(bufferSize) / int64(sizePerRecord), nil
|
||||
}
|
||||
|
|
|
@ -17,29 +17,59 @@
|
|||
package importutilv2
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2/binlog"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2/json"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2/numpy"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2/parquet"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
//go:generate mockery --name=Reader --structname=MockReader --output=./ --filename=mock_reader.go --with-expecter --inpackage
|
||||
type Reader interface {
|
||||
Read() (*storage.InsertData, error)
|
||||
Close()
|
||||
}
|
||||
|
||||
func NewReader(cm storage.ChunkManager,
|
||||
func NewReader(ctx context.Context,
|
||||
cm storage.ChunkManager,
|
||||
schema *schemapb.CollectionSchema,
|
||||
paths []string,
|
||||
importFile *internalpb.ImportFile,
|
||||
options Options,
|
||||
bufferSize int64,
|
||||
bufferSize int,
|
||||
) (Reader, error) {
|
||||
if IsBackup(options) {
|
||||
tsStart, tsEnd, err := ParseTimeRange(options)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return binlog.NewReader(cm, schema, paths, tsStart, tsEnd)
|
||||
paths := importFile.GetPaths()
|
||||
return binlog.NewReader(ctx, cm, schema, paths, tsStart, tsEnd)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
fileType, err := GetFileType(importFile)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
switch fileType {
|
||||
case JSON:
|
||||
reader, err := cm.Reader(ctx, importFile.GetPaths()[0])
|
||||
if err != nil {
|
||||
return nil, WrapReadFileError(importFile.GetPaths()[0], err)
|
||||
}
|
||||
return json.NewReader(reader, schema, bufferSize)
|
||||
case Numpy:
|
||||
return numpy.NewReader(ctx, schema, importFile.GetPaths(), cm, bufferSize)
|
||||
case Parquet:
|
||||
cmReader, err := cm.Reader(ctx, importFile.GetPaths()[0])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return parquet.NewReader(ctx, schema, cmReader, bufferSize)
|
||||
}
|
||||
return nil, merr.WrapErrImportFailed("unexpected import file")
|
||||
}
|
||||
|
|
|
@ -0,0 +1,89 @@
|
|||
// 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 importutilv2
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
|
||||
"github.com/samber/lo"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
type FileType int
|
||||
|
||||
const (
|
||||
Invalid FileType = 0
|
||||
JSON FileType = 1
|
||||
Numpy FileType = 2
|
||||
Parquet FileType = 3
|
||||
|
||||
JSONFileExt = ".json"
|
||||
NumpyFileExt = ".npy"
|
||||
ParquetFileExt = ".parquet"
|
||||
)
|
||||
|
||||
var FileTypeName = map[int]string{
|
||||
0: "Invalid",
|
||||
1: "JSON",
|
||||
2: "Numpy",
|
||||
3: "Parquet",
|
||||
}
|
||||
|
||||
func (f FileType) String() string {
|
||||
return FileTypeName[int(f)]
|
||||
}
|
||||
|
||||
func WrapReadFileError(file string, err error) error {
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to read the file '%s', error: %s", file, err.Error()))
|
||||
}
|
||||
|
||||
func GetFileType(file *internalpb.ImportFile) (FileType, error) {
|
||||
if len(file.GetPaths()) == 0 {
|
||||
return Invalid, merr.WrapErrImportFailed("no file to import")
|
||||
}
|
||||
exts := lo.Map(file.GetPaths(), func(path string, _ int) string {
|
||||
return filepath.Ext(path)
|
||||
})
|
||||
|
||||
ext := exts[0]
|
||||
for i := 1; i < len(exts); i++ {
|
||||
if exts[i] != ext {
|
||||
return Invalid, merr.WrapErrImportFailed(
|
||||
fmt.Sprintf("inconsistency in file types, (%s) vs (%s)",
|
||||
file.GetPaths()[0], file.GetPaths()[i]))
|
||||
}
|
||||
}
|
||||
|
||||
switch ext {
|
||||
case JSONFileExt:
|
||||
if len(file.GetPaths()) != 1 {
|
||||
return Invalid, merr.WrapErrImportFailed("for JSON import, accepts only one file")
|
||||
}
|
||||
return JSON, nil
|
||||
case NumpyFileExt:
|
||||
return Numpy, nil
|
||||
case ParquetFileExt:
|
||||
if len(file.GetPaths()) != 1 {
|
||||
return Invalid, merr.WrapErrImportFailed("for Parquet import, accepts only one file")
|
||||
}
|
||||
return Parquet, nil
|
||||
}
|
||||
return Invalid, merr.WrapErrImportFailed(fmt.Sprintf("unexpect file type, files=%v", file.GetPaths()))
|
||||
}
|
|
@ -2846,6 +2846,8 @@ type dataNodeConfig struct {
|
|||
ChannelWorkPoolSize ParamItem `refreshable:"true"`
|
||||
|
||||
UpdateChannelCheckpointMaxParallel ParamItem `refreshable:"true"`
|
||||
|
||||
MaxConcurrentImportTaskNum ParamItem `refreshable:"true"`
|
||||
}
|
||||
|
||||
func (p *dataNodeConfig) init(base *BaseTable) {
|
||||
|
@ -3080,6 +3082,16 @@ func (p *dataNodeConfig) init(base *BaseTable) {
|
|||
DefaultValue: "1000",
|
||||
}
|
||||
p.UpdateChannelCheckpointMaxParallel.Init(base.mgr)
|
||||
|
||||
p.MaxConcurrentImportTaskNum = ParamItem{
|
||||
Key: "datanode.import.maxConcurrentTaskNum",
|
||||
Version: "2.4.0",
|
||||
Doc: "The maximum number of import/pre-import tasks allowed to run concurrently on a datanode.",
|
||||
DefaultValue: "16",
|
||||
PanicIfEmpty: false,
|
||||
Export: true,
|
||||
}
|
||||
p.MaxConcurrentImportTaskNum.Init(base.mgr)
|
||||
}
|
||||
|
||||
// /////////////////////////////////////////////////////////////////////////////
|
||||
|
|
|
@ -407,6 +407,10 @@ func TestComponentParam(t *testing.T) {
|
|||
updateChannelCheckpointMaxParallel := Params.UpdateChannelCheckpointMaxParallel.GetAsInt()
|
||||
t.Logf("updateChannelCheckpointMaxParallel: %d", updateChannelCheckpointMaxParallel)
|
||||
assert.Equal(t, 1000, Params.UpdateChannelCheckpointMaxParallel.GetAsInt())
|
||||
|
||||
maxConcurrentImportTaskNum := Params.MaxConcurrentImportTaskNum.GetAsInt()
|
||||
t.Logf("maxConcurrentImportTaskNum: %d", maxConcurrentImportTaskNum)
|
||||
assert.Equal(t, 16, maxConcurrentImportTaskNum)
|
||||
})
|
||||
|
||||
t.Run("test indexNodeConfig", func(t *testing.T) {
|
||||
|
|
|
@ -23,6 +23,7 @@ import (
|
|||
"unsafe"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
|
||||
|
@ -33,7 +34,15 @@ import (
|
|||
|
||||
const DynamicFieldMaxLength = 512
|
||||
|
||||
func GetAvgLengthOfVarLengthField(fieldSchema *schemapb.FieldSchema) (int, error) {
|
||||
type getVariableFieldLengthPolicy int
|
||||
|
||||
const (
|
||||
max getVariableFieldLengthPolicy = 0
|
||||
avg getVariableFieldLengthPolicy = 1
|
||||
custom getVariableFieldLengthPolicy = 2
|
||||
)
|
||||
|
||||
func getVarFieldLength(fieldSchema *schemapb.FieldSchema, policy getVariableFieldLengthPolicy) (int, error) {
|
||||
maxLength := 0
|
||||
var err error
|
||||
|
||||
|
@ -52,22 +61,43 @@ func GetAvgLengthOfVarLengthField(fieldSchema *schemapb.FieldSchema) (int, error
|
|||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
switch policy {
|
||||
case max:
|
||||
return maxLength, nil
|
||||
case avg:
|
||||
return maxLength / 2, nil
|
||||
case custom:
|
||||
// TODO this is a hack and may not accurate, we should rely on estimate size per record
|
||||
// However we should report size and datacoord calculate based on size
|
||||
// https://github.com/milvus-io/milvus/issues/17687
|
||||
if maxLength > 256 {
|
||||
return 256, nil
|
||||
}
|
||||
return maxLength, nil
|
||||
default:
|
||||
return 0, fmt.Errorf("unrecognized getVariableFieldLengthPolicy %v", policy)
|
||||
}
|
||||
case schemapb.DataType_Array, schemapb.DataType_JSON:
|
||||
return DynamicFieldMaxLength, nil
|
||||
default:
|
||||
return 0, fmt.Errorf("field %s is not a variable-length type", fieldSchema.DataType.String())
|
||||
}
|
||||
|
||||
// TODO this is a hack and may not accurate, we should rely on estimate size per record
|
||||
// However we should report size and datacoord calculate based on size
|
||||
if maxLength > 256 {
|
||||
return 256, nil
|
||||
}
|
||||
return maxLength, nil
|
||||
}
|
||||
|
||||
// EstimateSizePerRecord returns the estimate size of a record in a collection
|
||||
func EstimateSizePerRecord(schema *schemapb.CollectionSchema) (int, error) {
|
||||
return estimateSizeBy(schema, custom)
|
||||
}
|
||||
|
||||
func EstimateMaxSizePerRecord(schema *schemapb.CollectionSchema) (int, error) {
|
||||
return estimateSizeBy(schema, max)
|
||||
}
|
||||
|
||||
func EstimateAvgSizePerRecord(schema *schemapb.CollectionSchema) (int, error) {
|
||||
return estimateSizeBy(schema, avg)
|
||||
}
|
||||
|
||||
func estimateSizeBy(schema *schemapb.CollectionSchema, policy getVariableFieldLengthPolicy) (int, error) {
|
||||
res := 0
|
||||
for _, fs := range schema.Fields {
|
||||
switch fs.DataType {
|
||||
|
@ -80,7 +110,7 @@ func EstimateSizePerRecord(schema *schemapb.CollectionSchema) (int, error) {
|
|||
case schemapb.DataType_Int64, schemapb.DataType_Double:
|
||||
res += 8
|
||||
case schemapb.DataType_VarChar, schemapb.DataType_Array, schemapb.DataType_JSON:
|
||||
maxLengthPerRow, err := GetAvgLengthOfVarLengthField(fs)
|
||||
maxLengthPerRow, err := getVarFieldLength(fs, policy)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
@ -1014,19 +1044,21 @@ func IsPrimaryFieldDataExist(datas []*schemapb.FieldData, primaryFieldSchema *sc
|
|||
return primaryFieldData != nil
|
||||
}
|
||||
|
||||
func AppendSystemFields(schema *schemapb.CollectionSchema) {
|
||||
schema.Fields = append(schema.Fields, &schemapb.FieldSchema{
|
||||
func AppendSystemFields(schema *schemapb.CollectionSchema) *schemapb.CollectionSchema {
|
||||
newSchema := proto.Clone(schema).(*schemapb.CollectionSchema)
|
||||
newSchema.Fields = append(newSchema.Fields, &schemapb.FieldSchema{
|
||||
FieldID: int64(common.RowIDField),
|
||||
Name: common.RowIDFieldName,
|
||||
IsPrimaryKey: false,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
})
|
||||
schema.Fields = append(schema.Fields, &schemapb.FieldSchema{
|
||||
newSchema.Fields = append(newSchema.Fields, &schemapb.FieldSchema{
|
||||
FieldID: int64(common.TimeStampField),
|
||||
Name: common.TimeStampFieldName,
|
||||
IsPrimaryKey: false,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
})
|
||||
return newSchema
|
||||
}
|
||||
|
||||
func AppendIDs(dst *schemapb.IDs, src *schemapb.IDs, idx int) {
|
||||
|
|
Loading…
Reference in New Issue