mirror of https://github.com/milvus-io/milvus.git
Remove all code and logic related to import v1. issue: https://github.com/milvus-io/milvus/issues/28521 pr: https://github.com/milvus-io/milvus/pull/31403 Signed-off-by: bigsheeper <yihao.dai@zilliz.com>pull/31542/head^2
parent
5d3aa2a496
commit
1e0bf5acd2
|
@ -176,8 +176,6 @@ rootCoord:
|
|||
maxDatabaseNum: 64 # Maximum number of database
|
||||
maxPartitionNum: 4096 # Maximum number of partitions in a collection
|
||||
minSegmentSizeToEnableIndex: 1024 # It's a threshold. When the segment size is less than this value, the segment will not be indexed
|
||||
importTaskExpiration: 900 # (in seconds) Duration after which an import task will expire (be killed). Default 900 seconds (15 minutes).
|
||||
importTaskRetention: 86400 # (in seconds) Milvus will keep the record of import tasks for at least `importTaskRetention` seconds. Default 86400, seconds (24 hours).
|
||||
enableActiveStandby: false
|
||||
# can specify ip for example
|
||||
# ip: 127.0.0.1
|
||||
|
@ -601,9 +599,6 @@ common:
|
|||
# need to set a separated topic to stand for currently consumed timestamp for each channel
|
||||
timeticker: 'timetick-channel'
|
||||
|
||||
ImportMaxFileSize: 17179869184 # 16 * 1024 * 1024 * 1024
|
||||
# max file size to import for bulkInsert
|
||||
|
||||
locks:
|
||||
metrics:
|
||||
enable: false
|
||||
|
|
|
@ -27,7 +27,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
|
@ -39,8 +38,6 @@ type Cluster interface {
|
|||
Watch(ctx context.Context, ch string, collectionID UniqueID) error
|
||||
Flush(ctx context.Context, nodeID int64, channel string, segments []*datapb.SegmentInfo) error
|
||||
FlushChannels(ctx context.Context, nodeID int64, flushTs Timestamp, channels []string) error
|
||||
Import(ctx context.Context, nodeID int64, it *datapb.ImportTaskRequest)
|
||||
AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)
|
||||
PreImport(nodeID int64, in *datapb.PreImportRequest) error
|
||||
ImportV2(nodeID int64, in *datapb.ImportRequest) error
|
||||
QueryPreImport(nodeID int64, in *datapb.QueryPreImportRequest) (*datapb.QueryPreImportResponse, error)
|
||||
|
@ -150,22 +147,6 @@ func (c *ClusterImpl) FlushChannels(ctx context.Context, nodeID int64, flushTs T
|
|||
return c.sessionManager.FlushChannels(ctx, nodeID, req)
|
||||
}
|
||||
|
||||
// Import sends import requests to DataNodes whose ID==nodeID.
|
||||
func (c *ClusterImpl) Import(ctx context.Context, nodeID int64, it *datapb.ImportTaskRequest) {
|
||||
c.sessionManager.Import(ctx, nodeID, it)
|
||||
}
|
||||
|
||||
func (c *ClusterImpl) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
// Look for the DataNode that watches the channel.
|
||||
ok, nodeID := c.channelManager.GetNodeIDByChannelName(req.GetChannelName())
|
||||
if !ok {
|
||||
err := merr.WrapErrChannelNotFound(req.GetChannelName(), "no DataNode watches this channel")
|
||||
log.Error("no DataNode found for channel", zap.String("channelName", req.GetChannelName()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
return c.sessionManager.AddImportSegment(ctx, nodeID, req)
|
||||
}
|
||||
|
||||
func (c *ClusterImpl) PreImport(nodeID int64, in *datapb.PreImportRequest) error {
|
||||
return c.sessionManager.PreImport(nodeID, in)
|
||||
}
|
||||
|
|
|
@ -28,8 +28,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/kv/mocks"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/testutils"
|
||||
)
|
||||
|
||||
|
@ -175,39 +173,3 @@ func (suite *ClusterSuite) TestFlushChannels() {
|
|||
suite.NoError(err)
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *ClusterSuite) TestImport() {
|
||||
suite.mockSession.EXPECT().Import(mock.Anything, mock.Anything, mock.Anything).Return().Once()
|
||||
cluster := NewClusterImpl(suite.mockSession, suite.mockChManager)
|
||||
suite.NotPanics(func() {
|
||||
cluster.Import(context.Background(), 1, nil)
|
||||
})
|
||||
}
|
||||
|
||||
func (suite *ClusterSuite) TestAddImportSegment() {
|
||||
suite.Run("channel not fount", func() {
|
||||
suite.SetupTest()
|
||||
suite.mockChManager.EXPECT().GetNodeIDByChannelName(mock.Anything).Return(false, 0)
|
||||
cluster := NewClusterImpl(suite.mockSession, suite.mockChManager)
|
||||
resp, err := cluster.AddImportSegment(context.Background(), &datapb.AddImportSegmentRequest{
|
||||
ChannelName: "ch-1",
|
||||
})
|
||||
|
||||
suite.ErrorIs(err, merr.ErrChannelNotFound)
|
||||
suite.Nil(resp)
|
||||
})
|
||||
|
||||
suite.Run("normal", func() {
|
||||
suite.SetupTest()
|
||||
suite.mockChManager.EXPECT().GetNodeIDByChannelName(mock.Anything).Return(true, 0)
|
||||
suite.mockSession.EXPECT().AddImportSegment(mock.Anything, mock.Anything, mock.Anything).Return(&datapb.AddImportSegmentResponse{}, nil)
|
||||
|
||||
cluster := NewClusterImpl(suite.mockSession, suite.mockChManager)
|
||||
resp, err := cluster.AddImportSegment(context.Background(), &datapb.AddImportSegmentRequest{
|
||||
ChannelName: "ch-1",
|
||||
})
|
||||
|
||||
suite.NoError(err)
|
||||
suite.NotNil(resp)
|
||||
})
|
||||
}
|
||||
|
|
|
@ -155,7 +155,7 @@ func Test_garbageCollector_scan(t *testing.T) {
|
|||
gc.close()
|
||||
})
|
||||
t.Run("hit, no gc", func(t *testing.T) {
|
||||
segment := buildSegment(1, 10, 100, "ch", false)
|
||||
segment := buildSegment(1, 10, 100, "ch")
|
||||
segment.State = commonpb.SegmentState_Flushed
|
||||
segment.Binlogs = []*datapb.FieldBinlog{getFieldBinlogPaths(0, inserts[0])}
|
||||
segment.Statslogs = []*datapb.FieldBinlog{getFieldBinlogPaths(0, stats[0])}
|
||||
|
@ -182,7 +182,7 @@ func Test_garbageCollector_scan(t *testing.T) {
|
|||
})
|
||||
|
||||
t.Run("dropped gc one", func(t *testing.T) {
|
||||
segment := buildSegment(1, 10, 100, "ch", false)
|
||||
segment := buildSegment(1, 10, 100, "ch")
|
||||
segment.State = commonpb.SegmentState_Dropped
|
||||
segment.DroppedAt = uint64(time.Now().Add(-time.Hour).UnixNano())
|
||||
segment.Binlogs = []*datapb.FieldBinlog{getFieldBinlogPaths(0, inserts[0])}
|
||||
|
|
|
@ -477,34 +477,6 @@ func (m *meta) UpdateSegment(segmentID int64, operators ...SegmentOperator) erro
|
|||
return nil
|
||||
}
|
||||
|
||||
// UnsetIsImporting removes the `isImporting` flag of a segment.
|
||||
func (m *meta) UnsetIsImporting(segmentID UniqueID) error {
|
||||
log.Debug("meta update: unsetting isImport state of segment",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
curSegInfo := m.segments.GetSegment(segmentID)
|
||||
if curSegInfo == nil {
|
||||
return fmt.Errorf("segment not found %d", segmentID)
|
||||
}
|
||||
// Persist segment updates first.
|
||||
clonedSegment := curSegInfo.Clone()
|
||||
clonedSegment.IsImporting = false
|
||||
if isSegmentHealthy(clonedSegment) {
|
||||
if err := m.catalog.AlterSegments(m.ctx, []*datapb.SegmentInfo{clonedSegment.SegmentInfo}); err != nil {
|
||||
log.Warn("meta update: unsetting isImport state of segment - failed to unset segment isImporting state",
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
// Update in-memory meta.
|
||||
m.segments.SetIsImporting(segmentID, false)
|
||||
log.Info("meta update: unsetting isImport state of segment - complete",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return nil
|
||||
}
|
||||
|
||||
type updateSegmentPack struct {
|
||||
meta *meta
|
||||
segments map[int64]*SegmentInfo
|
||||
|
@ -684,10 +656,8 @@ func UpdateDmlPosition(segmentID int64, dmlPosition *msgpb.MsgPosition) UpdateOp
|
|||
}
|
||||
}
|
||||
|
||||
// update segment checkpoint and num rows
|
||||
// if was importing segment
|
||||
// only update rows.
|
||||
func UpdateCheckPointOperator(segmentID int64, importing bool, checkpoints []*datapb.CheckPoint) UpdateOperator {
|
||||
// UpdateCheckPointOperator updates segment checkpoint and num rows
|
||||
func UpdateCheckPointOperator(segmentID int64, checkpoints []*datapb.CheckPoint) UpdateOperator {
|
||||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
|
@ -696,25 +666,21 @@ func UpdateCheckPointOperator(segmentID int64, importing bool, checkpoints []*da
|
|||
return false
|
||||
}
|
||||
|
||||
if importing {
|
||||
segment.NumOfRows = segment.currRows
|
||||
} else {
|
||||
for _, cp := range checkpoints {
|
||||
if cp.SegmentID != segmentID {
|
||||
// Don't think this is gonna to happen, ignore for now.
|
||||
log.Warn("checkpoint in segment is not same as flush segment to update, igreo", zap.Int64("current", segmentID), zap.Int64("checkpoint segment", cp.SegmentID))
|
||||
continue
|
||||
}
|
||||
|
||||
if segment.DmlPosition != nil && segment.DmlPosition.Timestamp >= cp.Position.Timestamp {
|
||||
log.Warn("checkpoint in segment is larger than reported", zap.Any("current", segment.GetDmlPosition()), zap.Any("reported", cp.GetPosition()))
|
||||
// segment position in etcd is larger than checkpoint, then dont change it
|
||||
continue
|
||||
}
|
||||
|
||||
segment.NumOfRows = cp.NumOfRows
|
||||
segment.DmlPosition = cp.GetPosition()
|
||||
for _, cp := range checkpoints {
|
||||
if cp.SegmentID != segmentID {
|
||||
// Don't think this is gonna to happen, ignore for now.
|
||||
log.Warn("checkpoint in segment is not same as flush segment to update, igreo", zap.Int64("current", segmentID), zap.Int64("checkpoint segment", cp.SegmentID))
|
||||
continue
|
||||
}
|
||||
|
||||
if segment.DmlPosition != nil && segment.DmlPosition.Timestamp >= cp.Position.Timestamp {
|
||||
log.Warn("checkpoint in segment is larger than reported", zap.Any("current", segment.GetDmlPosition()), zap.Any("reported", cp.GetPosition()))
|
||||
// segment position in etcd is larger than checkpoint, then dont change it
|
||||
continue
|
||||
}
|
||||
|
||||
segment.NumOfRows = cp.NumOfRows
|
||||
segment.DmlPosition = cp.GetPosition()
|
||||
}
|
||||
|
||||
count := segmentutil.CalcRowCountFromBinLog(segment.SegmentInfo)
|
||||
|
@ -1272,7 +1238,7 @@ func (m *meta) copyNewDeltalogs(latestCompactFromInfos []*SegmentInfo, logIDsInP
|
|||
}
|
||||
|
||||
// buildSegment utility function for compose datapb.SegmentInfo struct with provided info
|
||||
func buildSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, channelName string, isImporting bool) *SegmentInfo {
|
||||
func buildSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, channelName string) *SegmentInfo {
|
||||
info := &datapb.SegmentInfo{
|
||||
ID: segmentID,
|
||||
CollectionID: collectionID,
|
||||
|
@ -1280,7 +1246,6 @@ func buildSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueI
|
|||
InsertChannel: channelName,
|
||||
NumOfRows: 0,
|
||||
State: commonpb.SegmentState_Growing,
|
||||
IsImporting: isImporting,
|
||||
}
|
||||
return NewSegmentInfo(info)
|
||||
}
|
||||
|
|
|
@ -412,13 +412,13 @@ func TestMeta_Basic(t *testing.T) {
|
|||
// create seg0 for partition0, seg0/seg1 for partition1
|
||||
segID0_0, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
segInfo0_0 := buildSegment(collID, partID0, segID0_0, channelName, true)
|
||||
segInfo0_0 := buildSegment(collID, partID0, segID0_0, channelName)
|
||||
segID1_0, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
segInfo1_0 := buildSegment(collID, partID1, segID1_0, channelName, false)
|
||||
segInfo1_0 := buildSegment(collID, partID1, segID1_0, channelName)
|
||||
segID1_1, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
segInfo1_1 := buildSegment(collID, partID1, segID1_1, channelName, false)
|
||||
segInfo1_1 := buildSegment(collID, partID1, segID1_1, channelName)
|
||||
|
||||
// check AddSegment
|
||||
err = meta.AddSegment(context.TODO(), segInfo0_0)
|
||||
|
@ -467,28 +467,6 @@ func TestMeta_Basic(t *testing.T) {
|
|||
info0_0 = meta.GetHealthySegment(segID0_0)
|
||||
assert.NotNil(t, info0_0)
|
||||
assert.EqualValues(t, commonpb.SegmentState_Flushed, info0_0.State)
|
||||
|
||||
info0_0 = meta.GetHealthySegment(segID0_0)
|
||||
assert.NotNil(t, info0_0)
|
||||
assert.Equal(t, true, info0_0.GetIsImporting())
|
||||
err = meta.UnsetIsImporting(segID0_0)
|
||||
assert.NoError(t, err)
|
||||
info0_0 = meta.GetHealthySegment(segID0_0)
|
||||
assert.NotNil(t, info0_0)
|
||||
assert.Equal(t, false, info0_0.GetIsImporting())
|
||||
|
||||
// UnsetIsImporting on segment that does not exist.
|
||||
err = meta.UnsetIsImporting(segID1_0)
|
||||
assert.Error(t, err)
|
||||
|
||||
info1_1 := meta.GetHealthySegment(segID1_1)
|
||||
assert.NotNil(t, info1_1)
|
||||
assert.Equal(t, false, info1_1.GetIsImporting())
|
||||
err = meta.UnsetIsImporting(segID1_1)
|
||||
assert.NoError(t, err)
|
||||
info1_1 = meta.GetHealthySegment(segID1_1)
|
||||
assert.NotNil(t, info1_1)
|
||||
assert.Equal(t, false, info1_1.GetIsImporting())
|
||||
})
|
||||
|
||||
t.Run("Test segment with kv fails", func(t *testing.T) {
|
||||
|
@ -543,7 +521,7 @@ func TestMeta_Basic(t *testing.T) {
|
|||
// add seg1 with 100 rows
|
||||
segID0, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
segInfo0 := buildSegment(collID, partID0, segID0, channelName, false)
|
||||
segInfo0 := buildSegment(collID, partID0, segID0, channelName)
|
||||
segInfo0.NumOfRows = rowCount0
|
||||
err = meta.AddSegment(context.TODO(), segInfo0)
|
||||
assert.NoError(t, err)
|
||||
|
@ -551,7 +529,7 @@ func TestMeta_Basic(t *testing.T) {
|
|||
// add seg2 with 300 rows
|
||||
segID1, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
segInfo1 := buildSegment(collID, partID0, segID1, channelName, false)
|
||||
segInfo1 := buildSegment(collID, partID0, segID1, channelName)
|
||||
segInfo1.NumOfRows = rowCount1
|
||||
err = meta.AddSegment(context.TODO(), segInfo1)
|
||||
assert.NoError(t, err)
|
||||
|
@ -609,7 +587,7 @@ func TestMeta_Basic(t *testing.T) {
|
|||
// add seg0 with size0
|
||||
segID0, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
segInfo0 := buildSegment(collID, partID0, segID0, channelName, false)
|
||||
segInfo0 := buildSegment(collID, partID0, segID0, channelName)
|
||||
segInfo0.size.Store(size0)
|
||||
err = meta.AddSegment(context.TODO(), segInfo0)
|
||||
assert.NoError(t, err)
|
||||
|
@ -617,7 +595,7 @@ func TestMeta_Basic(t *testing.T) {
|
|||
// add seg1 with size1
|
||||
segID1, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
segInfo1 := buildSegment(collID, partID0, segID1, channelName, false)
|
||||
segInfo1 := buildSegment(collID, partID0, segID1, channelName)
|
||||
segInfo1.size.Store(size1)
|
||||
err = meta.AddSegment(context.TODO(), segInfo1)
|
||||
assert.NoError(t, err)
|
||||
|
@ -689,7 +667,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
|
|||
[]*datapb.FieldBinlog{{Binlogs: []*datapb.Binlog{{EntriesNum: 1, TimestampFrom: 100, TimestampTo: 200, LogSize: 1000, LogPath: getDeltaLogPath("deltalog1", 1)}}}},
|
||||
),
|
||||
UpdateStartPosition([]*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}),
|
||||
UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}),
|
||||
UpdateCheckPointOperator(1, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}),
|
||||
)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -757,7 +735,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
err = meta.UpdateSegmentsInfo(
|
||||
UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}),
|
||||
UpdateCheckPointOperator(1, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}),
|
||||
)
|
||||
assert.NoError(t, err)
|
||||
|
||||
|
@ -796,7 +774,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
err = meta.UpdateSegmentsInfo(
|
||||
UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 2, NumOfRows: 10}}),
|
||||
UpdateCheckPointOperator(1, []*datapb.CheckPoint{{SegmentID: 2, NumOfRows: 10}}),
|
||||
)
|
||||
|
||||
assert.NoError(t, err)
|
||||
|
@ -830,7 +808,7 @@ func TestUpdateSegmentsInfo(t *testing.T) {
|
|||
[]*datapb.FieldBinlog{{Binlogs: []*datapb.Binlog{{EntriesNum: 1, TimestampFrom: 100, TimestampTo: 200, LogSize: 1000, LogPath: getDeltaLogPath("deltalog", 1)}}}},
|
||||
),
|
||||
UpdateStartPosition([]*datapb.SegmentStartPosition{{SegmentID: 1, StartPosition: &msgpb.MsgPosition{MsgID: []byte{1, 2, 3}}}}),
|
||||
UpdateCheckPointOperator(1, false, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}),
|
||||
UpdateCheckPointOperator(1, []*datapb.CheckPoint{{SegmentID: 1, NumOfRows: 10}}),
|
||||
)
|
||||
|
||||
assert.Error(t, err)
|
||||
|
|
|
@ -22,61 +22,6 @@ func (_m *MockCluster) EXPECT() *MockCluster_Expecter {
|
|||
return &MockCluster_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// AddImportSegment provides a mock function with given fields: ctx, req
|
||||
func (_m *MockCluster) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
ret := _m.Called(ctx, req)
|
||||
|
||||
var r0 *datapb.AddImportSegmentResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)); ok {
|
||||
return rf(ctx, req)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.AddImportSegmentRequest) *datapb.AddImportSegmentResponse); ok {
|
||||
r0 = rf(ctx, req)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*datapb.AddImportSegmentResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.AddImportSegmentRequest) error); ok {
|
||||
r1 = rf(ctx, req)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockCluster_AddImportSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AddImportSegment'
|
||||
type MockCluster_AddImportSegment_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// AddImportSegment is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - req *datapb.AddImportSegmentRequest
|
||||
func (_e *MockCluster_Expecter) AddImportSegment(ctx interface{}, req interface{}) *MockCluster_AddImportSegment_Call {
|
||||
return &MockCluster_AddImportSegment_Call{Call: _e.mock.On("AddImportSegment", ctx, req)}
|
||||
}
|
||||
|
||||
func (_c *MockCluster_AddImportSegment_Call) Run(run func(ctx context.Context, req *datapb.AddImportSegmentRequest)) *MockCluster_AddImportSegment_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*datapb.AddImportSegmentRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCluster_AddImportSegment_Call) Return(_a0 *datapb.AddImportSegmentResponse, _a1 error) *MockCluster_AddImportSegment_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCluster_AddImportSegment_Call) RunAndReturn(run func(context.Context, *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)) *MockCluster_AddImportSegment_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Close provides a mock function with given fields:
|
||||
func (_m *MockCluster) Close() {
|
||||
_m.Called()
|
||||
|
@ -129,8 +74,8 @@ type MockCluster_DropImport_Call struct {
|
|||
}
|
||||
|
||||
// DropImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.DropImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.DropImportRequest
|
||||
func (_e *MockCluster_Expecter) DropImport(nodeID interface{}, in interface{}) *MockCluster_DropImport_Call {
|
||||
return &MockCluster_DropImport_Call{Call: _e.mock.On("DropImport", nodeID, in)}
|
||||
}
|
||||
|
@ -172,10 +117,10 @@ type MockCluster_Flush_Call struct {
|
|||
}
|
||||
|
||||
// Flush is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - channel string
|
||||
// - segments []*datapb.SegmentInfo
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - channel string
|
||||
// - segments []*datapb.SegmentInfo
|
||||
func (_e *MockCluster_Expecter) Flush(ctx interface{}, nodeID interface{}, channel interface{}, segments interface{}) *MockCluster_Flush_Call {
|
||||
return &MockCluster_Flush_Call{Call: _e.mock.On("Flush", ctx, nodeID, channel, segments)}
|
||||
}
|
||||
|
@ -217,10 +162,10 @@ type MockCluster_FlushChannels_Call struct {
|
|||
}
|
||||
|
||||
// FlushChannels is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - flushTs uint64
|
||||
// - channels []string
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - flushTs uint64
|
||||
// - channels []string
|
||||
func (_e *MockCluster_Expecter) FlushChannels(ctx interface{}, nodeID interface{}, flushTs interface{}, channels interface{}) *MockCluster_FlushChannels_Call {
|
||||
return &MockCluster_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, nodeID, flushTs, channels)}
|
||||
}
|
||||
|
@ -285,41 +230,6 @@ func (_c *MockCluster_GetSessions_Call) RunAndReturn(run func() []*Session) *Moc
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: ctx, nodeID, it
|
||||
func (_m *MockCluster) Import(ctx context.Context, nodeID int64, it *datapb.ImportTaskRequest) {
|
||||
_m.Called(ctx, nodeID, it)
|
||||
}
|
||||
|
||||
// MockCluster_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type MockCluster_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - it *datapb.ImportTaskRequest
|
||||
func (_e *MockCluster_Expecter) Import(ctx interface{}, nodeID interface{}, it interface{}) *MockCluster_Import_Call {
|
||||
return &MockCluster_Import_Call{Call: _e.mock.On("Import", ctx, nodeID, it)}
|
||||
}
|
||||
|
||||
func (_c *MockCluster_Import_Call) Run(run func(ctx context.Context, nodeID int64, it *datapb.ImportTaskRequest)) *MockCluster_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(int64), args[2].(*datapb.ImportTaskRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCluster_Import_Call) Return() *MockCluster_Import_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCluster_Import_Call) RunAndReturn(run func(context.Context, int64, *datapb.ImportTaskRequest)) *MockCluster_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ImportV2 provides a mock function with given fields: nodeID, in
|
||||
func (_m *MockCluster) ImportV2(nodeID int64, in *datapb.ImportRequest) error {
|
||||
ret := _m.Called(nodeID, in)
|
||||
|
@ -340,8 +250,8 @@ type MockCluster_ImportV2_Call struct {
|
|||
}
|
||||
|
||||
// ImportV2 is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.ImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.ImportRequest
|
||||
func (_e *MockCluster_Expecter) ImportV2(nodeID interface{}, in interface{}) *MockCluster_ImportV2_Call {
|
||||
return &MockCluster_ImportV2_Call{Call: _e.mock.On("ImportV2", nodeID, in)}
|
||||
}
|
||||
|
@ -383,8 +293,8 @@ type MockCluster_PreImport_Call struct {
|
|||
}
|
||||
|
||||
// PreImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.PreImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.PreImportRequest
|
||||
func (_e *MockCluster_Expecter) PreImport(nodeID interface{}, in interface{}) *MockCluster_PreImport_Call {
|
||||
return &MockCluster_PreImport_Call{Call: _e.mock.On("PreImport", nodeID, in)}
|
||||
}
|
||||
|
@ -438,8 +348,8 @@ type MockCluster_QueryImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryImportRequest
|
||||
func (_e *MockCluster_Expecter) QueryImport(nodeID interface{}, in interface{}) *MockCluster_QueryImport_Call {
|
||||
return &MockCluster_QueryImport_Call{Call: _e.mock.On("QueryImport", nodeID, in)}
|
||||
}
|
||||
|
@ -493,8 +403,8 @@ type MockCluster_QueryPreImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryPreImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryPreImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryPreImportRequest
|
||||
func (_e *MockCluster_Expecter) QueryPreImport(nodeID interface{}, in interface{}) *MockCluster_QueryPreImport_Call {
|
||||
return &MockCluster_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", nodeID, in)}
|
||||
}
|
||||
|
@ -536,7 +446,7 @@ type MockCluster_Register_Call struct {
|
|||
}
|
||||
|
||||
// Register is a helper method to define mock.On call
|
||||
// - node *NodeInfo
|
||||
// - node *NodeInfo
|
||||
func (_e *MockCluster_Expecter) Register(node interface{}) *MockCluster_Register_Call {
|
||||
return &MockCluster_Register_Call{Call: _e.mock.On("Register", node)}
|
||||
}
|
||||
|
@ -578,8 +488,8 @@ type MockCluster_Startup_Call struct {
|
|||
}
|
||||
|
||||
// Startup is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodes []*NodeInfo
|
||||
// - ctx context.Context
|
||||
// - nodes []*NodeInfo
|
||||
func (_e *MockCluster_Expecter) Startup(ctx interface{}, nodes interface{}) *MockCluster_Startup_Call {
|
||||
return &MockCluster_Startup_Call{Call: _e.mock.On("Startup", ctx, nodes)}
|
||||
}
|
||||
|
@ -621,7 +531,7 @@ type MockCluster_UnRegister_Call struct {
|
|||
}
|
||||
|
||||
// UnRegister is a helper method to define mock.On call
|
||||
// - node *NodeInfo
|
||||
// - node *NodeInfo
|
||||
func (_e *MockCluster_Expecter) UnRegister(node interface{}) *MockCluster_UnRegister_Call {
|
||||
return &MockCluster_UnRegister_Call{Call: _e.mock.On("UnRegister", node)}
|
||||
}
|
||||
|
@ -663,9 +573,9 @@ type MockCluster_Watch_Call struct {
|
|||
}
|
||||
|
||||
// Watch is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - ch string
|
||||
// - collectionID int64
|
||||
// - ctx context.Context
|
||||
// - ch string
|
||||
// - collectionID int64
|
||||
func (_e *MockCluster_Expecter) Watch(ctx interface{}, ch interface{}, collectionID interface{}) *MockCluster_Watch_Call {
|
||||
return &MockCluster_Watch_Call{Call: _e.mock.On("Watch", ctx, ch, collectionID)}
|
||||
}
|
||||
|
|
|
@ -24,62 +24,6 @@ func (_m *MockSessionManager) EXPECT() *MockSessionManager_Expecter {
|
|||
return &MockSessionManager_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// AddImportSegment provides a mock function with given fields: ctx, nodeID, req
|
||||
func (_m *MockSessionManager) AddImportSegment(ctx context.Context, nodeID int64, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
ret := _m.Called(ctx, nodeID, req)
|
||||
|
||||
var r0 *datapb.AddImportSegmentResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, int64, *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)); ok {
|
||||
return rf(ctx, nodeID, req)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, int64, *datapb.AddImportSegmentRequest) *datapb.AddImportSegmentResponse); ok {
|
||||
r0 = rf(ctx, nodeID, req)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*datapb.AddImportSegmentResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, int64, *datapb.AddImportSegmentRequest) error); ok {
|
||||
r1 = rf(ctx, nodeID, req)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockSessionManager_AddImportSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AddImportSegment'
|
||||
type MockSessionManager_AddImportSegment_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// AddImportSegment is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - req *datapb.AddImportSegmentRequest
|
||||
func (_e *MockSessionManager_Expecter) AddImportSegment(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_AddImportSegment_Call {
|
||||
return &MockSessionManager_AddImportSegment_Call{Call: _e.mock.On("AddImportSegment", ctx, nodeID, req)}
|
||||
}
|
||||
|
||||
func (_c *MockSessionManager_AddImportSegment_Call) Run(run func(ctx context.Context, nodeID int64, req *datapb.AddImportSegmentRequest)) *MockSessionManager_AddImportSegment_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(int64), args[2].(*datapb.AddImportSegmentRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockSessionManager_AddImportSegment_Call) Return(_a0 *datapb.AddImportSegmentResponse, _a1 error) *MockSessionManager_AddImportSegment_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockSessionManager_AddImportSegment_Call) RunAndReturn(run func(context.Context, int64, *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)) *MockSessionManager_AddImportSegment_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// AddSession provides a mock function with given fields: node
|
||||
func (_m *MockSessionManager) AddSession(node *NodeInfo) {
|
||||
_m.Called(node)
|
||||
|
@ -91,7 +35,7 @@ type MockSessionManager_AddSession_Call struct {
|
|||
}
|
||||
|
||||
// AddSession is a helper method to define mock.On call
|
||||
// - node *NodeInfo
|
||||
// - node *NodeInfo
|
||||
func (_e *MockSessionManager_Expecter) AddSession(node interface{}) *MockSessionManager_AddSession_Call {
|
||||
return &MockSessionManager_AddSession_Call{Call: _e.mock.On("AddSession", node)}
|
||||
}
|
||||
|
@ -145,9 +89,9 @@ type MockSessionManager_CheckChannelOperationProgress_Call struct {
|
|||
}
|
||||
|
||||
// CheckChannelOperationProgress is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - info *datapb.ChannelWatchInfo
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - info *datapb.ChannelWatchInfo
|
||||
func (_e *MockSessionManager_Expecter) CheckChannelOperationProgress(ctx interface{}, nodeID interface{}, info interface{}) *MockSessionManager_CheckChannelOperationProgress_Call {
|
||||
return &MockSessionManager_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", ctx, nodeID, info)}
|
||||
}
|
||||
|
@ -189,7 +133,7 @@ type MockSessionManager_CheckHealth_Call struct {
|
|||
}
|
||||
|
||||
// CheckHealth is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - ctx context.Context
|
||||
func (_e *MockSessionManager_Expecter) CheckHealth(ctx interface{}) *MockSessionManager_CheckHealth_Call {
|
||||
return &MockSessionManager_CheckHealth_Call{Call: _e.mock.On("CheckHealth", ctx)}
|
||||
}
|
||||
|
@ -263,9 +207,9 @@ type MockSessionManager_Compaction_Call struct {
|
|||
}
|
||||
|
||||
// Compaction is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - plan *datapb.CompactionPlan
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - plan *datapb.CompactionPlan
|
||||
func (_e *MockSessionManager_Expecter) Compaction(ctx interface{}, nodeID interface{}, plan interface{}) *MockSessionManager_Compaction_Call {
|
||||
return &MockSessionManager_Compaction_Call{Call: _e.mock.On("Compaction", ctx, nodeID, plan)}
|
||||
}
|
||||
|
@ -298,7 +242,7 @@ type MockSessionManager_DeleteSession_Call struct {
|
|||
}
|
||||
|
||||
// DeleteSession is a helper method to define mock.On call
|
||||
// - node *NodeInfo
|
||||
// - node *NodeInfo
|
||||
func (_e *MockSessionManager_Expecter) DeleteSession(node interface{}) *MockSessionManager_DeleteSession_Call {
|
||||
return &MockSessionManager_DeleteSession_Call{Call: _e.mock.On("DeleteSession", node)}
|
||||
}
|
||||
|
@ -340,8 +284,8 @@ type MockSessionManager_DropImport_Call struct {
|
|||
}
|
||||
|
||||
// DropImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.DropImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.DropImportRequest
|
||||
func (_e *MockSessionManager_Expecter) DropImport(nodeID interface{}, in interface{}) *MockSessionManager_DropImport_Call {
|
||||
return &MockSessionManager_DropImport_Call{Call: _e.mock.On("DropImport", nodeID, in)}
|
||||
}
|
||||
|
@ -374,9 +318,9 @@ type MockSessionManager_Flush_Call struct {
|
|||
}
|
||||
|
||||
// Flush is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - req *datapb.FlushSegmentsRequest
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - req *datapb.FlushSegmentsRequest
|
||||
func (_e *MockSessionManager_Expecter) Flush(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_Flush_Call {
|
||||
return &MockSessionManager_Flush_Call{Call: _e.mock.On("Flush", ctx, nodeID, req)}
|
||||
}
|
||||
|
@ -418,9 +362,9 @@ type MockSessionManager_FlushChannels_Call struct {
|
|||
}
|
||||
|
||||
// FlushChannels is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - req *datapb.FlushChannelsRequest
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - req *datapb.FlushChannelsRequest
|
||||
func (_e *MockSessionManager_Expecter) FlushChannels(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_FlushChannels_Call {
|
||||
return &MockSessionManager_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, nodeID, req)}
|
||||
}
|
||||
|
@ -581,41 +525,6 @@ func (_c *MockSessionManager_GetSessions_Call) RunAndReturn(run func() []*Sessio
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: ctx, nodeID, itr
|
||||
func (_m *MockSessionManager) Import(ctx context.Context, nodeID int64, itr *datapb.ImportTaskRequest) {
|
||||
_m.Called(ctx, nodeID, itr)
|
||||
}
|
||||
|
||||
// MockSessionManager_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type MockSessionManager_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - itr *datapb.ImportTaskRequest
|
||||
func (_e *MockSessionManager_Expecter) Import(ctx interface{}, nodeID interface{}, itr interface{}) *MockSessionManager_Import_Call {
|
||||
return &MockSessionManager_Import_Call{Call: _e.mock.On("Import", ctx, nodeID, itr)}
|
||||
}
|
||||
|
||||
func (_c *MockSessionManager_Import_Call) Run(run func(ctx context.Context, nodeID int64, itr *datapb.ImportTaskRequest)) *MockSessionManager_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(int64), args[2].(*datapb.ImportTaskRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockSessionManager_Import_Call) Return() *MockSessionManager_Import_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockSessionManager_Import_Call) RunAndReturn(run func(context.Context, int64, *datapb.ImportTaskRequest)) *MockSessionManager_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ImportV2 provides a mock function with given fields: nodeID, in
|
||||
func (_m *MockSessionManager) ImportV2(nodeID int64, in *datapb.ImportRequest) error {
|
||||
ret := _m.Called(nodeID, in)
|
||||
|
@ -636,8 +545,8 @@ type MockSessionManager_ImportV2_Call struct {
|
|||
}
|
||||
|
||||
// ImportV2 is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.ImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.ImportRequest
|
||||
func (_e *MockSessionManager_Expecter) ImportV2(nodeID interface{}, in interface{}) *MockSessionManager_ImportV2_Call {
|
||||
return &MockSessionManager_ImportV2_Call{Call: _e.mock.On("ImportV2", nodeID, in)}
|
||||
}
|
||||
|
@ -679,9 +588,9 @@ type MockSessionManager_NotifyChannelOperation_Call struct {
|
|||
}
|
||||
|
||||
// NotifyChannelOperation is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - req *datapb.ChannelOperationsRequest
|
||||
// - ctx context.Context
|
||||
// - nodeID int64
|
||||
// - req *datapb.ChannelOperationsRequest
|
||||
func (_e *MockSessionManager_Expecter) NotifyChannelOperation(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_NotifyChannelOperation_Call {
|
||||
return &MockSessionManager_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", ctx, nodeID, req)}
|
||||
}
|
||||
|
@ -723,8 +632,8 @@ type MockSessionManager_PreImport_Call struct {
|
|||
}
|
||||
|
||||
// PreImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.PreImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.PreImportRequest
|
||||
func (_e *MockSessionManager_Expecter) PreImport(nodeID interface{}, in interface{}) *MockSessionManager_PreImport_Call {
|
||||
return &MockSessionManager_PreImport_Call{Call: _e.mock.On("PreImport", nodeID, in)}
|
||||
}
|
||||
|
@ -778,8 +687,8 @@ type MockSessionManager_QueryImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryImportRequest
|
||||
func (_e *MockSessionManager_Expecter) QueryImport(nodeID interface{}, in interface{}) *MockSessionManager_QueryImport_Call {
|
||||
return &MockSessionManager_QueryImport_Call{Call: _e.mock.On("QueryImport", nodeID, in)}
|
||||
}
|
||||
|
@ -833,8 +742,8 @@ type MockSessionManager_QueryPreImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryPreImport is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryPreImportRequest
|
||||
// - nodeID int64
|
||||
// - in *datapb.QueryPreImportRequest
|
||||
func (_e *MockSessionManager_Expecter) QueryPreImport(nodeID interface{}, in interface{}) *MockSessionManager_QueryPreImport_Call {
|
||||
return &MockSessionManager_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", nodeID, in)}
|
||||
}
|
||||
|
@ -876,8 +785,8 @@ type MockSessionManager_SyncSegments_Call struct {
|
|||
}
|
||||
|
||||
// SyncSegments is a helper method to define mock.On call
|
||||
// - nodeID int64
|
||||
// - req *datapb.SyncSegmentsRequest
|
||||
// - nodeID int64
|
||||
// - req *datapb.SyncSegmentsRequest
|
||||
func (_e *MockSessionManager_Expecter) SyncSegments(nodeID interface{}, req interface{}) *MockSessionManager_SyncSegments_Call {
|
||||
return &MockSessionManager_SyncSegments_Call{Call: _e.mock.On("SyncSegments", nodeID, req)}
|
||||
}
|
||||
|
|
|
@ -169,12 +169,11 @@ func newTestSchema() *schemapb.CollectionSchema {
|
|||
}
|
||||
|
||||
type mockDataNodeClient struct {
|
||||
id int64
|
||||
state commonpb.StateCode
|
||||
ch chan interface{}
|
||||
compactionStateResp *datapb.CompactionStateResponse
|
||||
addImportSegmentResp *datapb.AddImportSegmentResponse
|
||||
compactionResp *commonpb.Status
|
||||
id int64
|
||||
state commonpb.StateCode
|
||||
ch chan interface{}
|
||||
compactionStateResp *datapb.CompactionStateResponse
|
||||
compactionResp *commonpb.Status
|
||||
}
|
||||
|
||||
func newMockDataNodeClient(id int64, ch chan interface{}) (*mockDataNodeClient, error) {
|
||||
|
@ -182,9 +181,6 @@ func newMockDataNodeClient(id int64, ch chan interface{}) (*mockDataNodeClient,
|
|||
id: id,
|
||||
state: commonpb.StateCode_Initializing,
|
||||
ch: ch,
|
||||
addImportSegmentResp: &datapb.AddImportSegmentResponse{
|
||||
Status: merr.Success(),
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
|
@ -287,14 +283,6 @@ func (c *mockDataNodeClient) GetCompactionState(ctx context.Context, req *datapb
|
|||
return c.compactionStateResp, nil
|
||||
}
|
||||
|
||||
func (c *mockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
|
||||
}
|
||||
|
||||
func (c *mockDataNodeClient) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest, opts ...grpc.CallOption) (*datapb.AddImportSegmentResponse, error) {
|
||||
return c.addImportSegmentResp, nil
|
||||
}
|
||||
|
||||
func (c *mockDataNodeClient) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
|
||||
}
|
||||
|
@ -598,24 +586,6 @@ func (m *mockRootCoordClient) GetMetrics(ctx context.Context, req *milvuspb.GetM
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (m *mockRootCoordClient) Import(ctx context.Context, req *milvuspb.ImportRequest, opts ...grpc.CallOption) (*milvuspb.ImportResponse, error) {
|
||||
panic("not implemented") // TODO: Implement
|
||||
}
|
||||
|
||||
// Check import task state from datanode
|
||||
func (m *mockRootCoordClient) GetImportState(ctx context.Context, req *milvuspb.GetImportStateRequest, opts ...grpc.CallOption) (*milvuspb.GetImportStateResponse, error) {
|
||||
panic("not implemented") // TODO: Implement
|
||||
}
|
||||
|
||||
// Returns id array of all import tasks
|
||||
func (m *mockRootCoordClient) ListImportTasks(ctx context.Context, in *milvuspb.ListImportTasksRequest, opts ...grpc.CallOption) (*milvuspb.ListImportTasksResponse, error) {
|
||||
panic("not implemented") // TODO: Implement
|
||||
}
|
||||
|
||||
func (m *mockRootCoordClient) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
type mockCompactionTrigger struct {
|
||||
methods map[string]interface{}
|
||||
}
|
||||
|
|
|
@ -147,13 +147,6 @@ func (s *SegmentsInfo) SetState(segmentID UniqueID, state commonpb.SegmentState)
|
|||
}
|
||||
}
|
||||
|
||||
// SetIsImporting sets the import status for a segment.
|
||||
func (s *SegmentsInfo) SetIsImporting(segmentID UniqueID, isImporting bool) {
|
||||
if segment, ok := s.segments[segmentID]; ok {
|
||||
s.segments[segmentID] = segment.Clone(SetIsImporting(isImporting))
|
||||
}
|
||||
}
|
||||
|
||||
// SetDmlPosition sets DmlPosition info (checkpoint for recovery) for SegmentInfo with provided segmentID
|
||||
// if SegmentInfo not found, do nothing
|
||||
func (s *SegmentsInfo) SetDmlPosition(segmentID UniqueID, pos *msgpb.MsgPosition) {
|
||||
|
@ -287,13 +280,6 @@ func SetState(state commonpb.SegmentState) SegmentInfoOption {
|
|||
}
|
||||
}
|
||||
|
||||
// SetIsImporting is the option to set import state for segment info.
|
||||
func SetIsImporting(isImporting bool) SegmentInfoOption {
|
||||
return func(segment *SegmentInfo) {
|
||||
segment.IsImporting = isImporting
|
||||
}
|
||||
}
|
||||
|
||||
// SetDmlPosition is the option to set dml position for segment info
|
||||
func SetDmlPosition(pos *msgpb.MsgPosition) SegmentInfoOption {
|
||||
return func(segment *SegmentInfo) {
|
||||
|
|
|
@ -75,9 +75,6 @@ type Manager interface {
|
|||
|
||||
// AllocSegment allocates rows and record the allocation.
|
||||
AllocSegment(ctx context.Context, collectionID, partitionID UniqueID, channelName string, requestRows int64) ([]*Allocation, error)
|
||||
// allocSegmentForImport allocates one segment allocation for bulk insert.
|
||||
// TODO: Remove this method and AllocSegment() above instead.
|
||||
allocSegmentForImport(ctx context.Context, collectionID, partitionID UniqueID, channelName string, requestRows int64, taskID int64) (*Allocation, error)
|
||||
AllocImportSegment(ctx context.Context, taskID int64, collectionID UniqueID, partitionID UniqueID, channelName string) (*SegmentInfo, error)
|
||||
// DropSegment drops the segment from manager.
|
||||
DropSegment(ctx context.Context, segmentID UniqueID)
|
||||
|
@ -244,7 +241,7 @@ func (s *SegmentManager) maybeResetLastExpireForSegments() error {
|
|||
if len(s.segments) > 0 {
|
||||
var latestTs uint64
|
||||
allocateErr := retry.Do(context.Background(), func() error {
|
||||
ts, tryErr := s.genExpireTs(context.Background(), false)
|
||||
ts, tryErr := s.genExpireTs(context.Background())
|
||||
log.Warn("failed to get ts from rootCoord for globalLastExpire", zap.Error(tryErr))
|
||||
if tryErr != nil {
|
||||
return tryErr
|
||||
|
@ -302,7 +299,7 @@ func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID
|
|||
requestRows, int64(maxCountPerSegment), datapb.SegmentLevel_L1)
|
||||
|
||||
// create new segments and add allocations
|
||||
expireTs, err := s.genExpireTs(ctx, false)
|
||||
expireTs, err := s.genExpireTs(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -331,37 +328,6 @@ func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID
|
|||
return allocations, nil
|
||||
}
|
||||
|
||||
// allocSegmentForImport allocates one segment allocation for bulk insert.
|
||||
func (s *SegmentManager) allocSegmentForImport(ctx context.Context, collectionID UniqueID,
|
||||
partitionID UniqueID, channelName string, requestRows int64, importTaskID int64,
|
||||
) (*Allocation, error) {
|
||||
_, sp := otel.Tracer(typeutil.DataCoordRole).Start(ctx, "Alloc-ImportSegment")
|
||||
defer sp.End()
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
|
||||
// Init allocation.
|
||||
allocation := getAllocation(requestRows)
|
||||
// Create new segments and add allocations to meta.
|
||||
// To avoid mixing up with growing segments, the segment state is "Importing"
|
||||
expireTs, err := s.genExpireTs(ctx, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName, commonpb.SegmentState_Importing, datapb.SegmentLevel_L1)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
allocation.ExpireTime = expireTs
|
||||
allocation.SegmentID = segment.GetID()
|
||||
if err := s.meta.AddAllocation(segment.GetID(), allocation); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return allocation, nil
|
||||
}
|
||||
|
||||
func satisfy(segment *SegmentInfo, collectionID, partitionID UniqueID, channel string) bool {
|
||||
return segment.GetCollectionID() == collectionID && segment.GetPartitionID() == partitionID &&
|
||||
segment.GetInsertChannel() == channel
|
||||
|
@ -371,17 +337,13 @@ func isGrowing(segment *SegmentInfo) bool {
|
|||
return segment.GetState() == commonpb.SegmentState_Growing
|
||||
}
|
||||
|
||||
func (s *SegmentManager) genExpireTs(ctx context.Context, isImported bool) (Timestamp, error) {
|
||||
func (s *SegmentManager) genExpireTs(ctx context.Context) (Timestamp, error) {
|
||||
ts, err := s.allocator.allocTimestamp(ctx)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
physicalTs, logicalTs := tsoutil.ParseTS(ts)
|
||||
expirePhysicalTs := physicalTs.Add(time.Duration(Params.DataCoordCfg.SegAssignmentExpiration.GetAsFloat()) * time.Millisecond)
|
||||
// for imported segment, clean up ImportTaskExpiration
|
||||
if isImported {
|
||||
expirePhysicalTs = physicalTs.Add(time.Duration(Params.RootCoordCfg.ImportTaskExpiration.GetAsFloat()) * time.Second)
|
||||
}
|
||||
expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs))
|
||||
return expireTs, nil
|
||||
}
|
||||
|
@ -466,9 +428,6 @@ func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID Unique
|
|||
Level: level,
|
||||
LastExpireTime: 0,
|
||||
}
|
||||
if segmentState == commonpb.SegmentState_Importing {
|
||||
segmentInfo.IsImporting = true
|
||||
}
|
||||
segment := NewSegmentInfo(segmentInfo)
|
||||
if err := s.meta.AddSegment(ctx, segment); err != nil {
|
||||
log.Error("failed to add segment to DataCoord", zap.Error(err))
|
||||
|
|
|
@ -243,47 +243,6 @@ func TestLastExpireReset(t *testing.T) {
|
|||
assert.Equal(t, segmentID3, newAlloc[0].SegmentID) // segment3 still can be used to allocate
|
||||
}
|
||||
|
||||
func TestAllocSegmentForImport(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
paramtable.Init()
|
||||
mockAllocator := newMockAllocator()
|
||||
meta, err := newMemoryMeta()
|
||||
assert.NoError(t, err)
|
||||
segmentManager, _ := newSegmentManager(meta, mockAllocator)
|
||||
|
||||
schema := newTestSchema()
|
||||
collID, err := mockAllocator.allocID(ctx)
|
||||
assert.NoError(t, err)
|
||||
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
|
||||
|
||||
t.Run("normal allocation", func(t *testing.T) {
|
||||
allocation, err := segmentManager.allocSegmentForImport(ctx, collID, 100, "c1", 100, 0)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, allocation)
|
||||
assert.EqualValues(t, 100, allocation.NumOfRows)
|
||||
assert.NotEqualValues(t, 0, allocation.SegmentID)
|
||||
assert.NotEqualValues(t, 0, allocation.ExpireTime)
|
||||
})
|
||||
|
||||
t.Run("allocation fails 1", func(t *testing.T) {
|
||||
failsAllocator := &FailsAllocator{
|
||||
allocTsSucceed: true,
|
||||
}
|
||||
segmentManager, _ := newSegmentManager(meta, failsAllocator)
|
||||
_, err := segmentManager.allocSegmentForImport(ctx, collID, 100, "c1", 100, 0)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("allocation fails 2", func(t *testing.T) {
|
||||
failsAllocator := &FailsAllocator{
|
||||
allocIDSucceed: true,
|
||||
}
|
||||
segmentManager, _ := newSegmentManager(meta, failsAllocator)
|
||||
_, err := segmentManager.allocSegmentForImport(ctx, collID, 100, "c1", 100, 0)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func TestSegmentManager_AllocImportSegment(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
mockErr := errors.New("mock error")
|
||||
|
@ -524,36 +483,6 @@ func TestExpireAllocation(t *testing.T) {
|
|||
assert.EqualValues(t, 0, len(segment.allocations))
|
||||
}
|
||||
|
||||
func TestCleanExpiredBulkloadSegment(t *testing.T) {
|
||||
t.Run("expiredBulkloadSegment", func(t *testing.T) {
|
||||
paramtable.Init()
|
||||
mockAllocator := newMockAllocator()
|
||||
meta, err := newMemoryMeta()
|
||||
assert.NoError(t, err)
|
||||
|
||||
schema := newTestSchema()
|
||||
collID, err := mockAllocator.allocID(context.Background())
|
||||
assert.NoError(t, err)
|
||||
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
|
||||
segmentManager, _ := newSegmentManager(meta, mockAllocator)
|
||||
allocation, err := segmentManager.allocSegmentForImport(context.TODO(), collID, 0, "c1", 2, 1)
|
||||
assert.NoError(t, err)
|
||||
|
||||
ids, err := segmentManager.GetFlushableSegments(context.TODO(), "c1", allocation.ExpireTime)
|
||||
assert.NoError(t, err)
|
||||
assert.EqualValues(t, len(ids), 0)
|
||||
|
||||
assert.EqualValues(t, len(segmentManager.segments), 1)
|
||||
|
||||
ids, err = segmentManager.GetFlushableSegments(context.TODO(), "c1", allocation.ExpireTime+1)
|
||||
assert.NoError(t, err)
|
||||
assert.Empty(t, ids)
|
||||
assert.EqualValues(t, len(ids), 0)
|
||||
|
||||
assert.EqualValues(t, len(segmentManager.segments), 0)
|
||||
})
|
||||
}
|
||||
|
||||
func TestGetFlushableSegments(t *testing.T) {
|
||||
t.Run("get flushable segments between small interval", func(t *testing.T) {
|
||||
paramtable.Init()
|
||||
|
@ -952,41 +881,3 @@ func TestSegmentManager_DropSegmentsOfChannel(t *testing.T) {
|
|||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestSegmentManager_FlushImportSegments(t *testing.T) {
|
||||
alloc := NewNMockAllocator(t)
|
||||
alloc.EXPECT().allocID(mock.Anything).Return(0, nil)
|
||||
alloc.EXPECT().allocTimestamp(mock.Anything).Return(1000, nil)
|
||||
mm, err := newMemoryMeta()
|
||||
assert.NoError(t, err)
|
||||
|
||||
schema := newTestSchema()
|
||||
assert.NoError(t, err)
|
||||
mm.AddCollection(&collectionInfo{ID: collID, Schema: schema})
|
||||
segmentManager, _ := newSegmentManager(mm, alloc)
|
||||
allocation, err := segmentManager.allocSegmentForImport(context.TODO(), collID, 1, "c1", 2, 3)
|
||||
assert.NoError(t, err)
|
||||
|
||||
segmentID := allocation.SegmentID
|
||||
segment := mm.GetSegment(segmentID)
|
||||
assert.Equal(t, commonpb.SegmentState_Importing, segment.GetState())
|
||||
|
||||
// normal
|
||||
err = segmentManager.FlushImportSegments(context.TODO(), collID, []UniqueID{segmentID})
|
||||
assert.NoError(t, err)
|
||||
segment = mm.GetSegment(segmentID)
|
||||
assert.Equal(t, commonpb.SegmentState_Flushed, segment.GetState())
|
||||
|
||||
// no segment
|
||||
err = segmentManager.FlushImportSegments(context.TODO(), collID, []UniqueID{6})
|
||||
assert.NoError(t, err)
|
||||
|
||||
// collection not match
|
||||
mm.AddCollection(&collectionInfo{ID: 6, Schema: schema})
|
||||
allocation, err = segmentManager.allocSegmentForImport(context.TODO(), 6, 1, "c1", 2, 3)
|
||||
assert.NoError(t, err)
|
||||
err = segmentManager.FlushImportSegments(context.TODO(), collID, []UniqueID{allocation.SegmentID})
|
||||
assert.NoError(t, err)
|
||||
segment = mm.GetSegment(allocation.SegmentID)
|
||||
assert.Equal(t, commonpb.SegmentState_Importing, segment.GetState())
|
||||
}
|
||||
|
|
|
@ -48,7 +48,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
grpcmock "github.com/milvus-io/milvus/internal/util/mock"
|
||||
|
@ -108,13 +107,6 @@ func (r *mockRootCoord) DescribeCollectionInternal(ctx context.Context, req *mil
|
|||
return r.RootCoordClient.DescribeCollection(ctx, req)
|
||||
}
|
||||
|
||||
func (r *mockRootCoord) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "something bad",
|
||||
}, nil
|
||||
}
|
||||
|
||||
// func TestGetComponentStates(t *testing.T) {
|
||||
// svr := newTestServer(t)
|
||||
// defer closeTestServer(t, svr)
|
||||
|
@ -3037,109 +3029,6 @@ func TestDataCoordServer_SetSegmentState(t *testing.T) {
|
|||
})
|
||||
}
|
||||
|
||||
func TestDataCoord_Import(t *testing.T) {
|
||||
storage.CheckBucketRetryAttempts = 2
|
||||
|
||||
t.Run("normal case", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
svr.sessionManager.AddSession(&NodeInfo{
|
||||
NodeID: 0,
|
||||
Address: "localhost:8080",
|
||||
})
|
||||
err := svr.channelManager.AddNode(0)
|
||||
assert.NoError(t, err)
|
||||
err = svr.channelManager.Watch(svr.ctx, &channelMeta{Name: "ch1", CollectionID: 0})
|
||||
assert.NoError(t, err)
|
||||
|
||||
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.GetErrorCode())
|
||||
closeTestServer(t, svr)
|
||||
})
|
||||
|
||||
t.Run("no free node", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
|
||||
err := svr.channelManager.AddNode(0)
|
||||
assert.NoError(t, err)
|
||||
err = svr.channelManager.Watch(svr.ctx, &channelMeta{Name: "ch1", CollectionID: 0})
|
||||
assert.NoError(t, err)
|
||||
|
||||
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
},
|
||||
WorkingNodes: []int64{0},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.EqualValues(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode())
|
||||
closeTestServer(t, svr)
|
||||
})
|
||||
|
||||
t.Run("no datanode available", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
Params.Save("minio.address", "minio:9000")
|
||||
defer Params.Reset("minio.address")
|
||||
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.EqualValues(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode())
|
||||
closeTestServer(t, svr)
|
||||
})
|
||||
|
||||
t.Run("with closed server", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
closeTestServer(t, svr)
|
||||
|
||||
resp, err := svr.Import(svr.ctx, &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.ErrorIs(t, merr.Error(resp.GetStatus()), merr.ErrServiceNotReady)
|
||||
})
|
||||
|
||||
t.Run("test update segment stat", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
|
||||
status, err := svr.UpdateSegmentStatistics(context.TODO(), &datapb.UpdateSegmentStatisticsRequest{
|
||||
Stats: []*commonpb.SegmentStats{{
|
||||
SegmentID: 100,
|
||||
NumRows: int64(1),
|
||||
}},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode())
|
||||
closeTestServer(t, svr)
|
||||
})
|
||||
|
||||
t.Run("test update segment stat w/ closed server", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
closeTestServer(t, svr)
|
||||
|
||||
status, err := svr.UpdateSegmentStatistics(context.TODO(), &datapb.UpdateSegmentStatisticsRequest{
|
||||
Stats: []*commonpb.SegmentStats{{
|
||||
SegmentID: 100,
|
||||
NumRows: int64(1),
|
||||
}},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.ErrorIs(t, merr.Error(status), merr.ErrServiceNotReady)
|
||||
})
|
||||
}
|
||||
|
||||
func TestDataCoord_SegmentStatistics(t *testing.T) {
|
||||
t.Run("test update imported segment stat", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
|
@ -3196,120 +3085,6 @@ func TestDataCoord_SegmentStatistics(t *testing.T) {
|
|||
})
|
||||
}
|
||||
|
||||
func TestDataCoord_SaveImportSegment(t *testing.T) {
|
||||
t.Run("test add segment", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
defer closeTestServer(t, svr)
|
||||
svr.meta.AddCollection(&collectionInfo{
|
||||
ID: 100,
|
||||
})
|
||||
seg := buildSegment(100, 100, 100, "ch1", false)
|
||||
svr.meta.AddSegment(context.TODO(), seg)
|
||||
svr.sessionManager.AddSession(&NodeInfo{
|
||||
NodeID: 110,
|
||||
Address: "localhost:8080",
|
||||
})
|
||||
err := svr.channelManager.AddNode(110)
|
||||
assert.NoError(t, err)
|
||||
err = svr.channelManager.Watch(context.TODO(), &channelMeta{Name: "ch1", CollectionID: 100})
|
||||
assert.NoError(t, err)
|
||||
err = svr.meta.UpdateChannelCheckpoint("ch1", &msgpb.MsgPosition{
|
||||
ChannelName: "ch1",
|
||||
MsgID: []byte{0, 0, 0, 0, 0, 0, 0, 0},
|
||||
Timestamp: 1000,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
|
||||
status, err := svr.SaveImportSegment(context.TODO(), &datapb.SaveImportSegmentRequest{
|
||||
SegmentId: 100,
|
||||
ChannelName: "ch1",
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
RowNum: int64(1),
|
||||
SaveBinlogPathReq: &datapb.SaveBinlogPathsRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
SourceID: paramtable.GetNodeID(),
|
||||
},
|
||||
SegmentID: 100,
|
||||
CollectionID: 100,
|
||||
Importing: true,
|
||||
StartPositions: []*datapb.SegmentStartPosition{
|
||||
{
|
||||
StartPosition: &msgpb.MsgPosition{
|
||||
ChannelName: "ch1",
|
||||
Timestamp: 1,
|
||||
},
|
||||
SegmentID: 100,
|
||||
},
|
||||
},
|
||||
CheckPoints: []*datapb.CheckPoint{
|
||||
{
|
||||
SegmentID: 100,
|
||||
Position: &msgpb.MsgPosition{
|
||||
ChannelName: "ch1",
|
||||
Timestamp: 1,
|
||||
},
|
||||
NumOfRows: int64(1),
|
||||
},
|
||||
},
|
||||
},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("test add segment w/ bad channelName", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
defer closeTestServer(t, svr)
|
||||
|
||||
err := svr.channelManager.AddNode(110)
|
||||
assert.NoError(t, err)
|
||||
err = svr.channelManager.Watch(context.TODO(), &channelMeta{Name: "ch1", CollectionID: 100})
|
||||
assert.NoError(t, err)
|
||||
|
||||
status, err := svr.SaveImportSegment(context.TODO(), &datapb.SaveImportSegmentRequest{
|
||||
SegmentId: 100,
|
||||
ChannelName: "non-channel",
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
RowNum: int64(1),
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("test add segment w/ closed server", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
closeTestServer(t, svr)
|
||||
|
||||
status, err := svr.SaveImportSegment(context.TODO(), &datapb.SaveImportSegmentRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.ErrorIs(t, merr.Error(status), merr.ErrServiceNotReady)
|
||||
})
|
||||
}
|
||||
|
||||
func TestDataCoord_UnsetIsImportingState(t *testing.T) {
|
||||
t.Run("normal case", func(t *testing.T) {
|
||||
svr := newTestServer(t, nil)
|
||||
defer closeTestServer(t, svr)
|
||||
seg := buildSegment(100, 100, 100, "ch1", false)
|
||||
svr.meta.AddSegment(context.TODO(), seg)
|
||||
|
||||
status, err := svr.UnsetIsImportingState(context.Background(), &datapb.UnsetIsImportingStateRequest{
|
||||
SegmentIds: []int64{100},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode())
|
||||
|
||||
// Trying to unset state of a segment that does not exist.
|
||||
status, err = svr.UnsetIsImportingState(context.Background(), &datapb.UnsetIsImportingStateRequest{
|
||||
SegmentIds: []int64{999},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode())
|
||||
})
|
||||
}
|
||||
|
||||
func TestDataCoordServer_UpdateChannelCheckpoint(t *testing.T) {
|
||||
mockVChannel := "fake-by-dev-rootcoord-dml-1-testchannelcp-v0"
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
|
@ -41,7 +40,6 @@ import (
|
|||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
||||
|
@ -66,35 +64,13 @@ func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetSt
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (s *Server) flushForImport(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
|
||||
err := s.segmentManager.FlushImportSegments(ctx, req.GetCollectionID(), req.GetSegmentIDs())
|
||||
if err != nil {
|
||||
return &datapb.FlushResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
// To expedite the process of index building.
|
||||
for _, segmentID := range req.GetSegmentIDs() {
|
||||
select {
|
||||
case s.buildIndexCh <- segmentID:
|
||||
default:
|
||||
}
|
||||
}
|
||||
log.Info("flush for import done", zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Int64s("segmentIDs", req.GetSegmentIDs()))
|
||||
return &datapb.FlushResponse{
|
||||
Status: merr.Success(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Flush notify segment to flush
|
||||
// this api only guarantees all the segments requested is sealed
|
||||
// these segments will be flushed only after the Flush policy is fulfilled
|
||||
func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.Int64("dbID", req.GetDbID()),
|
||||
zap.Int64("collectionID", req.GetCollectionID()),
|
||||
zap.Bool("isImporting", req.GetIsImport()))
|
||||
zap.Int64("collectionID", req.GetCollectionID()))
|
||||
log.Info("receive flush request")
|
||||
ctx, sp := otel.Tracer(typeutil.DataCoordRole).Start(ctx, "DataCoord-Flush")
|
||||
defer sp.End()
|
||||
|
@ -105,10 +81,6 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F
|
|||
}, nil
|
||||
}
|
||||
|
||||
if req.GetIsImport() {
|
||||
return s.flushForImport(ctx, req)
|
||||
}
|
||||
|
||||
// generate a timestamp timeOfSeal, all data before timeOfSeal is guaranteed to be sealed or flushed
|
||||
ts, err := s.allocator.allocTimestamp(ctx)
|
||||
if err != nil {
|
||||
|
@ -207,8 +179,6 @@ func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
|
|||
zap.Int64("partitionID", r.GetPartitionID()),
|
||||
zap.String("channelName", r.GetChannelName()),
|
||||
zap.Uint32("count", r.GetCount()),
|
||||
zap.Bool("isImport", r.GetIsImport()),
|
||||
zap.Int64("import task ID", r.GetImportTaskID()),
|
||||
zap.String("segment level", r.GetLevel().String()),
|
||||
)
|
||||
|
||||
|
@ -222,25 +192,12 @@ func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
|
|||
// Add the channel to cluster for watching.
|
||||
s.cluster.Watch(ctx, r.ChannelName, r.CollectionID)
|
||||
|
||||
segmentAllocations := make([]*Allocation, 0)
|
||||
if r.GetIsImport() {
|
||||
// Have segment manager allocate and return the segment allocation info.
|
||||
segAlloc, err := s.segmentManager.allocSegmentForImport(ctx,
|
||||
r.GetCollectionID(), r.GetPartitionID(), r.GetChannelName(), int64(r.GetCount()), r.GetImportTaskID())
|
||||
if err != nil {
|
||||
log.Warn("failed to alloc segment for import", zap.Any("request", r), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
segmentAllocations = append(segmentAllocations, segAlloc)
|
||||
} else {
|
||||
// Have segment manager allocate and return the segment allocation info.
|
||||
segAlloc, err := s.segmentManager.AllocSegment(ctx,
|
||||
r.CollectionID, r.PartitionID, r.ChannelName, int64(r.Count))
|
||||
if err != nil {
|
||||
log.Warn("failed to alloc segment", zap.Any("request", r), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
segmentAllocations = append(segmentAllocations, segAlloc...)
|
||||
// Have segment manager allocate and return the segment allocation info.
|
||||
segmentAllocations, err := s.segmentManager.AllocSegment(ctx,
|
||||
r.CollectionID, r.PartitionID, r.ChannelName, int64(r.Count))
|
||||
if err != nil {
|
||||
log.Warn("failed to alloc segment", zap.Any("request", r), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
log.Info("success to assign segments", zap.Int64("collectionID", r.GetCollectionID()), zap.Any("assignments", segmentAllocations))
|
||||
|
@ -477,13 +434,11 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||
log.Info("receive SaveBinlogPaths request",
|
||||
zap.Bool("isFlush", req.GetFlushed()),
|
||||
zap.Bool("isDropped", req.GetDropped()),
|
||||
zap.Bool("isImport", req.GetImporting()),
|
||||
zap.Any("checkpoints", req.GetCheckPoints()))
|
||||
|
||||
// for compatibility issue , if len(channelName) not exist, skip the check
|
||||
// No need to check import channel--node matching in data import case.
|
||||
// Also avoid to handle segment not found error if not the owner of shard
|
||||
if !req.GetImporting() && len(channelName) != 0 {
|
||||
if len(channelName) != 0 {
|
||||
if !s.channelManager.Match(nodeID, channelName) {
|
||||
err := merr.WrapErrChannelNotFound(channelName, fmt.Sprintf("for node %d", nodeID))
|
||||
log.Warn("node is not matched with channel", zap.String("channel", channelName), zap.Error(err))
|
||||
|
@ -538,7 +493,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||
operators = append(operators,
|
||||
UpdateBinlogsOperator(req.GetSegmentID(), req.GetField2BinlogPaths(), req.GetField2StatslogPaths(), req.GetDeltalogs()),
|
||||
UpdateStartPosition(req.GetStartPositions()),
|
||||
UpdateCheckPointOperator(req.GetSegmentID(), req.GetImporting(), req.GetCheckPoints()),
|
||||
UpdateCheckPointOperator(req.GetSegmentID(), req.GetCheckPoints()),
|
||||
)
|
||||
|
||||
if Params.CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
|
@ -569,7 +524,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||
s.flushCh <- req.SegmentID
|
||||
|
||||
// notify compaction
|
||||
if !req.Importing && paramtable.Get().DataCoordCfg.EnableCompaction.GetAsBool() {
|
||||
if paramtable.Get().DataCoordCfg.EnableCompaction.GetAsBool() {
|
||||
err := s.compactionTrigger.triggerSingleCompaction(req.GetCollectionID(), req.GetPartitionID(),
|
||||
req.GetSegmentID(), req.GetChannel(), false)
|
||||
if err != nil {
|
||||
|
@ -1406,48 +1361,6 @@ func (s *Server) GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAll
|
|||
return resp, nil
|
||||
}
|
||||
|
||||
// Import distributes the import tasks to DataNodes.
|
||||
// It returns a failed status if no DataNode is available or if any error occurs.
|
||||
func (s *Server) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
log := log.Ctx(ctx)
|
||||
log.Info("DataCoord receives import request", zap.Any("req", req))
|
||||
resp := &datapb.ImportTaskResponse{
|
||||
Status: merr.Success(),
|
||||
}
|
||||
|
||||
if err := merr.CheckHealthy(s.GetStateCode()); err != nil {
|
||||
return &datapb.ImportTaskResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
|
||||
nodes := s.sessionManager.GetSessionIDs()
|
||||
if len(nodes) == 0 {
|
||||
log.Warn("import failed as all DataNodes are offline")
|
||||
resp.Status = merr.Status(merr.WrapErrNodeLackAny("no live DataNode"))
|
||||
return resp, nil
|
||||
}
|
||||
log.Info("available DataNodes are", zap.Int64s("nodeIDs", nodes))
|
||||
|
||||
avaNodes := getDiff(nodes, req.GetWorkingNodes())
|
||||
if len(avaNodes) > 0 {
|
||||
// If there exists available DataNodes, pick one at random.
|
||||
resp.DatanodeId = avaNodes[rand.Intn(len(avaNodes))]
|
||||
log.Info("picking a free DataNode",
|
||||
zap.Any("all DataNodes", nodes),
|
||||
zap.Int64("picking free DataNode with ID", resp.GetDatanodeId()))
|
||||
s.cluster.Import(s.ctx, resp.GetDatanodeId(), req)
|
||||
} else {
|
||||
// No DataNode is available, reject the import request.
|
||||
msg := "all DataNodes are busy working on data import, the task has been rejected and wait for idle DataNode"
|
||||
log.Info(msg, zap.Int64("taskID", req.GetImportTask().GetTaskId()))
|
||||
resp.Status = merr.Status(merr.WrapErrNodeLackAny("no available DataNode"))
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
// UpdateSegmentStatistics updates a segment's stats.
|
||||
func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
|
||||
if err := merr.CheckHealthy(s.GetStateCode()); err != nil {
|
||||
|
@ -1587,75 +1500,6 @@ func getDiff(base, remove []int64) []int64 {
|
|||
return diff
|
||||
}
|
||||
|
||||
// SaveImportSegment saves the segment binlog paths and puts this segment to its belonging DataNode as a flushed segment.
|
||||
func (s *Server) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.Int64("collectionID", req.GetCollectionId()),
|
||||
zap.Int64("segmentID", req.GetSegmentId()),
|
||||
zap.Int64("partitionID", req.GetPartitionId()),
|
||||
zap.String("channelName", req.GetChannelName()),
|
||||
zap.Int64("# of rows", req.GetRowNum()),
|
||||
)
|
||||
log.Info("DataCoord putting segment to the right DataNode and saving binlog path")
|
||||
if err := merr.CheckHealthy(s.GetStateCode()); err != nil {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
_, err := s.cluster.AddImportSegment(ctx,
|
||||
&datapb.AddImportSegmentRequest{
|
||||
Base: commonpbutil.NewMsgBase(
|
||||
commonpbutil.WithTimeStamp(req.GetBase().GetTimestamp()),
|
||||
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
||||
),
|
||||
SegmentId: req.GetSegmentId(),
|
||||
ChannelName: req.GetChannelName(),
|
||||
CollectionId: req.GetCollectionId(),
|
||||
PartitionId: req.GetPartitionId(),
|
||||
RowNum: req.GetRowNum(),
|
||||
StatsLog: req.GetSaveBinlogPathReq().GetField2StatslogPaths(),
|
||||
})
|
||||
if err != nil {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// Fill in position message ID by channel checkpoint.
|
||||
channelCP := s.meta.GetChannelCheckpoint(req.GetChannelName())
|
||||
if channelCP == nil {
|
||||
log.Warn("SaveImportSegment get nil channel checkpoint")
|
||||
return merr.Status(merr.WrapErrImportFailed(fmt.Sprintf("nil checkpoint when saving import segment, segmentID=%d, channel=%s",
|
||||
req.GetSegmentId(), req.GetChannelName()))), nil
|
||||
}
|
||||
req.SaveBinlogPathReq.StartPositions[0].StartPosition.MsgID = channelCP.GetMsgID()
|
||||
req.SaveBinlogPathReq.CheckPoints[0].Position.MsgID = channelCP.GetMsgID()
|
||||
|
||||
// Start saving bin log paths.
|
||||
rsp, err := s.SaveBinlogPaths(context.Background(), req.GetSaveBinlogPathReq())
|
||||
if err := VerifyResponse(rsp, err); err != nil {
|
||||
log.Error("failed to SaveBinlogPaths", zap.Error(err))
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
// UnsetIsImportingState unsets the isImporting states of the given segments.
|
||||
// An error status will be returned and error will be logged, if we failed to update *all* segments.
|
||||
func (s *Server) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx)
|
||||
log.Info("unsetting isImport state of segments",
|
||||
zap.Int64s("segments", req.GetSegmentIds()))
|
||||
var reportErr error
|
||||
for _, segID := range req.GetSegmentIds() {
|
||||
if err := s.meta.UnsetIsImporting(segID); err != nil {
|
||||
// Fail-open.
|
||||
log.Error("failed to unset segment is importing state",
|
||||
zap.Int64("segmentID", segID),
|
||||
)
|
||||
reportErr = err
|
||||
}
|
||||
}
|
||||
|
||||
return merr.Status(reportErr), nil
|
||||
}
|
||||
|
||||
// MarkSegmentsDropped marks the given segments as `Dropped`.
|
||||
// An error status will be returned and error will be logged, if we failed to mark *all* segments.
|
||||
// Deprecated, do not use it
|
||||
|
|
|
@ -420,36 +420,6 @@ func (s *ServerSuite) TestSaveBinlogPath_NormalCase() {
|
|||
s.EqualValues(segment.NumOfRows, 10)
|
||||
}
|
||||
|
||||
func (s *ServerSuite) TestFlushForImport() {
|
||||
schema := newTestSchema()
|
||||
s.testServer.meta.AddCollection(&collectionInfo{ID: 0, Schema: schema, Partitions: []int64{}})
|
||||
|
||||
// normal
|
||||
allocation, err := s.testServer.segmentManager.allocSegmentForImport(
|
||||
context.TODO(), 0, 1, "ch-1", 1, 1)
|
||||
s.NoError(err)
|
||||
segmentID := allocation.SegmentID
|
||||
req := &datapb.FlushRequest{
|
||||
CollectionID: 0,
|
||||
SegmentIDs: []UniqueID{segmentID},
|
||||
}
|
||||
resp, err := s.testServer.flushForImport(context.TODO(), req)
|
||||
s.NoError(err)
|
||||
s.EqualValues(int32(0), resp.GetStatus().GetCode())
|
||||
|
||||
// failed
|
||||
allocation, err = s.testServer.segmentManager.allocSegmentForImport(
|
||||
context.TODO(), 0, 1, "ch-1", 1, 1)
|
||||
s.NoError(err)
|
||||
catalog := mocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
||||
s.testServer.meta.catalog = catalog
|
||||
req.SegmentIDs = []UniqueID{allocation.SegmentID}
|
||||
resp, err = s.testServer.flushForImport(context.TODO(), req)
|
||||
s.NoError(err)
|
||||
s.NotEqual(int32(0), resp.GetStatus().GetCode())
|
||||
}
|
||||
|
||||
func (s *ServerSuite) TestFlush_NormalCase() {
|
||||
req := &datapb.FlushRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
|
@ -491,71 +461,6 @@ func (s *ServerSuite) TestFlush_NormalCase() {
|
|||
s.EqualValues(segID, ids[0])
|
||||
}
|
||||
|
||||
func (s *ServerSuite) TestFlush_BulkLoadSegment() {
|
||||
req := &datapb.FlushRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Flush,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: 0,
|
||||
},
|
||||
DbID: 0,
|
||||
CollectionID: 0,
|
||||
}
|
||||
s.mockChMgr.EXPECT().GetNodeChannelsByCollectionID(mock.Anything).Return(map[int64][]string{
|
||||
1: {"channel-1"},
|
||||
})
|
||||
|
||||
mockCluster := NewMockCluster(s.T())
|
||||
mockCluster.EXPECT().FlushChannels(mock.Anything, mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(nil)
|
||||
mockCluster.EXPECT().Close().Maybe()
|
||||
s.testServer.cluster = mockCluster
|
||||
|
||||
schema := newTestSchema()
|
||||
s.testServer.meta.AddCollection(&collectionInfo{ID: 0, Schema: schema, Partitions: []int64{}})
|
||||
|
||||
allocations, err := s.testServer.segmentManager.allocSegmentForImport(context.TODO(), 0, 1, "channel-1", 1, 100)
|
||||
s.NoError(err)
|
||||
expireTs := allocations.ExpireTime
|
||||
segID := allocations.SegmentID
|
||||
|
||||
resp, err := s.testServer.Flush(context.TODO(), req)
|
||||
s.NoError(err)
|
||||
s.EqualValues(commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
s.EqualValues(0, len(resp.SegmentIDs))
|
||||
// should not flush anything since this is a normal flush
|
||||
s.testServer.meta.SetCurrentRows(segID, 1)
|
||||
ids, err := s.testServer.segmentManager.GetFlushableSegments(context.TODO(), "channel-1", expireTs)
|
||||
s.NoError(err)
|
||||
s.EqualValues(0, len(ids))
|
||||
|
||||
req = &datapb.FlushRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Flush,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: 0,
|
||||
},
|
||||
DbID: 0,
|
||||
CollectionID: 0,
|
||||
SegmentIDs: []int64{segID},
|
||||
IsImport: true,
|
||||
}
|
||||
|
||||
resp, err = s.testServer.Flush(context.TODO(), req)
|
||||
s.NoError(err)
|
||||
s.EqualValues(commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
segment := s.testServer.meta.GetSegment(segID)
|
||||
s.Equal(commonpb.SegmentState_Flushed, segment.GetState())
|
||||
|
||||
err = s.testServer.meta.UnsetIsImporting(segID)
|
||||
s.NoError(err)
|
||||
ids, err = s.testServer.segmentManager.GetFlushableSegments(context.TODO(), "channel-1", expireTs)
|
||||
s.NoError(err)
|
||||
s.EqualValues(0, len(ids))
|
||||
}
|
||||
|
||||
func (s *ServerSuite) TestFlush_ClosedServer() {
|
||||
s.TearDownTest()
|
||||
req := &datapb.FlushRequest{
|
||||
|
@ -671,39 +576,6 @@ func (s *ServerSuite) TestAssignSegmentID() {
|
|||
s.EqualValues(1000, assign.Count)
|
||||
})
|
||||
|
||||
s.Run("assign segment for bulkload", func() {
|
||||
s.SetupTest()
|
||||
defer s.TearDownTest()
|
||||
|
||||
schema := newTestSchema()
|
||||
s.testServer.meta.AddCollection(&collectionInfo{
|
||||
ID: collID,
|
||||
Schema: schema,
|
||||
Partitions: []int64{},
|
||||
})
|
||||
req := &datapb.SegmentIDRequest{
|
||||
Count: 1000,
|
||||
ChannelName: channel0,
|
||||
CollectionID: collID,
|
||||
PartitionID: partID,
|
||||
IsImport: true,
|
||||
}
|
||||
|
||||
resp, err := s.testServer.AssignSegmentID(context.TODO(), &datapb.AssignSegmentIDRequest{
|
||||
NodeID: 0,
|
||||
PeerRole: "",
|
||||
SegmentIDRequests: []*datapb.SegmentIDRequest{req},
|
||||
})
|
||||
s.NoError(err)
|
||||
s.EqualValues(1, len(resp.SegIDAssignments))
|
||||
assign := resp.SegIDAssignments[0]
|
||||
s.EqualValues(commonpb.ErrorCode_Success, assign.GetStatus().GetErrorCode())
|
||||
s.EqualValues(collID, assign.CollectionID)
|
||||
s.EqualValues(partID, assign.PartitionID)
|
||||
s.EqualValues(channel0, assign.ChannelName)
|
||||
s.EqualValues(1000, assign.Count)
|
||||
})
|
||||
|
||||
s.Run("with closed server", func() {
|
||||
s.SetupTest()
|
||||
s.TearDownTest()
|
||||
|
|
|
@ -42,10 +42,7 @@ import (
|
|||
)
|
||||
|
||||
const (
|
||||
flushTimeout = 15 * time.Second
|
||||
// TODO: evaluate and update import timeout.
|
||||
importTimeout = 3 * time.Hour
|
||||
|
||||
flushTimeout = 15 * time.Second
|
||||
importTaskTimeout = 10 * time.Second
|
||||
)
|
||||
|
||||
|
@ -59,11 +56,9 @@ type SessionManager interface {
|
|||
FlushChannels(ctx context.Context, nodeID int64, req *datapb.FlushChannelsRequest) error
|
||||
Compaction(ctx context.Context, nodeID int64, plan *datapb.CompactionPlan) error
|
||||
SyncSegments(nodeID int64, req *datapb.SyncSegmentsRequest) error
|
||||
Import(ctx context.Context, nodeID int64, itr *datapb.ImportTaskRequest)
|
||||
GetCompactionPlansResults() (map[int64]*typeutil.Pair[int64, *datapb.CompactionPlanResult], error)
|
||||
NotifyChannelOperation(ctx context.Context, nodeID int64, req *datapb.ChannelOperationsRequest) error
|
||||
CheckChannelOperationProgress(ctx context.Context, nodeID int64, info *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error)
|
||||
AddImportSegment(ctx context.Context, nodeID int64, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)
|
||||
PreImport(nodeID int64, in *datapb.PreImportRequest) error
|
||||
ImportV2(nodeID int64, in *datapb.ImportRequest) error
|
||||
QueryPreImport(nodeID int64, in *datapb.QueryPreImportRequest) (*datapb.QueryPreImportResponse, error)
|
||||
|
@ -247,29 +242,6 @@ func (c *SessionManagerImpl) SyncSegments(nodeID int64, req *datapb.SyncSegments
|
|||
return nil
|
||||
}
|
||||
|
||||
// Import is a grpc interface. It will send request to DataNode with provided `nodeID` asynchronously.
|
||||
func (c *SessionManagerImpl) Import(ctx context.Context, nodeID int64, itr *datapb.ImportTaskRequest) {
|
||||
go c.execImport(ctx, nodeID, itr)
|
||||
}
|
||||
|
||||
// execImport gets the corresponding DataNode with its ID and calls its Import method.
|
||||
func (c *SessionManagerImpl) execImport(ctx context.Context, nodeID int64, itr *datapb.ImportTaskRequest) {
|
||||
cli, err := c.getClient(ctx, nodeID)
|
||||
if err != nil {
|
||||
log.Warn("failed to get client for import", zap.Int64("nodeID", nodeID), zap.Error(err))
|
||||
return
|
||||
}
|
||||
ctx, cancel := context.WithTimeout(ctx, importTimeout)
|
||||
defer cancel()
|
||||
resp, err := cli.Import(ctx, itr)
|
||||
if err := VerifyResponse(resp, err); err != nil {
|
||||
log.Warn("failed to import", zap.Int64("node", nodeID), zap.Error(err))
|
||||
return
|
||||
}
|
||||
|
||||
log.Info("success to import", zap.Int64("node", nodeID), zap.Any("import task", itr))
|
||||
}
|
||||
|
||||
// GetCompactionPlanResults returns map[planID]*pair[nodeID, *CompactionPlanResults]
|
||||
func (c *SessionManagerImpl) GetCompactionPlansResults() (map[int64]*typeutil.Pair[int64, *datapb.CompactionPlanResult], error) {
|
||||
ctx := context.Background()
|
||||
|
@ -384,25 +356,6 @@ func (c *SessionManagerImpl) CheckChannelOperationProgress(ctx context.Context,
|
|||
return resp, nil
|
||||
}
|
||||
|
||||
func (c *SessionManagerImpl) AddImportSegment(ctx context.Context, nodeID int64, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
// Call DataNode to add the new segment to its own flow graph.
|
||||
cli, err := c.getClient(ctx, nodeID)
|
||||
if err != nil {
|
||||
log.Error("failed to get DataNode client for SaveImportSegment",
|
||||
zap.Int64("DataNode ID", nodeID),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
resp, err := cli.AddImportSegment(ctx, req)
|
||||
if err := VerifyResponse(resp.GetStatus(), err); err != nil {
|
||||
log.Error("failed to add segment", zap.Int64("nodeID", nodeID), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
log.Info("succeed to add segment", zap.Int64("nodeID", nodeID), zap.Any("add segment req", req))
|
||||
return resp, err
|
||||
}
|
||||
|
||||
func (c *SessionManagerImpl) PreImport(nodeID int64, in *datapb.PreImportRequest) error {
|
||||
log := log.With(
|
||||
zap.Int64("nodeID", nodeID),
|
||||
|
|
|
@ -6,7 +6,6 @@ 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/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
@ -39,7 +38,6 @@ func NewCoordBroker(rc types.RootCoordClient, dc types.DataCoordClient, serverID
|
|||
type RootCoord interface {
|
||||
DescribeCollection(ctx context.Context, collectionID typeutil.UniqueID, ts typeutil.Timestamp) (*milvuspb.DescribeCollectionResponse, error)
|
||||
ShowPartitions(ctx context.Context, dbName, collectionName string) (map[string]int64, error)
|
||||
ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) error
|
||||
AllocTimestamp(ctx context.Context, num uint32) (ts uint64, count uint32, err error)
|
||||
}
|
||||
|
||||
|
@ -52,5 +50,4 @@ type DataCoord interface {
|
|||
SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) error
|
||||
DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error)
|
||||
UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) error
|
||||
SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) error
|
||||
}
|
||||
|
|
|
@ -146,15 +146,3 @@ func (dc *dataCoordBroker) UpdateSegmentStatistics(ctx context.Context, req *dat
|
|||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (dc *dataCoordBroker) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) error {
|
||||
log := log.Ctx(ctx)
|
||||
|
||||
resp, err := dc.client.SaveImportSegment(ctx, req)
|
||||
if err := merr.CheckRPCCall(resp, err); err != nil {
|
||||
log.Warn("failed to UpdateSegmentStatistics", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -328,47 +328,6 @@ func (s *dataCoordSuite) TestUpdateSegmentStatistics() {
|
|||
})
|
||||
}
|
||||
|
||||
func (s *dataCoordSuite) TestSaveImportSegment() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
segmentID := int64(1001)
|
||||
collectionID := int64(100)
|
||||
|
||||
req := &datapb.SaveImportSegmentRequest{
|
||||
SegmentId: segmentID,
|
||||
CollectionId: collectionID,
|
||||
}
|
||||
|
||||
s.Run("normal_case", func() {
|
||||
s.dc.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).
|
||||
Run(func(_ context.Context, r *datapb.SaveImportSegmentRequest, _ ...grpc.CallOption) {
|
||||
s.Equal(collectionID, req.GetCollectionId())
|
||||
s.Equal(segmentID, req.GetSegmentId())
|
||||
}).
|
||||
Return(merr.Status(nil), nil)
|
||||
err := s.broker.SaveImportSegment(ctx, req)
|
||||
s.NoError(err)
|
||||
s.resetMock()
|
||||
})
|
||||
|
||||
s.Run("datacoord_return_failure_status", func() {
|
||||
s.dc.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).
|
||||
Return(nil, errors.New("mock"))
|
||||
err := s.broker.SaveImportSegment(ctx, req)
|
||||
s.Error(err)
|
||||
s.resetMock()
|
||||
})
|
||||
|
||||
s.Run("datacoord_return_failure_status", func() {
|
||||
s.dc.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).
|
||||
Return(merr.Status(errors.New("mock")), nil)
|
||||
err := s.broker.SaveImportSegment(ctx, req)
|
||||
s.Error(err)
|
||||
s.resetMock()
|
||||
})
|
||||
}
|
||||
|
||||
func TestDataCoordBroker(t *testing.T) {
|
||||
suite.Run(t, new(dataCoordSuite))
|
||||
}
|
||||
|
|
|
@ -11,8 +11,6 @@ import (
|
|||
mock "github.com/stretchr/testify/mock"
|
||||
|
||||
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
|
||||
rootcoordpb "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
)
|
||||
|
||||
// MockBroker is an autogenerated mock type for the Broker type
|
||||
|
@ -323,49 +321,6 @@ func (_c *MockBroker_GetSegmentInfo_Call) RunAndReturn(run func(context.Context,
|
|||
return _c
|
||||
}
|
||||
|
||||
// ReportImport provides a mock function with given fields: ctx, req
|
||||
func (_m *MockBroker) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) error {
|
||||
ret := _m.Called(ctx, req)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ImportResult) error); ok {
|
||||
r0 = rf(ctx, req)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockBroker_ReportImport_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ReportImport'
|
||||
type MockBroker_ReportImport_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// ReportImport is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - req *rootcoordpb.ImportResult
|
||||
func (_e *MockBroker_Expecter) ReportImport(ctx interface{}, req interface{}) *MockBroker_ReportImport_Call {
|
||||
return &MockBroker_ReportImport_Call{Call: _e.mock.On("ReportImport", ctx, req)}
|
||||
}
|
||||
|
||||
func (_c *MockBroker_ReportImport_Call) Run(run func(ctx context.Context, req *rootcoordpb.ImportResult)) *MockBroker_ReportImport_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*rootcoordpb.ImportResult))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroker_ReportImport_Call) Return(_a0 error) *MockBroker_ReportImport_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroker_ReportImport_Call) RunAndReturn(run func(context.Context, *rootcoordpb.ImportResult) error) *MockBroker_ReportImport_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ReportTimeTick provides a mock function with given fields: ctx, msgs
|
||||
func (_m *MockBroker) ReportTimeTick(ctx context.Context, msgs []*msgpb.DataNodeTtMsg) error {
|
||||
ret := _m.Called(ctx, msgs)
|
||||
|
@ -452,49 +407,6 @@ func (_c *MockBroker_SaveBinlogPaths_Call) RunAndReturn(run func(context.Context
|
|||
return _c
|
||||
}
|
||||
|
||||
// SaveImportSegment provides a mock function with given fields: ctx, req
|
||||
func (_m *MockBroker) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) error {
|
||||
ret := _m.Called(ctx, req)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SaveImportSegmentRequest) error); ok {
|
||||
r0 = rf(ctx, req)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockBroker_SaveImportSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveImportSegment'
|
||||
type MockBroker_SaveImportSegment_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SaveImportSegment is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - req *datapb.SaveImportSegmentRequest
|
||||
func (_e *MockBroker_Expecter) SaveImportSegment(ctx interface{}, req interface{}) *MockBroker_SaveImportSegment_Call {
|
||||
return &MockBroker_SaveImportSegment_Call{Call: _e.mock.On("SaveImportSegment", ctx, req)}
|
||||
}
|
||||
|
||||
func (_c *MockBroker_SaveImportSegment_Call) Run(run func(ctx context.Context, req *datapb.SaveImportSegmentRequest)) *MockBroker_SaveImportSegment_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*datapb.SaveImportSegmentRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroker_SaveImportSegment_Call) Return(_a0 error) *MockBroker_SaveImportSegment_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockBroker_SaveImportSegment_Call) RunAndReturn(run func(context.Context, *datapb.SaveImportSegmentRequest) error) *MockBroker_SaveImportSegment_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ShowPartitions provides a mock function with given fields: ctx, dbName, collectionName
|
||||
func (_m *MockBroker) ShowPartitions(ctx context.Context, dbName string, collectionName string) (map[string]int64, error) {
|
||||
ret := _m.Called(ctx, dbName, collectionName)
|
||||
|
|
|
@ -101,14 +101,3 @@ func (rc *rootCoordBroker) AllocTimestamp(ctx context.Context, num uint32) (uint
|
|||
}
|
||||
return resp.GetTimestamp(), resp.GetCount(), nil
|
||||
}
|
||||
|
||||
func (rc *rootCoordBroker) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) error {
|
||||
log := log.Ctx(ctx)
|
||||
resp, err := rc.client.ReportImport(ctx, req)
|
||||
|
||||
if err := merr.CheckRPCCall(resp, err); err != nil {
|
||||
log.Warn("failed to ReportImport", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -194,48 +194,6 @@ func (s *rootCoordSuite) TestAllocTimestamp() {
|
|||
})
|
||||
}
|
||||
|
||||
func (s *rootCoordSuite) TestReportImport() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
taskID := rand.Int63()
|
||||
|
||||
req := &rootcoordpb.ImportResult{
|
||||
Status: merr.Status(nil),
|
||||
TaskId: taskID,
|
||||
}
|
||||
|
||||
s.Run("normal_case", func() {
|
||||
s.rc.EXPECT().ReportImport(mock.Anything, mock.Anything).
|
||||
Run(func(_ context.Context, req *rootcoordpb.ImportResult, _ ...grpc.CallOption) {
|
||||
s.Equal(taskID, req.GetTaskId())
|
||||
}).
|
||||
Return(merr.Status(nil), nil)
|
||||
|
||||
err := s.broker.ReportImport(ctx, req)
|
||||
s.NoError(err)
|
||||
s.resetMock()
|
||||
})
|
||||
|
||||
s.Run("rootcoord_return_error", func() {
|
||||
s.rc.EXPECT().ReportImport(mock.Anything, mock.Anything).
|
||||
Return(nil, errors.New("mock"))
|
||||
|
||||
err := s.broker.ReportImport(ctx, req)
|
||||
s.Error(err)
|
||||
s.resetMock()
|
||||
})
|
||||
|
||||
s.Run("rootcoord_return_failure_status", func() {
|
||||
s.rc.EXPECT().ReportImport(mock.Anything, mock.Anything).
|
||||
Return(merr.Status(errors.New("mock")), nil)
|
||||
|
||||
err := s.broker.ReportImport(ctx, req)
|
||||
s.Error(err)
|
||||
s.resetMock()
|
||||
})
|
||||
}
|
||||
|
||||
func TestRootCoordBroker(t *testing.T) {
|
||||
suite.Run(t, new(rootCoordSuite))
|
||||
}
|
||||
|
|
|
@ -35,7 +35,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/importutil"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
|
@ -81,8 +80,6 @@ func TestMain(t *testing.M) {
|
|||
}
|
||||
|
||||
func TestDataNode(t *testing.T) {
|
||||
importutil.ReportImportAttempts = 1
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
|
|
|
@ -276,14 +276,6 @@ func (ds *DataCoordFactory) ReportDataNodeTtMsgs(ctx context.Context, req *datap
|
|||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (ds *DataCoordFactory) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (ds *DataCoordFactory) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func (ds *DataCoordFactory) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
@ -1060,23 +1052,6 @@ func (m *RootCoordFactory) GetComponentStates(ctx context.Context, req *milvuspb
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (m *RootCoordFactory) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
if ctx != nil && ctx.Value(ctxKey{}) != nil {
|
||||
if v := ctx.Value(ctxKey{}).(string); v == returnError {
|
||||
return nil, fmt.Errorf("injected error")
|
||||
}
|
||||
}
|
||||
if m.ReportImportErr {
|
||||
return merr.Success(), fmt.Errorf("mock report import error")
|
||||
}
|
||||
if m.ReportImportNotSuccess {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
}, nil
|
||||
}
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
// FailMessageStreamFactory mock MessageStreamFactory failure
|
||||
type FailMessageStreamFactory struct {
|
||||
dependency.Factory
|
||||
|
|
|
@ -22,40 +22,25 @@ package datanode
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"path"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"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"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/importutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/tracer"
|
||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/metautil"
|
||||
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
// WatchDmChannels is not in use
|
||||
|
@ -402,121 +387,6 @@ func (node *DataNode) CheckChannelOperationProgress(ctx context.Context, req *da
|
|||
return node.channelManager.GetProgress(req), nil
|
||||
}
|
||||
|
||||
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
||||
func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
|
||||
logFields := []zap.Field{
|
||||
zap.Int64("task ID", req.GetImportTask().GetTaskId()),
|
||||
zap.Int64("collectionID", req.GetImportTask().GetCollectionId()),
|
||||
zap.Int64("partitionID", req.GetImportTask().GetPartitionId()),
|
||||
zap.String("database name", req.GetImportTask().GetDatabaseName()),
|
||||
zap.Strings("channel names", req.GetImportTask().GetChannelNames()),
|
||||
zap.Int64s("working dataNodes", req.WorkingNodes),
|
||||
zap.Int64("node ID", node.GetNodeID()),
|
||||
}
|
||||
log.Info("DataNode receive import request", logFields...)
|
||||
defer func() {
|
||||
log.Info("DataNode finish import request", logFields...)
|
||||
}()
|
||||
|
||||
importResult := &rootcoordpb.ImportResult{
|
||||
Status: merr.Success(),
|
||||
TaskId: req.GetImportTask().TaskId,
|
||||
DatanodeId: node.GetNodeID(),
|
||||
State: commonpb.ImportState_ImportStarted,
|
||||
Segments: make([]int64, 0),
|
||||
AutoIds: make([]int64, 0),
|
||||
RowCount: 0,
|
||||
}
|
||||
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.ProgressPercent, Value: "0"})
|
||||
|
||||
// Spawn a new context to ignore cancellation from parental context.
|
||||
newCtx, cancel := context.WithTimeout(context.TODO(), paramtable.Get().DataNodeCfg.BulkInsertTimeoutSeconds.GetAsDuration(time.Second))
|
||||
defer cancel()
|
||||
|
||||
// function to report import state to RootCoord.
|
||||
// retry 10 times, if the rootcoord is down, the report function will cost 20+ seconds
|
||||
reportFunc := reportImportFunc(node)
|
||||
returnFailFunc := func(msg string, err error) (*commonpb.Status, error) {
|
||||
logFields = append(logFields, zap.Error(err))
|
||||
log.Warn(msg, logFields...)
|
||||
importResult.State = commonpb.ImportState_ImportFailed
|
||||
importResult.Infos = append(importResult.Infos, &commonpb.KeyValuePair{Key: importutil.FailedReason, Value: err.Error()})
|
||||
|
||||
reportFunc(importResult)
|
||||
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
logFields = append(logFields, zap.Error(err))
|
||||
log.Warn("DataNode import failed, node is not healthy", logFields...)
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// get a timestamp for all the rows
|
||||
// Ignore cancellation from parent context.
|
||||
ts, _, err := node.broker.AllocTimestamp(newCtx, 1)
|
||||
if err != nil {
|
||||
return returnFailFunc("DataNode alloc ts failed", err)
|
||||
}
|
||||
|
||||
// get collection schema and shard number
|
||||
metaService := newMetaService(node.broker, req.GetImportTask().GetCollectionId())
|
||||
colInfo, err := metaService.getCollectionInfo(newCtx, req.GetImportTask().GetCollectionId(), 0)
|
||||
if err != nil {
|
||||
return returnFailFunc("failed to get collection info for collection ID", err)
|
||||
}
|
||||
|
||||
var partitionIDs []int64
|
||||
if req.GetImportTask().GetPartitionId() == 0 {
|
||||
if !typeutil.HasPartitionKey(colInfo.GetSchema()) {
|
||||
err = errors.New("try auto-distribute data but the collection has no partition key")
|
||||
return returnFailFunc(err.Error(), err)
|
||||
}
|
||||
// TODO: prefer to set partitionIDs in coord instead of get here.
|
||||
// the colInfo doesn't have a correct database name(it is empty). use the database name passed from rootcoord.
|
||||
partitions, err := node.broker.ShowPartitions(ctx, req.GetImportTask().GetDatabaseName(), colInfo.GetCollectionName())
|
||||
if err != nil {
|
||||
return returnFailFunc("failed to get partition id list", err)
|
||||
}
|
||||
_, partitionIDs, err = typeutil.RearrangePartitionsForPartitionKey(partitions)
|
||||
if err != nil {
|
||||
return returnFailFunc("failed to rearrange target partitions", err)
|
||||
}
|
||||
} else {
|
||||
partitionIDs = []int64{req.GetImportTask().GetPartitionId()}
|
||||
}
|
||||
|
||||
collectionInfo, err := importutil.NewCollectionInfo(colInfo.GetSchema(), colInfo.GetShardsNum(), partitionIDs)
|
||||
if err != nil {
|
||||
return returnFailFunc("invalid collection info to import", err)
|
||||
}
|
||||
|
||||
// parse files and generate segments
|
||||
segmentSize := Params.DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024
|
||||
importWrapper := importutil.NewImportWrapper(newCtx, collectionInfo, segmentSize, Params.DataNodeCfg.BinLogMaxSize.GetAsInt64(),
|
||||
node.allocator.GetIDAlloactor(), node.chunkManager, importResult, reportFunc)
|
||||
importWrapper.SetCallbackFunctions(assignSegmentFunc(node, req),
|
||||
createBinLogsFunc(node, req, colInfo.GetSchema(), ts),
|
||||
saveSegmentFunc(node, req, importResult, ts))
|
||||
// todo: pass tsStart and tsStart after import_wrapper support
|
||||
tsStart, tsEnd, err := importutil.ParseTSFromOptions(req.GetImportTask().GetInfos())
|
||||
isBackup := importutil.IsBackup(req.GetImportTask().GetInfos())
|
||||
if err != nil {
|
||||
return returnFailFunc("failed to parse timestamp from import options", err)
|
||||
}
|
||||
logFields = append(logFields, zap.Uint64("start_ts", tsStart), zap.Uint64("end_ts", tsEnd))
|
||||
log.Info("import time range", logFields...)
|
||||
err = importWrapper.Import(req.GetImportTask().GetFiles(),
|
||||
importutil.ImportOptions{OnlyValidate: false, TsStartPoint: tsStart, TsEndPoint: tsEnd, IsBackup: isBackup})
|
||||
if err != nil {
|
||||
return returnFailFunc("failed to import files", err)
|
||||
}
|
||||
|
||||
resp := merr.Success()
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
func (node *DataNode) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx).With(zap.Int64("nodeId", node.GetNodeID()),
|
||||
zap.Time("flushTs", tsoutil.PhysicalTime(req.GetFlushTs())),
|
||||
|
@ -540,421 +410,6 @@ func (node *DataNode) FlushChannels(ctx context.Context, req *datapb.FlushChanne
|
|||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
// AddImportSegment adds the import segment to the current DataNode.
|
||||
func (node *DataNode) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
logFields := []zap.Field{
|
||||
zap.Int64("segmentID", req.GetSegmentId()),
|
||||
zap.Int64("collectionID", req.GetCollectionId()),
|
||||
zap.Int64("partitionID", req.GetPartitionId()),
|
||||
zap.String("channelName", req.GetChannelName()),
|
||||
zap.Int64("# of rows", req.GetRowNum()),
|
||||
}
|
||||
log.Info("adding segment to DataNode flow graph", logFields...)
|
||||
// Fetch the flow graph on the given v-channel.
|
||||
var ds *dataSyncService
|
||||
// Retry in case the channel hasn't been watched yet.
|
||||
err := retry.Do(ctx, func() error {
|
||||
var ok bool
|
||||
ds, ok = node.flowgraphManager.GetFlowgraphService(req.GetChannelName())
|
||||
if !ok {
|
||||
return errors.New("channel not found")
|
||||
}
|
||||
return nil
|
||||
}, retry.Attempts(getFlowGraphServiceAttempts))
|
||||
if err != nil {
|
||||
logFields = append(logFields, zap.Int64("node ID", node.GetNodeID()))
|
||||
log.Error("channel not found in current DataNode", logFields...)
|
||||
return &datapb.AddImportSegmentResponse{
|
||||
Status: &commonpb.Status{
|
||||
// TODO: Add specific error code.
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "channel not found in current DataNode",
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
// Add the new segment to the channel.
|
||||
if len(ds.metacache.GetSegmentIDsBy(metacache.WithSegmentIDs(req.GetSegmentId()), metacache.WithSegmentState(commonpb.SegmentState_Flushed))) == 0 {
|
||||
log.Info("adding a new segment to channel", logFields...)
|
||||
// no error will be throw
|
||||
err := binlog.DecompressBinLog(storage.StatsBinlog, req.GetCollectionId(), req.GetPartitionId(), req.GetSegmentId(), req.GetStatsLog())
|
||||
if err != nil {
|
||||
log.Warn("failed to DecompressBinLog", zap.Error(err))
|
||||
return &datapb.AddImportSegmentResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
pks, err := loadStats(ctx, node.chunkManager, ds.metacache.Schema(), req.GetSegmentId(), req.GetStatsLog())
|
||||
if err != nil {
|
||||
log.Warn("failed to get segment pk stats", zap.Error(err))
|
||||
return &datapb.AddImportSegmentResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Add segment as a flushed segment, but set `importing` to true to add extra information of the segment.
|
||||
// By 'extra information' we mean segment info while adding a `SegmentType_Flushed` typed segment.
|
||||
// ds.metacache.
|
||||
ds.metacache.AddSegment(&datapb.SegmentInfo{
|
||||
ID: req.GetSegmentId(),
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
CollectionID: req.GetCollectionId(),
|
||||
PartitionID: req.GetPartitionId(),
|
||||
InsertChannel: req.GetChannelName(),
|
||||
NumOfRows: req.GetRowNum(),
|
||||
Statslogs: req.GetStatsLog(),
|
||||
StartPosition: &msgpb.MsgPosition{
|
||||
ChannelName: req.GetChannelName(),
|
||||
Timestamp: req.GetBase().GetTimestamp(),
|
||||
},
|
||||
DmlPosition: &msgpb.MsgPosition{
|
||||
ChannelName: req.GetChannelName(),
|
||||
Timestamp: req.GetBase().GetTimestamp(),
|
||||
},
|
||||
}, func(info *datapb.SegmentInfo) *metacache.BloomFilterSet {
|
||||
bfs := metacache.NewBloomFilterSet(pks...)
|
||||
return bfs
|
||||
})
|
||||
}
|
||||
|
||||
return &datapb.AddImportSegmentResponse{
|
||||
Status: merr.Success(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func assignSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest) importutil.AssignSegmentFunc {
|
||||
return func(shardID int, partID int64) (int64, string, error) {
|
||||
chNames := req.GetImportTask().GetChannelNames()
|
||||
importTaskID := req.GetImportTask().GetTaskId()
|
||||
logFields := []zap.Field{
|
||||
zap.Int64("task ID", importTaskID),
|
||||
zap.Int("shard ID", shardID),
|
||||
zap.Int64("partitionID", partID),
|
||||
zap.Int("# of channels", len(chNames)),
|
||||
zap.Strings("channel names", chNames),
|
||||
}
|
||||
if shardID >= len(chNames) {
|
||||
log.Error("import task returns invalid shard ID", logFields...)
|
||||
return 0, "", fmt.Errorf("syncSegmentID Failed: invalid shard ID %d", shardID)
|
||||
}
|
||||
|
||||
tr := timerecord.NewTimeRecorder("assign segment function")
|
||||
defer tr.Elapse("finished")
|
||||
|
||||
colID := req.GetImportTask().GetCollectionId()
|
||||
segmentIDReq := composeAssignSegmentIDRequest(1, shardID, chNames, colID, partID)
|
||||
targetChName := segmentIDReq.GetSegmentIDRequests()[0].GetChannelName()
|
||||
logFields = append(logFields, zap.Int64("collection ID", colID))
|
||||
logFields = append(logFields, zap.String("target channel name", targetChName))
|
||||
log.Info("assign segment for the import task", logFields...)
|
||||
ids, err := node.broker.AssignSegmentID(context.Background(), segmentIDReq.GetSegmentIDRequests()...)
|
||||
if err != nil {
|
||||
return 0, "", errors.Wrap(err, "failed to AssignSegmentID")
|
||||
}
|
||||
|
||||
if len(ids) == 0 {
|
||||
return 0, "", merr.WrapErrSegmentNotFound(0, "failed to assign segment id")
|
||||
}
|
||||
|
||||
segmentID := ids[0]
|
||||
logFields = append(logFields, zap.Int64("segmentID", segmentID))
|
||||
log.Info("new segment assigned", logFields...)
|
||||
|
||||
// call report to notify the rootcoord update the segment id list for this task
|
||||
// ignore the returned error, since even report failed the segments still can be cleaned
|
||||
// retry 10 times, if the rootcoord is down, the report function will cost 20+ seconds
|
||||
importResult := &rootcoordpb.ImportResult{
|
||||
Status: merr.Success(),
|
||||
TaskId: req.GetImportTask().TaskId,
|
||||
DatanodeId: node.GetNodeID(),
|
||||
State: commonpb.ImportState_ImportStarted,
|
||||
Segments: []int64{segmentID},
|
||||
AutoIds: make([]int64, 0),
|
||||
RowCount: 0,
|
||||
}
|
||||
reportFunc := reportImportFunc(node)
|
||||
reportFunc(importResult)
|
||||
|
||||
return segmentID, targetChName, nil
|
||||
}
|
||||
}
|
||||
|
||||
func createBinLogsFunc(node *DataNode, req *datapb.ImportTaskRequest, schema *schemapb.CollectionSchema, ts Timestamp) importutil.CreateBinlogsFunc {
|
||||
return func(fields importutil.BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
|
||||
var rowNum int
|
||||
for _, field := range fields {
|
||||
rowNum = field.RowNum()
|
||||
break
|
||||
}
|
||||
|
||||
chNames := req.GetImportTask().GetChannelNames()
|
||||
importTaskID := req.GetImportTask().GetTaskId()
|
||||
logFields := []zap.Field{
|
||||
zap.Int64("task ID", importTaskID),
|
||||
zap.Int64("partitionID", partID),
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Int("# of channels", len(chNames)),
|
||||
zap.Strings("channel names", chNames),
|
||||
}
|
||||
|
||||
if rowNum <= 0 {
|
||||
log.Info("fields data is empty, no need to generate binlog", logFields...)
|
||||
return nil, nil, nil
|
||||
}
|
||||
logFields = append(logFields, zap.Int("row count", rowNum))
|
||||
|
||||
colID := req.GetImportTask().GetCollectionId()
|
||||
fieldInsert, fieldStats, err := createBinLogs(rowNum, schema, ts, fields, node, segmentID, colID, partID)
|
||||
if err != nil {
|
||||
logFields = append(logFields, zap.Any("err", err))
|
||||
log.Error("failed to create binlogs", logFields...)
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
logFields = append(logFields, zap.Int("insert log count", len(fieldInsert)), zap.Int("stats log count", len(fieldStats)))
|
||||
log.Info("new binlog created", logFields...)
|
||||
|
||||
return fieldInsert, fieldStats, err
|
||||
}
|
||||
}
|
||||
|
||||
func saveSegmentFunc(node *DataNode, req *datapb.ImportTaskRequest, res *rootcoordpb.ImportResult, ts Timestamp) importutil.SaveSegmentFunc {
|
||||
importTaskID := req.GetImportTask().GetTaskId()
|
||||
return func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64,
|
||||
targetChName string, rowCount int64, partID int64,
|
||||
) error {
|
||||
logFields := []zap.Field{
|
||||
zap.Int64("task ID", importTaskID),
|
||||
zap.Int64("partitionID", partID),
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.String("target channel name", targetChName),
|
||||
zap.Int64("row count", rowCount),
|
||||
zap.Uint64("ts", ts),
|
||||
}
|
||||
log.Info("adding segment to the correct DataNode flow graph and saving binlog paths", logFields...)
|
||||
|
||||
err := retry.Do(context.Background(), func() error {
|
||||
// Ask DataCoord to save binlog path and add segment to the corresponding DataNode flow graph.
|
||||
err := node.broker.SaveImportSegment(context.Background(), &datapb.SaveImportSegmentRequest{
|
||||
Base: commonpbutil.NewMsgBase(
|
||||
commonpbutil.WithTimeStamp(ts), // Pass current timestamp downstream.
|
||||
commonpbutil.WithSourceID(node.GetNodeID()),
|
||||
),
|
||||
SegmentId: segmentID,
|
||||
ChannelName: targetChName,
|
||||
CollectionId: req.GetImportTask().GetCollectionId(),
|
||||
PartitionId: partID,
|
||||
RowNum: rowCount,
|
||||
SaveBinlogPathReq: &datapb.SaveBinlogPathsRequest{
|
||||
Base: commonpbutil.NewMsgBase(
|
||||
commonpbutil.WithTimeStamp(ts),
|
||||
commonpbutil.WithSourceID(node.GetNodeID()),
|
||||
),
|
||||
SegmentID: segmentID,
|
||||
CollectionID: req.GetImportTask().GetCollectionId(),
|
||||
Field2BinlogPaths: fieldsInsert,
|
||||
Field2StatslogPaths: fieldsStats,
|
||||
// Set start positions of a SaveBinlogPathRequest explicitly.
|
||||
StartPositions: []*datapb.SegmentStartPosition{
|
||||
{
|
||||
StartPosition: &msgpb.MsgPosition{
|
||||
ChannelName: targetChName,
|
||||
Timestamp: ts,
|
||||
},
|
||||
SegmentID: segmentID,
|
||||
},
|
||||
},
|
||||
CheckPoints: []*datapb.CheckPoint{
|
||||
{
|
||||
SegmentID: segmentID,
|
||||
Position: &msgpb.MsgPosition{
|
||||
ChannelName: targetChName,
|
||||
Timestamp: ts,
|
||||
},
|
||||
NumOfRows: rowCount,
|
||||
},
|
||||
},
|
||||
Importing: true,
|
||||
},
|
||||
})
|
||||
// Only retrying when DataCoord is unhealthy or err != nil, otherwise return immediately.
|
||||
if err != nil {
|
||||
if errors.Is(err, merr.ErrServiceNotReady) {
|
||||
return retry.Unrecoverable(err)
|
||||
}
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}, retry.Attempts(60)) // about 3min
|
||||
if err != nil {
|
||||
log.Warn("failed to save import segment", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("segment imported and persisted", logFields...)
|
||||
res.Segments = append(res.Segments, segmentID)
|
||||
res.RowCount += rowCount
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func composeAssignSegmentIDRequest(rowNum int, shardID int, chNames []string,
|
||||
collID int64, partID int64,
|
||||
) *datapb.AssignSegmentIDRequest {
|
||||
// use the first field's row count as segment row count
|
||||
// all the fields row count are same, checked by ImportWrapper
|
||||
// ask DataCoord to alloc a new segment
|
||||
segReqs := []*datapb.SegmentIDRequest{
|
||||
{
|
||||
ChannelName: chNames[shardID],
|
||||
Count: uint32(rowNum),
|
||||
CollectionID: collID,
|
||||
PartitionID: partID,
|
||||
IsImport: true,
|
||||
},
|
||||
}
|
||||
segmentIDReq := &datapb.AssignSegmentIDRequest{
|
||||
NodeID: 0,
|
||||
PeerRole: typeutil.ProxyRole,
|
||||
SegmentIDRequests: segReqs,
|
||||
}
|
||||
return segmentIDReq
|
||||
}
|
||||
|
||||
func createBinLogs(rowNum int, schema *schemapb.CollectionSchema, ts Timestamp,
|
||||
fields map[storage.FieldID]storage.FieldData, node *DataNode, segmentID, colID, partID UniqueID,
|
||||
) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
tsFieldData := make([]int64, rowNum)
|
||||
for i := range tsFieldData {
|
||||
tsFieldData[i] = int64(ts)
|
||||
}
|
||||
fields[common.TimeStampField] = &storage.Int64FieldData{
|
||||
Data: tsFieldData,
|
||||
}
|
||||
|
||||
if err := node.broker.UpdateSegmentStatistics(context.TODO(), &datapb.UpdateSegmentStatisticsRequest{
|
||||
Stats: []*commonpb.SegmentStats{
|
||||
{
|
||||
SegmentID: segmentID,
|
||||
NumRows: int64(rowNum),
|
||||
},
|
||||
},
|
||||
}); err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
insertData := &InsertData{
|
||||
Data: fields,
|
||||
}
|
||||
// data.updateSize(int64(rowNum))
|
||||
meta := &etcdpb.CollectionMeta{
|
||||
ID: colID,
|
||||
Schema: schema,
|
||||
}
|
||||
iCodec := storage.NewInsertCodecWithSchema(meta)
|
||||
|
||||
binLogs, err := iCodec.Serialize(partID, segmentID, insertData)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
start, _, err := node.allocator.Alloc(uint32(len(binLogs)))
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
field2Insert := make(map[UniqueID]*datapb.Binlog, len(binLogs))
|
||||
kvs := make(map[string][]byte, len(binLogs))
|
||||
field2Logidx := make(map[UniqueID]UniqueID, len(binLogs))
|
||||
for idx, blob := range binLogs {
|
||||
fieldID, err := strconv.ParseInt(blob.GetKey(), 10, 64)
|
||||
if err != nil {
|
||||
log.Error("Flush failed ... cannot parse string to fieldID ..", zap.Error(err))
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
logidx := start + int64(idx)
|
||||
|
||||
k := metautil.JoinIDPath(colID, partID, segmentID, fieldID, logidx)
|
||||
|
||||
key := path.Join(node.chunkManager.RootPath(), common.SegmentInsertLogPath, k)
|
||||
kvs[key] = blob.Value[:]
|
||||
field2Insert[fieldID] = &datapb.Binlog{
|
||||
EntriesNum: int64(rowNum),
|
||||
TimestampFrom: ts,
|
||||
TimestampTo: ts,
|
||||
LogPath: key,
|
||||
LogSize: int64(len(blob.Value)),
|
||||
}
|
||||
field2Logidx[fieldID] = logidx
|
||||
}
|
||||
|
||||
field2Stats := make(map[UniqueID]*datapb.Binlog)
|
||||
// write stats binlog
|
||||
statsBinLog, err := iCodec.SerializePkStatsByData(insertData)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
fieldID, err := strconv.ParseInt(statsBinLog.GetKey(), 10, 64)
|
||||
if err != nil {
|
||||
log.Error("Flush failed ... cannot parse string to fieldID ..", zap.Error(err))
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
logidx := field2Logidx[fieldID]
|
||||
|
||||
// no error raise if alloc=false
|
||||
k := metautil.JoinIDPath(colID, partID, segmentID, fieldID, logidx)
|
||||
|
||||
key := path.Join(node.chunkManager.RootPath(), common.SegmentStatslogPath, k)
|
||||
kvs[key] = statsBinLog.Value
|
||||
field2Stats[fieldID] = &datapb.Binlog{
|
||||
EntriesNum: int64(rowNum),
|
||||
TimestampFrom: ts,
|
||||
TimestampTo: ts,
|
||||
LogPath: key,
|
||||
LogSize: int64(len(statsBinLog.Value)),
|
||||
}
|
||||
|
||||
err = node.chunkManager.MultiWrite(ctx, kvs)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
var (
|
||||
fieldInsert []*datapb.FieldBinlog
|
||||
fieldStats []*datapb.FieldBinlog
|
||||
)
|
||||
for k, v := range field2Insert {
|
||||
fieldInsert = append(fieldInsert, &datapb.FieldBinlog{FieldID: k, Binlogs: []*datapb.Binlog{v}})
|
||||
}
|
||||
for k, v := range field2Stats {
|
||||
fieldStats = append(fieldStats, &datapb.FieldBinlog{FieldID: k, Binlogs: []*datapb.Binlog{v}})
|
||||
}
|
||||
return fieldInsert, fieldStats, nil
|
||||
}
|
||||
|
||||
func reportImportFunc(node *DataNode) importutil.ReportFunc {
|
||||
return func(importResult *rootcoordpb.ImportResult) error {
|
||||
err := retry.Do(context.Background(), func() error {
|
||||
err := node.broker.ReportImport(context.Background(), importResult)
|
||||
if err != nil {
|
||||
log.Error("failed to report import state to RootCoord", zap.Error(err))
|
||||
}
|
||||
return err
|
||||
}, retry.Attempts(node.reportImportRetryTimes))
|
||||
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
func logDupFlush(cID, segID int64) {
|
||||
log.Info("segment is already being flushed, ignoring flush request",
|
||||
zap.Int64("collectionID", cID),
|
||||
zap.Int64("segmentID", segID))
|
||||
}
|
||||
|
||||
func (node *DataNode) PreImport(ctx context.Context, req *datapb.PreImportRequest) (*commonpb.Status, error) {
|
||||
log := log.Ctx(ctx).With(zap.Int64("taskID", req.GetTaskID()),
|
||||
zap.Int64("jobID", req.GetJobID()),
|
||||
|
|
|
@ -19,12 +19,10 @@ package datanode
|
|||
import (
|
||||
"context"
|
||||
"math/rand"
|
||||
"path/filepath"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
@ -41,7 +39,6 @@ import (
|
|||
"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/importutil"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
|
@ -67,8 +64,6 @@ func TestDataNodeServicesSuite(t *testing.T) {
|
|||
}
|
||||
|
||||
func (s *DataNodeServicesSuite) SetupSuite() {
|
||||
importutil.ReportImportAttempts = 1
|
||||
|
||||
s.ctx, s.cancel = context.WithCancel(context.Background())
|
||||
etcdCli, err := etcd.GetEtcdClient(
|
||||
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
|
||||
|
@ -510,344 +505,6 @@ func (s *DataNodeServicesSuite) TestGetMetrics() {
|
|||
zap.String("response", resp.Response))
|
||||
}
|
||||
|
||||
func (s *DataNodeServicesSuite) TestImport() {
|
||||
s.node.rootCoord = &RootCoordFactory{
|
||||
collectionID: 100,
|
||||
pkType: schemapb.DataType_Int64,
|
||||
}
|
||||
|
||||
content := []byte(`{
|
||||
"rows":[
|
||||
{"bool_field": true, "int8_field": 10, "int16_field": 101, "int32_field": 1001, "int64_field": 10001, "float32_field": 3.14, "float64_field": 1.56, "varChar_field": "hello world", "binary_vector_field": [254, 0, 254, 0], "float_vector_field": [1.1, 1.2]},
|
||||
{"bool_field": false, "int8_field": 11, "int16_field": 102, "int32_field": 1002, "int64_field": 10002, "float32_field": 3.15, "float64_field": 2.56, "varChar_field": "hello world", "binary_vector_field": [253, 0, 253, 0], "float_vector_field": [2.1, 2.2]},
|
||||
{"bool_field": true, "int8_field": 12, "int16_field": 103, "int32_field": 1003, "int64_field": 10003, "float32_field": 3.16, "float64_field": 3.56, "varChar_field": "hello world", "binary_vector_field": [252, 0, 252, 0], "float_vector_field": [3.1, 3.2]},
|
||||
{"bool_field": false, "int8_field": 13, "int16_field": 104, "int32_field": 1004, "int64_field": 10004, "float32_field": 3.17, "float64_field": 4.56, "varChar_field": "hello world", "binary_vector_field": [251, 0, 251, 0], "float_vector_field": [4.1, 4.2]},
|
||||
{"bool_field": true, "int8_field": 14, "int16_field": 105, "int32_field": 1005, "int64_field": 10005, "float32_field": 3.18, "float64_field": 5.56, "varChar_field": "hello world", "binary_vector_field": [250, 0, 250, 0], "float_vector_field": [5.1, 5.2]}
|
||||
]
|
||||
}`)
|
||||
filePath := filepath.Join(s.node.chunkManager.RootPath(), "rows_1.json")
|
||||
err := s.node.chunkManager.Write(s.ctx, filePath, content)
|
||||
s.Require().NoError(err)
|
||||
|
||||
s.node.reportImportRetryTimes = 1 // save test time cost from 440s to 180s
|
||||
s.Run("test normal", func() {
|
||||
defer func() {
|
||||
s.TearDownTest()
|
||||
}()
|
||||
chName1 := "fake-by-dev-rootcoord-dml-testimport-1"
|
||||
chName2 := "fake-by-dev-rootcoord-dml-testimport-2"
|
||||
err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, &datapb.VchannelInfo{
|
||||
CollectionID: 100,
|
||||
ChannelName: chName1,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
FlushedSegmentIds: []int64{},
|
||||
}, &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, genTestTickler())
|
||||
s.Require().Nil(err)
|
||||
err = s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, &datapb.VchannelInfo{
|
||||
CollectionID: 100,
|
||||
ChannelName: chName2,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
FlushedSegmentIds: []int64{},
|
||||
}, &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, genTestTickler())
|
||||
s.Require().Nil(err)
|
||||
|
||||
_, ok := s.node.flowgraphManager.GetFlowgraphService(chName1)
|
||||
s.Require().True(ok)
|
||||
_, ok = s.node.flowgraphManager.GetFlowgraphService(chName2)
|
||||
s.Require().True(ok)
|
||||
|
||||
req := &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
ChannelNames: []string{chName1, chName2},
|
||||
Files: []string{filePath},
|
||||
RowBased: true,
|
||||
},
|
||||
}
|
||||
|
||||
s.broker.EXPECT().ReportImport(mock.Anything, mock.Anything).Return(nil)
|
||||
s.broker.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(nil)
|
||||
s.broker.EXPECT().AssignSegmentID(mock.Anything, mock.Anything).
|
||||
Return([]int64{10001}, nil)
|
||||
s.broker.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
s.node.Import(s.ctx, req)
|
||||
|
||||
stat, err := s.node.Import(context.WithValue(s.ctx, ctxKey{}, ""), req)
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().True(merr.Ok(stat))
|
||||
s.Assert().Equal("", stat.GetReason())
|
||||
|
||||
reqWithoutPartition := &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
ChannelNames: []string{chName1, chName2},
|
||||
Files: []string{filePath},
|
||||
RowBased: true,
|
||||
},
|
||||
}
|
||||
stat2, err := s.node.Import(context.WithValue(s.ctx, ctxKey{}, ""), reqWithoutPartition)
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().False(merr.Ok(stat2))
|
||||
})
|
||||
|
||||
s.Run("Test Import bad flow graph", func() {
|
||||
s.SetupTest()
|
||||
defer func() {
|
||||
s.TearDownTest()
|
||||
}()
|
||||
chName1 := "fake-by-dev-rootcoord-dml-testimport-1-badflowgraph"
|
||||
chName2 := "fake-by-dev-rootcoord-dml-testimport-2-badflowgraph"
|
||||
err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, &datapb.VchannelInfo{
|
||||
CollectionID: 100,
|
||||
ChannelName: chName1,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
FlushedSegmentIds: []int64{},
|
||||
}, &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, genTestTickler())
|
||||
s.Require().Nil(err)
|
||||
err = s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, &datapb.VchannelInfo{
|
||||
CollectionID: 999, // wrong collection ID.
|
||||
ChannelName: chName2,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
FlushedSegmentIds: []int64{},
|
||||
}, &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, genTestTickler())
|
||||
s.Require().Nil(err)
|
||||
|
||||
_, ok := s.node.flowgraphManager.GetFlowgraphService(chName1)
|
||||
s.Require().True(ok)
|
||||
_, ok = s.node.flowgraphManager.GetFlowgraphService(chName2)
|
||||
s.Require().True(ok)
|
||||
|
||||
s.broker.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(nil)
|
||||
s.broker.EXPECT().ReportImport(mock.Anything, mock.Anything).Return(nil)
|
||||
s.broker.EXPECT().AssignSegmentID(mock.Anything, mock.Anything).
|
||||
Return([]int64{10001}, nil)
|
||||
s.broker.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
req := &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
ChannelNames: []string{chName1, chName2},
|
||||
Files: []string{filePath},
|
||||
RowBased: true,
|
||||
},
|
||||
}
|
||||
stat, err := s.node.Import(context.WithValue(s.ctx, ctxKey{}, ""), req)
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().True(merr.Ok(stat))
|
||||
s.Assert().Equal("", stat.GetReason())
|
||||
})
|
||||
s.Run("test_Import_report_import_error", func() {
|
||||
s.SetupTest()
|
||||
s.node.reportImportRetryTimes = 1
|
||||
defer func() {
|
||||
s.TearDownTest()
|
||||
}()
|
||||
|
||||
s.broker.EXPECT().AssignSegmentID(mock.Anything, mock.Anything).
|
||||
Return([]int64{10001}, nil)
|
||||
s.broker.EXPECT().ReportImport(mock.Anything, mock.Anything).Return(errors.New("mocked"))
|
||||
s.broker.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(nil)
|
||||
s.broker.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
req := &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
ChannelNames: []string{"ch1", "ch2"},
|
||||
Files: []string{filePath},
|
||||
RowBased: true,
|
||||
},
|
||||
}
|
||||
stat, err := s.node.Import(s.ctx, req)
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().False(merr.Ok(stat))
|
||||
})
|
||||
|
||||
s.Run("test_import_error", func() {
|
||||
s.SetupTest()
|
||||
defer func() {
|
||||
s.TearDownTest()
|
||||
}()
|
||||
s.broker.ExpectedCalls = nil
|
||||
s.broker.EXPECT().DescribeCollection(mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(&milvuspb.DescribeCollectionResponse{
|
||||
Status: merr.Status(merr.WrapErrCollectionNotFound("collection")),
|
||||
}, nil)
|
||||
s.broker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).
|
||||
Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
s.broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
s.broker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
s.broker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
s.broker.EXPECT().AllocTimestamp(mock.Anything, mock.Anything).Call.Return(tsoutil.ComposeTSByTime(time.Now(), 0),
|
||||
func(_ context.Context, num uint32) uint32 { return num }, nil).Maybe()
|
||||
|
||||
s.broker.EXPECT().ReportImport(mock.Anything, mock.Anything).Return(nil)
|
||||
req := &datapb.ImportTaskRequest{
|
||||
ImportTask: &datapb.ImportTask{
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
},
|
||||
}
|
||||
stat, err := s.node.Import(context.WithValue(s.ctx, ctxKey{}, ""), req)
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().False(merr.Ok(stat))
|
||||
|
||||
stat, err = s.node.Import(context.WithValue(s.ctx, ctxKey{}, returnError), req)
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().False(merr.Ok(stat))
|
||||
|
||||
s.node.stateCode.Store(commonpb.StateCode_Abnormal)
|
||||
stat, err = s.node.Import(context.WithValue(s.ctx, ctxKey{}, ""), req)
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().False(merr.Ok(stat))
|
||||
})
|
||||
}
|
||||
|
||||
func (s *DataNodeServicesSuite) TestAddImportSegment() {
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64},
|
||||
{FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64},
|
||||
{FieldID: common.StartOfUserFieldID, DataType: schemapb.DataType_Int64, IsPrimaryKey: true, Name: "pk"},
|
||||
{FieldID: common.StartOfUserFieldID + 1, DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
}},
|
||||
},
|
||||
}
|
||||
s.Run("test AddSegment", func() {
|
||||
s.node.rootCoord = &RootCoordFactory{
|
||||
collectionID: 100,
|
||||
pkType: schemapb.DataType_Int64,
|
||||
}
|
||||
|
||||
chName1 := "fake-by-dev-rootcoord-dml-testaddsegment-1"
|
||||
chName2 := "fake-by-dev-rootcoord-dml-testaddsegment-2"
|
||||
err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, &datapb.VchannelInfo{
|
||||
CollectionID: 100,
|
||||
ChannelName: chName1,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
FlushedSegmentIds: []int64{},
|
||||
}, schema, genTestTickler())
|
||||
s.Require().NoError(err)
|
||||
err = s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, &datapb.VchannelInfo{
|
||||
CollectionID: 100,
|
||||
ChannelName: chName2,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
FlushedSegmentIds: []int64{},
|
||||
}, schema, genTestTickler())
|
||||
s.Require().NoError(err)
|
||||
|
||||
_, ok := s.node.flowgraphManager.GetFlowgraphService(chName1)
|
||||
s.Assert().True(ok)
|
||||
_, ok = s.node.flowgraphManager.GetFlowgraphService(chName2)
|
||||
s.Assert().True(ok)
|
||||
|
||||
resp, err := s.node.AddImportSegment(context.WithValue(s.ctx, ctxKey{}, ""), &datapb.AddImportSegmentRequest{
|
||||
SegmentId: 100,
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
ChannelName: chName1,
|
||||
RowNum: 500,
|
||||
})
|
||||
s.Assert().NoError(err)
|
||||
s.Assert().True(merr.Ok(resp.GetStatus()))
|
||||
s.Assert().Equal("", resp.GetStatus().GetReason())
|
||||
|
||||
getFlowGraphServiceAttempts = 3
|
||||
resp, err = s.node.AddImportSegment(context.WithValue(s.ctx, ctxKey{}, ""), &datapb.AddImportSegmentRequest{
|
||||
SegmentId: 100,
|
||||
CollectionId: 100,
|
||||
PartitionId: 100,
|
||||
ChannelName: "bad-ch-name",
|
||||
RowNum: 500,
|
||||
})
|
||||
s.Assert().NoError(err)
|
||||
// TODO ASSERT COMBINE ERROR
|
||||
s.Assert().False(merr.Ok(resp.GetStatus()))
|
||||
// s.Assert().Equal(merr.Code(merr.ErrChannelNotFound), stat.GetStatus().GetCode())
|
||||
})
|
||||
}
|
||||
|
||||
func (s *DataNodeServicesSuite) TestSyncSegments() {
|
||||
chanName := "fake-by-dev-rootcoord-dml-test-syncsegments-1"
|
||||
schema := &schemapb.CollectionSchema{
|
||||
|
|
|
@ -470,18 +470,6 @@ func (c *Client) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStat
|
|||
})
|
||||
}
|
||||
|
||||
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
||||
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
|
||||
req = typeutil.Clone(req)
|
||||
commonpbutil.UpdateMsgBase(
|
||||
req.GetBase(),
|
||||
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.grpcClient.GetNodeID())),
|
||||
)
|
||||
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*datapb.ImportTaskResponse, error) {
|
||||
return client.Import(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
// UpdateSegmentStatistics is the client side caller of UpdateSegmentStatistics.
|
||||
func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
req = typeutil.Clone(req)
|
||||
|
@ -506,29 +494,6 @@ func (c *Client) UpdateChannelCheckpoint(ctx context.Context, req *datapb.Update
|
|||
})
|
||||
}
|
||||
|
||||
// SaveImportSegment is the DataCoord client side code for SaveImportSegment call.
|
||||
func (c *Client) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
req = typeutil.Clone(req)
|
||||
commonpbutil.UpdateMsgBase(
|
||||
req.GetBase(),
|
||||
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.grpcClient.GetNodeID())),
|
||||
)
|
||||
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*commonpb.Status, error) {
|
||||
return client.SaveImportSegment(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
func (c *Client) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
req = typeutil.Clone(req)
|
||||
commonpbutil.UpdateMsgBase(
|
||||
req.GetBase(),
|
||||
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.grpcClient.GetNodeID())),
|
||||
)
|
||||
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*commonpb.Status, error) {
|
||||
return client.UnsetIsImportingState(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
func (c *Client) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
req = typeutil.Clone(req)
|
||||
commonpbutil.UpdateMsgBase(
|
||||
|
|
|
@ -1413,58 +1413,6 @@ func Test_SetSegmentState(t *testing.T) {
|
|||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
}
|
||||
|
||||
func Test_Import(t *testing.T) {
|
||||
paramtable.Init()
|
||||
|
||||
ctx := context.Background()
|
||||
client, err := NewClient(ctx)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, client)
|
||||
defer client.Close()
|
||||
|
||||
mockDC := mocks.NewMockDataCoordClient(t)
|
||||
mockGrpcClient := mocks.NewMockGrpcClient[datapb.DataCoordClient](t)
|
||||
mockGrpcClient.EXPECT().Close().Return(nil)
|
||||
mockGrpcClient.EXPECT().GetNodeID().Return(1)
|
||||
mockGrpcClient.EXPECT().ReCall(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, f func(datapb.DataCoordClient) (interface{}, error)) (interface{}, error) {
|
||||
return f(mockDC)
|
||||
})
|
||||
client.(*Client).grpcClient = mockGrpcClient
|
||||
|
||||
// test success
|
||||
mockDC.EXPECT().Import(mock.Anything, mock.Anything).Return(&datapb.ImportTaskResponse{
|
||||
Status: merr.Success(),
|
||||
}, nil)
|
||||
_, err = client.Import(ctx, &datapb.ImportTaskRequest{})
|
||||
assert.Nil(t, err)
|
||||
|
||||
// test return error status
|
||||
mockDC.ExpectedCalls = nil
|
||||
mockDC.EXPECT().Import(mock.Anything, mock.Anything).Return(&datapb.ImportTaskResponse{
|
||||
Status: merr.Status(merr.ErrServiceNotReady),
|
||||
}, nil)
|
||||
|
||||
rsp, err := client.Import(ctx, &datapb.ImportTaskRequest{})
|
||||
assert.NotEqual(t, int32(0), rsp.GetStatus().GetCode())
|
||||
assert.Nil(t, err)
|
||||
|
||||
// test return error
|
||||
mockDC.ExpectedCalls = nil
|
||||
mockDC.EXPECT().Import(mock.Anything, mock.Anything).Return(&datapb.ImportTaskResponse{
|
||||
Status: merr.Success(),
|
||||
}, mockErr)
|
||||
|
||||
_, err = client.Import(ctx, &datapb.ImportTaskRequest{})
|
||||
assert.NotNil(t, err)
|
||||
|
||||
// test ctx done
|
||||
ctx, cancel := context.WithTimeout(ctx, 10*time.Millisecond)
|
||||
defer cancel()
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
_, err = client.Import(ctx, &datapb.ImportTaskRequest{})
|
||||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
}
|
||||
|
||||
func Test_UpdateSegmentStatistics(t *testing.T) {
|
||||
paramtable.Init()
|
||||
|
||||
|
@ -1557,100 +1505,6 @@ func Test_UpdateChannelCheckpoint(t *testing.T) {
|
|||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
}
|
||||
|
||||
func Test_SaveImportSegment(t *testing.T) {
|
||||
paramtable.Init()
|
||||
|
||||
ctx := context.Background()
|
||||
client, err := NewClient(ctx)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, client)
|
||||
defer client.Close()
|
||||
|
||||
mockDC := mocks.NewMockDataCoordClient(t)
|
||||
mockGrpcClient := mocks.NewMockGrpcClient[datapb.DataCoordClient](t)
|
||||
mockGrpcClient.EXPECT().Close().Return(nil)
|
||||
mockGrpcClient.EXPECT().GetNodeID().Return(1)
|
||||
mockGrpcClient.EXPECT().ReCall(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, f func(datapb.DataCoordClient) (interface{}, error)) (interface{}, error) {
|
||||
return f(mockDC)
|
||||
})
|
||||
client.(*Client).grpcClient = mockGrpcClient
|
||||
|
||||
// test success
|
||||
mockDC.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).Return(merr.Success(), nil)
|
||||
_, err = client.SaveImportSegment(ctx, &datapb.SaveImportSegmentRequest{})
|
||||
assert.Nil(t, err)
|
||||
|
||||
// test return error status
|
||||
mockDC.ExpectedCalls = nil
|
||||
mockDC.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).Return(
|
||||
merr.Status(merr.ErrServiceNotReady), nil)
|
||||
|
||||
rsp, err := client.SaveImportSegment(ctx, &datapb.SaveImportSegmentRequest{})
|
||||
assert.NotEqual(t, int32(0), rsp.GetCode())
|
||||
assert.Nil(t, err)
|
||||
|
||||
// test return error
|
||||
mockDC.ExpectedCalls = nil
|
||||
mockDC.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).Return(merr.Success(), mockErr)
|
||||
|
||||
_, err = client.SaveImportSegment(ctx, &datapb.SaveImportSegmentRequest{})
|
||||
assert.NotNil(t, err)
|
||||
|
||||
// test ctx done
|
||||
ctx, cancel := context.WithTimeout(ctx, 10*time.Millisecond)
|
||||
defer cancel()
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
_, err = client.SaveImportSegment(ctx, &datapb.SaveImportSegmentRequest{})
|
||||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
}
|
||||
|
||||
func Test_UnsetIsImportingState(t *testing.T) {
|
||||
paramtable.Init()
|
||||
|
||||
ctx := context.Background()
|
||||
client, err := NewClient(ctx)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, client)
|
||||
defer client.Close()
|
||||
|
||||
mockDC := mocks.NewMockDataCoordClient(t)
|
||||
mockGrpcClient := mocks.NewMockGrpcClient[datapb.DataCoordClient](t)
|
||||
mockGrpcClient.EXPECT().Close().Return(nil)
|
||||
mockGrpcClient.EXPECT().GetNodeID().Return(1)
|
||||
mockGrpcClient.EXPECT().ReCall(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, f func(datapb.DataCoordClient) (interface{}, error)) (interface{}, error) {
|
||||
return f(mockDC)
|
||||
})
|
||||
client.(*Client).grpcClient = mockGrpcClient
|
||||
|
||||
// test success
|
||||
mockDC.EXPECT().UnsetIsImportingState(mock.Anything, mock.Anything).Return(merr.Success(), nil)
|
||||
_, err = client.UnsetIsImportingState(ctx, &datapb.UnsetIsImportingStateRequest{})
|
||||
assert.Nil(t, err)
|
||||
|
||||
// test return error status
|
||||
mockDC.ExpectedCalls = nil
|
||||
mockDC.EXPECT().UnsetIsImportingState(mock.Anything, mock.Anything).Return(merr.Status(merr.ErrServiceNotReady), nil)
|
||||
|
||||
rsp, err := client.UnsetIsImportingState(ctx, &datapb.UnsetIsImportingStateRequest{})
|
||||
assert.NotEqual(t, int32(0), rsp.GetCode())
|
||||
assert.Nil(t, err)
|
||||
|
||||
// test return error
|
||||
mockDC.ExpectedCalls = nil
|
||||
mockDC.EXPECT().UnsetIsImportingState(mock.Anything, mock.Anything).Return(
|
||||
merr.Success(), mockErr)
|
||||
|
||||
_, err = client.UnsetIsImportingState(ctx, &datapb.UnsetIsImportingStateRequest{})
|
||||
assert.NotNil(t, err)
|
||||
|
||||
// test ctx done
|
||||
ctx, cancel := context.WithTimeout(ctx, 10*time.Millisecond)
|
||||
defer cancel()
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
_, err = client.UnsetIsImportingState(ctx, &datapb.UnsetIsImportingStateRequest{})
|
||||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||
}
|
||||
|
||||
func Test_MarkSegmentsDropped(t *testing.T) {
|
||||
paramtable.Init()
|
||||
|
||||
|
|
|
@ -395,11 +395,6 @@ func (s *Server) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStat
|
|||
return s.dataCoord.SetSegmentState(ctx, req)
|
||||
}
|
||||
|
||||
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
||||
func (s *Server) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
return s.dataCoord.Import(ctx, req)
|
||||
}
|
||||
|
||||
// UpdateSegmentStatistics is the dataCoord service caller of UpdateSegmentStatistics.
|
||||
func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
|
||||
return s.dataCoord.UpdateSegmentStatistics(ctx, req)
|
||||
|
@ -410,17 +405,6 @@ func (s *Server) UpdateChannelCheckpoint(ctx context.Context, req *datapb.Update
|
|||
return s.dataCoord.UpdateChannelCheckpoint(ctx, req)
|
||||
}
|
||||
|
||||
// SaveImportSegment saves the import segment binlog paths data and then looks for the right DataNode to add the
|
||||
// segment to that DataNode.
|
||||
func (s *Server) SaveImportSegment(ctx context.Context, request *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
|
||||
return s.dataCoord.SaveImportSegment(ctx, request)
|
||||
}
|
||||
|
||||
// UnsetIsImportingState is the distributed caller of UnsetIsImportingState.
|
||||
func (s *Server) UnsetIsImportingState(ctx context.Context, request *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
return s.dataCoord.UnsetIsImportingState(ctx, request)
|
||||
}
|
||||
|
||||
// MarkSegmentsDropped is the distributed caller of MarkSegmentsDropped.
|
||||
func (s *Server) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest) (*commonpb.Status, error) {
|
||||
return s.dataCoord.MarkSegmentsDropped(ctx, req)
|
||||
|
|
|
@ -213,13 +213,6 @@ func Test_NewServer(t *testing.T) {
|
|||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("import", func(t *testing.T) {
|
||||
mockDataCoord.EXPECT().Import(mock.Anything, mock.Anything).Return(&datapb.ImportTaskResponse{}, nil)
|
||||
resp, err := server.Import(ctx, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("UpdateSegmentStatistics", func(t *testing.T) {
|
||||
mockDataCoord.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(merr.Success(), nil)
|
||||
resp, err := server.UpdateSegmentStatistics(ctx, nil)
|
||||
|
@ -234,20 +227,6 @@ func Test_NewServer(t *testing.T) {
|
|||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("SaveImportSegment", func(t *testing.T) {
|
||||
mockDataCoord.EXPECT().SaveImportSegment(mock.Anything, mock.Anything).Return(merr.Success(), nil)
|
||||
resp, err := server.SaveImportSegment(ctx, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("UnsetIsImportingState", func(t *testing.T) {
|
||||
mockDataCoord.EXPECT().UnsetIsImportingState(mock.Anything, mock.Anything).Return(merr.Success(), nil)
|
||||
resp, err := server.UnsetIsImportingState(ctx, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("MarkSegmentsDropped", func(t *testing.T) {
|
||||
mockDataCoord.EXPECT().MarkSegmentsDropped(mock.Anything, mock.Anything).Return(merr.Success(), nil)
|
||||
resp, err := server.MarkSegmentsDropped(ctx, nil)
|
||||
|
|
|
@ -190,17 +190,6 @@ func (c *Client) GetCompactionState(ctx context.Context, req *datapb.CompactionS
|
|||
})
|
||||
}
|
||||
|
||||
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
||||
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
req = typeutil.Clone(req)
|
||||
commonpbutil.UpdateMsgBase(
|
||||
req.GetBase(),
|
||||
commonpbutil.FillMsgBaseFromClient(c.serverID))
|
||||
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*commonpb.Status, error) {
|
||||
return client.Import(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
func (c *Client) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest, opts ...grpc.CallOption) (*datapb.ResendSegmentStatsResponse, error) {
|
||||
req = typeutil.Clone(req)
|
||||
commonpbutil.UpdateMsgBase(
|
||||
|
@ -211,17 +200,6 @@ func (c *Client) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegme
|
|||
})
|
||||
}
|
||||
|
||||
// AddImportSegment is the DataNode client side code for AddImportSegment call.
|
||||
func (c *Client) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest, opts ...grpc.CallOption) (*datapb.AddImportSegmentResponse, error) {
|
||||
req = typeutil.Clone(req)
|
||||
commonpbutil.UpdateMsgBase(
|
||||
req.GetBase(),
|
||||
commonpbutil.FillMsgBaseFromClient(c.serverID))
|
||||
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*datapb.AddImportSegmentResponse, error) {
|
||||
return client.AddImportSegment(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
// SyncSegments is the DataNode client side code for SyncSegments call.
|
||||
func (c *Client) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*commonpb.Status, error) {
|
||||
|
|
|
@ -69,15 +69,9 @@ func Test_NewClient(t *testing.T) {
|
|||
r6, err := client.Compaction(ctx, nil)
|
||||
retCheck(retNotNil, r6, err)
|
||||
|
||||
r7, err := client.Import(ctx, nil)
|
||||
retCheck(retNotNil, r7, err)
|
||||
|
||||
r8, err := client.ResendSegmentStats(ctx, nil)
|
||||
retCheck(retNotNil, r8, err)
|
||||
|
||||
r9, err := client.AddImportSegment(ctx, nil)
|
||||
retCheck(retNotNil, r9, err)
|
||||
|
||||
r10, err := client.ShowConfigurations(ctx, nil)
|
||||
retCheck(retNotNil, r10, err)
|
||||
|
||||
|
|
|
@ -363,18 +363,10 @@ func (s *Server) GetCompactionState(ctx context.Context, request *datapb.Compact
|
|||
return s.datanode.GetCompactionState(ctx, request)
|
||||
}
|
||||
|
||||
func (s *Server) Import(ctx context.Context, request *datapb.ImportTaskRequest) (*commonpb.Status, error) {
|
||||
return s.datanode.Import(ctx, request)
|
||||
}
|
||||
|
||||
func (s *Server) ResendSegmentStats(ctx context.Context, request *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
|
||||
return s.datanode.ResendSegmentStats(ctx, request)
|
||||
}
|
||||
|
||||
func (s *Server) AddImportSegment(ctx context.Context, request *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
return s.datanode.AddImportSegment(ctx, request)
|
||||
}
|
||||
|
||||
func (s *Server) SyncSegments(ctx context.Context, request *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
|
||||
return s.datanode.SyncSegments(ctx, request)
|
||||
}
|
||||
|
|
|
@ -40,20 +40,19 @@ import (
|
|||
type MockDataNode struct {
|
||||
nodeID typeutil.UniqueID
|
||||
|
||||
stateCode commonpb.StateCode
|
||||
states *milvuspb.ComponentStates
|
||||
status *commonpb.Status
|
||||
err error
|
||||
initErr error
|
||||
startErr error
|
||||
stopErr error
|
||||
regErr error
|
||||
strResp *milvuspb.StringResponse
|
||||
configResp *internalpb.ShowConfigurationsResponse
|
||||
metricResp *milvuspb.GetMetricsResponse
|
||||
resendResp *datapb.ResendSegmentStatsResponse
|
||||
addImportSegmentResp *datapb.AddImportSegmentResponse
|
||||
compactionResp *datapb.CompactionStateResponse
|
||||
stateCode commonpb.StateCode
|
||||
states *milvuspb.ComponentStates
|
||||
status *commonpb.Status
|
||||
err error
|
||||
initErr error
|
||||
startErr error
|
||||
stopErr error
|
||||
regErr error
|
||||
strResp *milvuspb.StringResponse
|
||||
configResp *internalpb.ShowConfigurationsResponse
|
||||
metricResp *milvuspb.GetMetricsResponse
|
||||
resendResp *datapb.ResendSegmentStatsResponse
|
||||
compactionResp *datapb.CompactionStateResponse
|
||||
}
|
||||
|
||||
func (m *MockDataNode) Init() error {
|
||||
|
@ -138,18 +137,10 @@ func (m *MockDataNode) GetCompactionState(ctx context.Context, req *datapb.Compa
|
|||
func (m *MockDataNode) SetEtcdClient(client *clientv3.Client) {
|
||||
}
|
||||
|
||||
func (m *MockDataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
|
||||
return m.status, m.err
|
||||
}
|
||||
|
||||
func (m *MockDataNode) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
|
||||
return m.resendResp, m.err
|
||||
}
|
||||
|
||||
func (m *MockDataNode) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
return m.addImportSegmentResp, m.err
|
||||
}
|
||||
|
||||
func (m *MockDataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
|
||||
return m.status, m.err
|
||||
}
|
||||
|
@ -295,15 +286,6 @@ func Test_NewServer(t *testing.T) {
|
|||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("Import", func(t *testing.T) {
|
||||
server.datanode = &MockDataNode{
|
||||
status: &commonpb.Status{},
|
||||
}
|
||||
resp, err := server.Import(ctx, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("ResendSegmentStats", func(t *testing.T) {
|
||||
server.datanode = &MockDataNode{
|
||||
resendResp: &datapb.ResendSegmentStatsResponse{},
|
||||
|
@ -313,18 +295,6 @@ func Test_NewServer(t *testing.T) {
|
|||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("add segment", func(t *testing.T) {
|
||||
server.datanode = &MockDataNode{
|
||||
status: &commonpb.Status{},
|
||||
addImportSegmentResp: &datapb.AddImportSegmentResponse{
|
||||
Status: merr.Success(),
|
||||
},
|
||||
}
|
||||
resp, err := server.AddImportSegment(ctx, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("NotifyChannelOperation", func(t *testing.T) {
|
||||
server.datanode = &MockDataNode{
|
||||
status: &commonpb.Status{},
|
||||
|
|
|
@ -442,34 +442,6 @@ func (c *Client) ListAliases(ctx context.Context, req *milvuspb.ListAliasesReque
|
|||
})
|
||||
}
|
||||
|
||||
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
||||
func (c *Client) Import(ctx context.Context, req *milvuspb.ImportRequest, opts ...grpc.CallOption) (*milvuspb.ImportResponse, error) {
|
||||
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*milvuspb.ImportResponse, error) {
|
||||
return client.Import(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
// Check import task state from datanode
|
||||
func (c *Client) GetImportState(ctx context.Context, req *milvuspb.GetImportStateRequest, opts ...grpc.CallOption) (*milvuspb.GetImportStateResponse, error) {
|
||||
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*milvuspb.GetImportStateResponse, error) {
|
||||
return client.GetImportState(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
// List id array of all import tasks
|
||||
func (c *Client) ListImportTasks(ctx context.Context, req *milvuspb.ListImportTasksRequest, opts ...grpc.CallOption) (*milvuspb.ListImportTasksResponse, error) {
|
||||
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*milvuspb.ListImportTasksResponse, error) {
|
||||
return client.ListImportTasks(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
// Report impot task state to rootcoord
|
||||
func (c *Client) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*commonpb.Status, error) {
|
||||
return client.ReportImport(ctx, req)
|
||||
})
|
||||
}
|
||||
|
||||
func (c *Client) CreateCredential(ctx context.Context, req *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*commonpb.Status, error) {
|
||||
return client.CreateCredential(ctx, req)
|
||||
|
|
|
@ -160,18 +160,6 @@ func Test_NewClient(t *testing.T) {
|
|||
r, err := client.ListAliases(ctx, nil)
|
||||
retCheck(retNotNil, r, err)
|
||||
}
|
||||
{
|
||||
r, err := client.Import(ctx, nil)
|
||||
retCheck(retNotNil, r, err)
|
||||
}
|
||||
{
|
||||
r, err := client.GetImportState(ctx, nil)
|
||||
retCheck(retNotNil, r, err)
|
||||
}
|
||||
{
|
||||
r, err := client.ReportImport(ctx, nil)
|
||||
retCheck(retNotNil, r, err)
|
||||
}
|
||||
{
|
||||
r, err := client.CreateCredential(ctx, nil)
|
||||
retCheck(retNotNil, r, err)
|
||||
|
@ -382,18 +370,6 @@ func Test_NewClient(t *testing.T) {
|
|||
rTimeout, err := client.ListAliases(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
}
|
||||
{
|
||||
rTimeout, err := client.Import(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
}
|
||||
{
|
||||
rTimeout, err := client.GetImportState(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
}
|
||||
{
|
||||
rTimeout, err := client.ReportImport(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
}
|
||||
{
|
||||
rTimeout, err := client.CreateCredential(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
|
@ -414,10 +390,6 @@ func Test_NewClient(t *testing.T) {
|
|||
rTimeout, err := client.ListCredUsers(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
}
|
||||
{
|
||||
rTimeout, err := client.ListImportTasks(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
}
|
||||
{
|
||||
rTimeout, err := client.InvalidateCollectionMetaCache(shortCtx, nil)
|
||||
retCheck(rTimeout, err)
|
||||
|
|
|
@ -461,26 +461,6 @@ func (s *Server) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest)
|
|||
return s.rootCoord.GetMetrics(ctx, in)
|
||||
}
|
||||
|
||||
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
||||
func (s *Server) Import(ctx context.Context, in *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error) {
|
||||
return s.rootCoord.Import(ctx, in)
|
||||
}
|
||||
|
||||
// Check import task state from datanode
|
||||
func (s *Server) GetImportState(ctx context.Context, in *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error) {
|
||||
return s.rootCoord.GetImportState(ctx, in)
|
||||
}
|
||||
|
||||
// Returns id array of all import tasks
|
||||
func (s *Server) ListImportTasks(ctx context.Context, in *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error) {
|
||||
return s.rootCoord.ListImportTasks(ctx, in)
|
||||
}
|
||||
|
||||
// Report impot task state to datacoord
|
||||
func (s *Server) ReportImport(ctx context.Context, in *rootcoordpb.ImportResult) (*commonpb.Status, error) {
|
||||
return s.rootCoord.ReportImport(ctx, in)
|
||||
}
|
||||
|
||||
func (s *Server) CreateCredential(ctx context.Context, request *internalpb.CredentialInfo) (*commonpb.Status, error) {
|
||||
return s.rootCoord.CreateCredential(ctx, request)
|
||||
}
|
||||
|
|
|
@ -68,8 +68,8 @@ type MockDataCoord_AlterIndex_Call struct {
|
|||
}
|
||||
|
||||
// AlterIndex is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.AlterIndexRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.AlterIndexRequest
|
||||
func (_e *MockDataCoord_Expecter) AlterIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_AlterIndex_Call {
|
||||
return &MockDataCoord_AlterIndex_Call{Call: _e.mock.On("AlterIndex", _a0, _a1)}
|
||||
}
|
||||
|
@ -123,8 +123,8 @@ type MockDataCoord_AssignSegmentID_Call struct {
|
|||
}
|
||||
|
||||
// AssignSegmentID is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.AssignSegmentIDRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.AssignSegmentIDRequest
|
||||
func (_e *MockDataCoord_Expecter) AssignSegmentID(_a0 interface{}, _a1 interface{}) *MockDataCoord_AssignSegmentID_Call {
|
||||
return &MockDataCoord_AssignSegmentID_Call{Call: _e.mock.On("AssignSegmentID", _a0, _a1)}
|
||||
}
|
||||
|
@ -178,8 +178,8 @@ type MockDataCoord_BroadcastAlteredCollection_Call struct {
|
|||
}
|
||||
|
||||
// BroadcastAlteredCollection is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.AlterCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.AlterCollectionRequest
|
||||
func (_e *MockDataCoord_Expecter) BroadcastAlteredCollection(_a0 interface{}, _a1 interface{}) *MockDataCoord_BroadcastAlteredCollection_Call {
|
||||
return &MockDataCoord_BroadcastAlteredCollection_Call{Call: _e.mock.On("BroadcastAlteredCollection", _a0, _a1)}
|
||||
}
|
||||
|
@ -233,8 +233,8 @@ type MockDataCoord_CheckHealth_Call struct {
|
|||
}
|
||||
|
||||
// CheckHealth is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CheckHealthRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CheckHealthRequest
|
||||
func (_e *MockDataCoord_Expecter) CheckHealth(_a0 interface{}, _a1 interface{}) *MockDataCoord_CheckHealth_Call {
|
||||
return &MockDataCoord_CheckHealth_Call{Call: _e.mock.On("CheckHealth", _a0, _a1)}
|
||||
}
|
||||
|
@ -288,8 +288,8 @@ type MockDataCoord_CreateIndex_Call struct {
|
|||
}
|
||||
|
||||
// CreateIndex is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.CreateIndexRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.CreateIndexRequest
|
||||
func (_e *MockDataCoord_Expecter) CreateIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_CreateIndex_Call {
|
||||
return &MockDataCoord_CreateIndex_Call{Call: _e.mock.On("CreateIndex", _a0, _a1)}
|
||||
}
|
||||
|
@ -343,8 +343,8 @@ type MockDataCoord_DescribeIndex_Call struct {
|
|||
}
|
||||
|
||||
// DescribeIndex is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.DescribeIndexRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.DescribeIndexRequest
|
||||
func (_e *MockDataCoord_Expecter) DescribeIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_DescribeIndex_Call {
|
||||
return &MockDataCoord_DescribeIndex_Call{Call: _e.mock.On("DescribeIndex", _a0, _a1)}
|
||||
}
|
||||
|
@ -398,8 +398,8 @@ type MockDataCoord_DropIndex_Call struct {
|
|||
}
|
||||
|
||||
// DropIndex is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.DropIndexRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.DropIndexRequest
|
||||
func (_e *MockDataCoord_Expecter) DropIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_DropIndex_Call {
|
||||
return &MockDataCoord_DropIndex_Call{Call: _e.mock.On("DropIndex", _a0, _a1)}
|
||||
}
|
||||
|
@ -453,8 +453,8 @@ type MockDataCoord_DropVirtualChannel_Call struct {
|
|||
}
|
||||
|
||||
// DropVirtualChannel is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.DropVirtualChannelRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.DropVirtualChannelRequest
|
||||
func (_e *MockDataCoord_Expecter) DropVirtualChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_DropVirtualChannel_Call {
|
||||
return &MockDataCoord_DropVirtualChannel_Call{Call: _e.mock.On("DropVirtualChannel", _a0, _a1)}
|
||||
}
|
||||
|
@ -508,8 +508,8 @@ type MockDataCoord_Flush_Call struct {
|
|||
}
|
||||
|
||||
// Flush is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.FlushRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.FlushRequest
|
||||
func (_e *MockDataCoord_Expecter) Flush(_a0 interface{}, _a1 interface{}) *MockDataCoord_Flush_Call {
|
||||
return &MockDataCoord_Flush_Call{Call: _e.mock.On("Flush", _a0, _a1)}
|
||||
}
|
||||
|
@ -563,8 +563,8 @@ type MockDataCoord_GcConfirm_Call struct {
|
|||
}
|
||||
|
||||
// GcConfirm is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GcConfirmRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GcConfirmRequest
|
||||
func (_e *MockDataCoord_Expecter) GcConfirm(_a0 interface{}, _a1 interface{}) *MockDataCoord_GcConfirm_Call {
|
||||
return &MockDataCoord_GcConfirm_Call{Call: _e.mock.On("GcConfirm", _a0, _a1)}
|
||||
}
|
||||
|
@ -618,8 +618,8 @@ type MockDataCoord_GcControl_Call struct {
|
|||
}
|
||||
|
||||
// GcControl is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GcControlRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GcControlRequest
|
||||
func (_e *MockDataCoord_Expecter) GcControl(_a0 interface{}, _a1 interface{}) *MockDataCoord_GcControl_Call {
|
||||
return &MockDataCoord_GcControl_Call{Call: _e.mock.On("GcControl", _a0, _a1)}
|
||||
}
|
||||
|
@ -673,8 +673,8 @@ type MockDataCoord_GetCollectionStatistics_Call struct {
|
|||
}
|
||||
|
||||
// GetCollectionStatistics is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetCollectionStatisticsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetCollectionStatisticsRequest
|
||||
func (_e *MockDataCoord_Expecter) GetCollectionStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetCollectionStatistics_Call {
|
||||
return &MockDataCoord_GetCollectionStatistics_Call{Call: _e.mock.On("GetCollectionStatistics", _a0, _a1)}
|
||||
}
|
||||
|
@ -728,8 +728,8 @@ type MockDataCoord_GetCompactionState_Call struct {
|
|||
}
|
||||
|
||||
// GetCompactionState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetCompactionStateRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetCompactionStateRequest
|
||||
func (_e *MockDataCoord_Expecter) GetCompactionState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetCompactionState_Call {
|
||||
return &MockDataCoord_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", _a0, _a1)}
|
||||
}
|
||||
|
@ -783,8 +783,8 @@ type MockDataCoord_GetCompactionStateWithPlans_Call struct {
|
|||
}
|
||||
|
||||
// GetCompactionStateWithPlans is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetCompactionPlansRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetCompactionPlansRequest
|
||||
func (_e *MockDataCoord_Expecter) GetCompactionStateWithPlans(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetCompactionStateWithPlans_Call {
|
||||
return &MockDataCoord_GetCompactionStateWithPlans_Call{Call: _e.mock.On("GetCompactionStateWithPlans", _a0, _a1)}
|
||||
}
|
||||
|
@ -838,8 +838,8 @@ type MockDataCoord_GetComponentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetComponentStates is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetComponentStatesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetComponentStatesRequest
|
||||
func (_e *MockDataCoord_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetComponentStates_Call {
|
||||
return &MockDataCoord_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)}
|
||||
}
|
||||
|
@ -893,8 +893,8 @@ type MockDataCoord_GetFlushAllState_Call struct {
|
|||
}
|
||||
|
||||
// GetFlushAllState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetFlushAllStateRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetFlushAllStateRequest
|
||||
func (_e *MockDataCoord_Expecter) GetFlushAllState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetFlushAllState_Call {
|
||||
return &MockDataCoord_GetFlushAllState_Call{Call: _e.mock.On("GetFlushAllState", _a0, _a1)}
|
||||
}
|
||||
|
@ -948,8 +948,8 @@ type MockDataCoord_GetFlushState_Call struct {
|
|||
}
|
||||
|
||||
// GetFlushState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetFlushStateRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetFlushStateRequest
|
||||
func (_e *MockDataCoord_Expecter) GetFlushState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetFlushState_Call {
|
||||
return &MockDataCoord_GetFlushState_Call{Call: _e.mock.On("GetFlushState", _a0, _a1)}
|
||||
}
|
||||
|
@ -1003,8 +1003,8 @@ type MockDataCoord_GetFlushedSegments_Call struct {
|
|||
}
|
||||
|
||||
// GetFlushedSegments is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetFlushedSegmentsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetFlushedSegmentsRequest
|
||||
func (_e *MockDataCoord_Expecter) GetFlushedSegments(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetFlushedSegments_Call {
|
||||
return &MockDataCoord_GetFlushedSegments_Call{Call: _e.mock.On("GetFlushedSegments", _a0, _a1)}
|
||||
}
|
||||
|
@ -1058,8 +1058,8 @@ type MockDataCoord_GetImportProgress_Call struct {
|
|||
}
|
||||
|
||||
// GetImportProgress is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetImportProgressRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetImportProgressRequest
|
||||
func (_e *MockDataCoord_Expecter) GetImportProgress(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetImportProgress_Call {
|
||||
return &MockDataCoord_GetImportProgress_Call{Call: _e.mock.On("GetImportProgress", _a0, _a1)}
|
||||
}
|
||||
|
@ -1113,8 +1113,8 @@ type MockDataCoord_GetIndexBuildProgress_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexBuildProgress is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexBuildProgressRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexBuildProgressRequest
|
||||
func (_e *MockDataCoord_Expecter) GetIndexBuildProgress(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexBuildProgress_Call {
|
||||
return &MockDataCoord_GetIndexBuildProgress_Call{Call: _e.mock.On("GetIndexBuildProgress", _a0, _a1)}
|
||||
}
|
||||
|
@ -1168,8 +1168,8 @@ type MockDataCoord_GetIndexInfos_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexInfos is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexInfoRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexInfoRequest
|
||||
func (_e *MockDataCoord_Expecter) GetIndexInfos(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexInfos_Call {
|
||||
return &MockDataCoord_GetIndexInfos_Call{Call: _e.mock.On("GetIndexInfos", _a0, _a1)}
|
||||
}
|
||||
|
@ -1223,8 +1223,8 @@ type MockDataCoord_GetIndexState_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexStateRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexStateRequest
|
||||
func (_e *MockDataCoord_Expecter) GetIndexState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexState_Call {
|
||||
return &MockDataCoord_GetIndexState_Call{Call: _e.mock.On("GetIndexState", _a0, _a1)}
|
||||
}
|
||||
|
@ -1278,8 +1278,8 @@ type MockDataCoord_GetIndexStatistics_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexStatistics is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexStatisticsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetIndexStatisticsRequest
|
||||
func (_e *MockDataCoord_Expecter) GetIndexStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexStatistics_Call {
|
||||
return &MockDataCoord_GetIndexStatistics_Call{Call: _e.mock.On("GetIndexStatistics", _a0, _a1)}
|
||||
}
|
||||
|
@ -1333,8 +1333,8 @@ type MockDataCoord_GetInsertBinlogPaths_Call struct {
|
|||
}
|
||||
|
||||
// GetInsertBinlogPaths is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetInsertBinlogPathsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetInsertBinlogPathsRequest
|
||||
func (_e *MockDataCoord_Expecter) GetInsertBinlogPaths(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetInsertBinlogPaths_Call {
|
||||
return &MockDataCoord_GetInsertBinlogPaths_Call{Call: _e.mock.On("GetInsertBinlogPaths", _a0, _a1)}
|
||||
}
|
||||
|
@ -1388,8 +1388,8 @@ type MockDataCoord_GetMetrics_Call struct {
|
|||
}
|
||||
|
||||
// GetMetrics is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetMetricsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetMetricsRequest
|
||||
func (_e *MockDataCoord_Expecter) GetMetrics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetMetrics_Call {
|
||||
return &MockDataCoord_GetMetrics_Call{Call: _e.mock.On("GetMetrics", _a0, _a1)}
|
||||
}
|
||||
|
@ -1443,8 +1443,8 @@ type MockDataCoord_GetPartitionStatistics_Call struct {
|
|||
}
|
||||
|
||||
// GetPartitionStatistics is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetPartitionStatisticsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetPartitionStatisticsRequest
|
||||
func (_e *MockDataCoord_Expecter) GetPartitionStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetPartitionStatistics_Call {
|
||||
return &MockDataCoord_GetPartitionStatistics_Call{Call: _e.mock.On("GetPartitionStatistics", _a0, _a1)}
|
||||
}
|
||||
|
@ -1498,8 +1498,8 @@ type MockDataCoord_GetRecoveryInfo_Call struct {
|
|||
}
|
||||
|
||||
// GetRecoveryInfo is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetRecoveryInfoRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetRecoveryInfoRequest
|
||||
func (_e *MockDataCoord_Expecter) GetRecoveryInfo(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetRecoveryInfo_Call {
|
||||
return &MockDataCoord_GetRecoveryInfo_Call{Call: _e.mock.On("GetRecoveryInfo", _a0, _a1)}
|
||||
}
|
||||
|
@ -1553,8 +1553,8 @@ type MockDataCoord_GetRecoveryInfoV2_Call struct {
|
|||
}
|
||||
|
||||
// GetRecoveryInfoV2 is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetRecoveryInfoRequestV2
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetRecoveryInfoRequestV2
|
||||
func (_e *MockDataCoord_Expecter) GetRecoveryInfoV2(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetRecoveryInfoV2_Call {
|
||||
return &MockDataCoord_GetRecoveryInfoV2_Call{Call: _e.mock.On("GetRecoveryInfoV2", _a0, _a1)}
|
||||
}
|
||||
|
@ -1608,8 +1608,8 @@ type MockDataCoord_GetSegmentIndexState_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentIndexState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetSegmentIndexStateRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.GetSegmentIndexStateRequest
|
||||
func (_e *MockDataCoord_Expecter) GetSegmentIndexState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentIndexState_Call {
|
||||
return &MockDataCoord_GetSegmentIndexState_Call{Call: _e.mock.On("GetSegmentIndexState", _a0, _a1)}
|
||||
}
|
||||
|
@ -1663,8 +1663,8 @@ type MockDataCoord_GetSegmentInfo_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentInfo is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentInfoRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentInfoRequest
|
||||
func (_e *MockDataCoord_Expecter) GetSegmentInfo(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentInfo_Call {
|
||||
return &MockDataCoord_GetSegmentInfo_Call{Call: _e.mock.On("GetSegmentInfo", _a0, _a1)}
|
||||
}
|
||||
|
@ -1718,8 +1718,8 @@ type MockDataCoord_GetSegmentInfoChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentInfoChannel is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentInfoChannelRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentInfoChannelRequest
|
||||
func (_e *MockDataCoord_Expecter) GetSegmentInfoChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentInfoChannel_Call {
|
||||
return &MockDataCoord_GetSegmentInfoChannel_Call{Call: _e.mock.On("GetSegmentInfoChannel", _a0, _a1)}
|
||||
}
|
||||
|
@ -1773,8 +1773,8 @@ type MockDataCoord_GetSegmentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentStates is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentStatesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentStatesRequest
|
||||
func (_e *MockDataCoord_Expecter) GetSegmentStates(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentStates_Call {
|
||||
return &MockDataCoord_GetSegmentStates_Call{Call: _e.mock.On("GetSegmentStates", _a0, _a1)}
|
||||
}
|
||||
|
@ -1828,8 +1828,8 @@ type MockDataCoord_GetSegmentsByStates_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentsByStates is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentsByStatesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.GetSegmentsByStatesRequest
|
||||
func (_e *MockDataCoord_Expecter) GetSegmentsByStates(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentsByStates_Call {
|
||||
return &MockDataCoord_GetSegmentsByStates_Call{Call: _e.mock.On("GetSegmentsByStates", _a0, _a1)}
|
||||
}
|
||||
|
@ -1883,8 +1883,8 @@ type MockDataCoord_GetStatisticsChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetStatisticsChannel is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetStatisticsChannelRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetStatisticsChannelRequest
|
||||
func (_e *MockDataCoord_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetStatisticsChannel_Call {
|
||||
return &MockDataCoord_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)}
|
||||
}
|
||||
|
@ -1938,8 +1938,8 @@ type MockDataCoord_GetTimeTickChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetTimeTickChannel is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetTimeTickChannelRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetTimeTickChannelRequest
|
||||
func (_e *MockDataCoord_Expecter) GetTimeTickChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetTimeTickChannel_Call {
|
||||
return &MockDataCoord_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel", _a0, _a1)}
|
||||
}
|
||||
|
@ -1961,61 +1961,6 @@ func (_c *MockDataCoord_GetTimeTickChannel_Call) RunAndReturn(run func(context.C
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataCoord) Import(_a0 context.Context, _a1 *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *datapb.ImportTaskResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest) *datapb.ImportTaskResponse); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*datapb.ImportTaskResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ImportTaskRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataCoord_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type MockDataCoord_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ImportTaskRequest
|
||||
func (_e *MockDataCoord_Expecter) Import(_a0 interface{}, _a1 interface{}) *MockDataCoord_Import_Call {
|
||||
return &MockDataCoord_Import_Call{Call: _e.mock.On("Import", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_Import_Call) Run(run func(_a0 context.Context, _a1 *datapb.ImportTaskRequest)) *MockDataCoord_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*datapb.ImportTaskRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_Import_Call) Return(_a0 *datapb.ImportTaskResponse, _a1 error) *MockDataCoord_Import_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_Import_Call) RunAndReturn(run func(context.Context, *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)) *MockDataCoord_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ImportV2 provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataCoord) ImportV2(_a0 context.Context, _a1 *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
@ -2048,8 +1993,8 @@ type MockDataCoord_ImportV2_Call struct {
|
|||
}
|
||||
|
||||
// ImportV2 is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ImportRequestInternal
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ImportRequestInternal
|
||||
func (_e *MockDataCoord_Expecter) ImportV2(_a0 interface{}, _a1 interface{}) *MockDataCoord_ImportV2_Call {
|
||||
return &MockDataCoord_ImportV2_Call{Call: _e.mock.On("ImportV2", _a0, _a1)}
|
||||
}
|
||||
|
@ -2144,8 +2089,8 @@ type MockDataCoord_ListImports_Call struct {
|
|||
}
|
||||
|
||||
// ListImports is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ListImportsRequestInternal
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ListImportsRequestInternal
|
||||
func (_e *MockDataCoord_Expecter) ListImports(_a0 interface{}, _a1 interface{}) *MockDataCoord_ListImports_Call {
|
||||
return &MockDataCoord_ListImports_Call{Call: _e.mock.On("ListImports", _a0, _a1)}
|
||||
}
|
||||
|
@ -2199,8 +2144,8 @@ type MockDataCoord_ListIndexes_Call struct {
|
|||
}
|
||||
|
||||
// ListIndexes is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.ListIndexesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *indexpb.ListIndexesRequest
|
||||
func (_e *MockDataCoord_Expecter) ListIndexes(_a0 interface{}, _a1 interface{}) *MockDataCoord_ListIndexes_Call {
|
||||
return &MockDataCoord_ListIndexes_Call{Call: _e.mock.On("ListIndexes", _a0, _a1)}
|
||||
}
|
||||
|
@ -2254,8 +2199,8 @@ type MockDataCoord_ManualCompaction_Call struct {
|
|||
}
|
||||
|
||||
// ManualCompaction is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ManualCompactionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ManualCompactionRequest
|
||||
func (_e *MockDataCoord_Expecter) ManualCompaction(_a0 interface{}, _a1 interface{}) *MockDataCoord_ManualCompaction_Call {
|
||||
return &MockDataCoord_ManualCompaction_Call{Call: _e.mock.On("ManualCompaction", _a0, _a1)}
|
||||
}
|
||||
|
@ -2309,8 +2254,8 @@ type MockDataCoord_MarkSegmentsDropped_Call struct {
|
|||
}
|
||||
|
||||
// MarkSegmentsDropped is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.MarkSegmentsDroppedRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.MarkSegmentsDroppedRequest
|
||||
func (_e *MockDataCoord_Expecter) MarkSegmentsDropped(_a0 interface{}, _a1 interface{}) *MockDataCoord_MarkSegmentsDropped_Call {
|
||||
return &MockDataCoord_MarkSegmentsDropped_Call{Call: _e.mock.On("MarkSegmentsDropped", _a0, _a1)}
|
||||
}
|
||||
|
@ -2405,8 +2350,8 @@ type MockDataCoord_ReportDataNodeTtMsgs_Call struct {
|
|||
}
|
||||
|
||||
// ReportDataNodeTtMsgs is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ReportDataNodeTtMsgsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ReportDataNodeTtMsgsRequest
|
||||
func (_e *MockDataCoord_Expecter) ReportDataNodeTtMsgs(_a0 interface{}, _a1 interface{}) *MockDataCoord_ReportDataNodeTtMsgs_Call {
|
||||
return &MockDataCoord_ReportDataNodeTtMsgs_Call{Call: _e.mock.On("ReportDataNodeTtMsgs", _a0, _a1)}
|
||||
}
|
||||
|
@ -2460,8 +2405,8 @@ type MockDataCoord_SaveBinlogPaths_Call struct {
|
|||
}
|
||||
|
||||
// SaveBinlogPaths is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.SaveBinlogPathsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.SaveBinlogPathsRequest
|
||||
func (_e *MockDataCoord_Expecter) SaveBinlogPaths(_a0 interface{}, _a1 interface{}) *MockDataCoord_SaveBinlogPaths_Call {
|
||||
return &MockDataCoord_SaveBinlogPaths_Call{Call: _e.mock.On("SaveBinlogPaths", _a0, _a1)}
|
||||
}
|
||||
|
@ -2483,61 +2428,6 @@ func (_c *MockDataCoord_SaveBinlogPaths_Call) RunAndReturn(run func(context.Cont
|
|||
return _c
|
||||
}
|
||||
|
||||
// SaveImportSegment provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataCoord) SaveImportSegment(_a0 context.Context, _a1 *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SaveImportSegmentRequest) (*commonpb.Status, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SaveImportSegmentRequest) *commonpb.Status); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.SaveImportSegmentRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataCoord_SaveImportSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveImportSegment'
|
||||
type MockDataCoord_SaveImportSegment_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SaveImportSegment is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.SaveImportSegmentRequest
|
||||
func (_e *MockDataCoord_Expecter) SaveImportSegment(_a0 interface{}, _a1 interface{}) *MockDataCoord_SaveImportSegment_Call {
|
||||
return &MockDataCoord_SaveImportSegment_Call{Call: _e.mock.On("SaveImportSegment", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_SaveImportSegment_Call) Run(run func(_a0 context.Context, _a1 *datapb.SaveImportSegmentRequest)) *MockDataCoord_SaveImportSegment_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*datapb.SaveImportSegmentRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_SaveImportSegment_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataCoord_SaveImportSegment_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_SaveImportSegment_Call) RunAndReturn(run func(context.Context, *datapb.SaveImportSegmentRequest) (*commonpb.Status, error)) *MockDataCoord_SaveImportSegment_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetAddress provides a mock function with given fields: address
|
||||
func (_m *MockDataCoord) SetAddress(address string) {
|
||||
_m.Called(address)
|
||||
|
@ -2549,7 +2439,7 @@ type MockDataCoord_SetAddress_Call struct {
|
|||
}
|
||||
|
||||
// SetAddress is a helper method to define mock.On call
|
||||
// - address string
|
||||
// - address string
|
||||
func (_e *MockDataCoord_Expecter) SetAddress(address interface{}) *MockDataCoord_SetAddress_Call {
|
||||
return &MockDataCoord_SetAddress_Call{Call: _e.mock.On("SetAddress", address)}
|
||||
}
|
||||
|
@ -2582,7 +2472,7 @@ type MockDataCoord_SetDataNodeCreator_Call struct {
|
|||
}
|
||||
|
||||
// SetDataNodeCreator is a helper method to define mock.On call
|
||||
// - _a0 func(context.Context , string , int64)(types.DataNodeClient , error)
|
||||
// - _a0 func(context.Context , string , int64)(types.DataNodeClient , error)
|
||||
func (_e *MockDataCoord_Expecter) SetDataNodeCreator(_a0 interface{}) *MockDataCoord_SetDataNodeCreator_Call {
|
||||
return &MockDataCoord_SetDataNodeCreator_Call{Call: _e.mock.On("SetDataNodeCreator", _a0)}
|
||||
}
|
||||
|
@ -2615,7 +2505,7 @@ type MockDataCoord_SetEtcdClient_Call struct {
|
|||
}
|
||||
|
||||
// SetEtcdClient is a helper method to define mock.On call
|
||||
// - etcdClient *clientv3.Client
|
||||
// - etcdClient *clientv3.Client
|
||||
func (_e *MockDataCoord_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataCoord_SetEtcdClient_Call {
|
||||
return &MockDataCoord_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)}
|
||||
}
|
||||
|
@ -2648,7 +2538,7 @@ type MockDataCoord_SetIndexNodeCreator_Call struct {
|
|||
}
|
||||
|
||||
// SetIndexNodeCreator is a helper method to define mock.On call
|
||||
// - _a0 func(context.Context , string , int64)(types.IndexNodeClient , error)
|
||||
// - _a0 func(context.Context , string , int64)(types.IndexNodeClient , error)
|
||||
func (_e *MockDataCoord_Expecter) SetIndexNodeCreator(_a0 interface{}) *MockDataCoord_SetIndexNodeCreator_Call {
|
||||
return &MockDataCoord_SetIndexNodeCreator_Call{Call: _e.mock.On("SetIndexNodeCreator", _a0)}
|
||||
}
|
||||
|
@ -2681,7 +2571,7 @@ type MockDataCoord_SetRootCoordClient_Call struct {
|
|||
}
|
||||
|
||||
// SetRootCoordClient is a helper method to define mock.On call
|
||||
// - rootCoord types.RootCoordClient
|
||||
// - rootCoord types.RootCoordClient
|
||||
func (_e *MockDataCoord_Expecter) SetRootCoordClient(rootCoord interface{}) *MockDataCoord_SetRootCoordClient_Call {
|
||||
return &MockDataCoord_SetRootCoordClient_Call{Call: _e.mock.On("SetRootCoordClient", rootCoord)}
|
||||
}
|
||||
|
@ -2735,8 +2625,8 @@ type MockDataCoord_SetSegmentState_Call struct {
|
|||
}
|
||||
|
||||
// SetSegmentState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.SetSegmentStateRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.SetSegmentStateRequest
|
||||
func (_e *MockDataCoord_Expecter) SetSegmentState(_a0 interface{}, _a1 interface{}) *MockDataCoord_SetSegmentState_Call {
|
||||
return &MockDataCoord_SetSegmentState_Call{Call: _e.mock.On("SetSegmentState", _a0, _a1)}
|
||||
}
|
||||
|
@ -2769,7 +2659,7 @@ type MockDataCoord_SetTiKVClient_Call struct {
|
|||
}
|
||||
|
||||
// SetTiKVClient is a helper method to define mock.On call
|
||||
// - client *txnkv.Client
|
||||
// - client *txnkv.Client
|
||||
func (_e *MockDataCoord_Expecter) SetTiKVClient(client interface{}) *MockDataCoord_SetTiKVClient_Call {
|
||||
return &MockDataCoord_SetTiKVClient_Call{Call: _e.mock.On("SetTiKVClient", client)}
|
||||
}
|
||||
|
@ -2823,8 +2713,8 @@ type MockDataCoord_ShowConfigurations_Call struct {
|
|||
}
|
||||
|
||||
// ShowConfigurations is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ShowConfigurationsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ShowConfigurationsRequest
|
||||
func (_e *MockDataCoord_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *MockDataCoord_ShowConfigurations_Call {
|
||||
return &MockDataCoord_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)}
|
||||
}
|
||||
|
@ -2928,61 +2818,6 @@ func (_c *MockDataCoord_Stop_Call) RunAndReturn(run func() error) *MockDataCoord
|
|||
return _c
|
||||
}
|
||||
|
||||
// UnsetIsImportingState provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataCoord) UnsetIsImportingState(_a0 context.Context, _a1 *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.UnsetIsImportingStateRequest) *commonpb.Status); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.UnsetIsImportingStateRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataCoord_UnsetIsImportingState_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UnsetIsImportingState'
|
||||
type MockDataCoord_UnsetIsImportingState_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// UnsetIsImportingState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.UnsetIsImportingStateRequest
|
||||
func (_e *MockDataCoord_Expecter) UnsetIsImportingState(_a0 interface{}, _a1 interface{}) *MockDataCoord_UnsetIsImportingState_Call {
|
||||
return &MockDataCoord_UnsetIsImportingState_Call{Call: _e.mock.On("UnsetIsImportingState", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_UnsetIsImportingState_Call) Run(run func(_a0 context.Context, _a1 *datapb.UnsetIsImportingStateRequest)) *MockDataCoord_UnsetIsImportingState_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*datapb.UnsetIsImportingStateRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_UnsetIsImportingState_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataCoord_UnsetIsImportingState_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoord_UnsetIsImportingState_Call) RunAndReturn(run func(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)) *MockDataCoord_UnsetIsImportingState_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// UpdateChannelCheckpoint provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataCoord) UpdateChannelCheckpoint(_a0 context.Context, _a1 *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
@ -3015,8 +2850,8 @@ type MockDataCoord_UpdateChannelCheckpoint_Call struct {
|
|||
}
|
||||
|
||||
// UpdateChannelCheckpoint is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.UpdateChannelCheckpointRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.UpdateChannelCheckpointRequest
|
||||
func (_e *MockDataCoord_Expecter) UpdateChannelCheckpoint(_a0 interface{}, _a1 interface{}) *MockDataCoord_UpdateChannelCheckpoint_Call {
|
||||
return &MockDataCoord_UpdateChannelCheckpoint_Call{Call: _e.mock.On("UpdateChannelCheckpoint", _a0, _a1)}
|
||||
}
|
||||
|
@ -3070,8 +2905,8 @@ type MockDataCoord_UpdateSegmentStatistics_Call struct {
|
|||
}
|
||||
|
||||
// UpdateSegmentStatistics is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.UpdateSegmentStatisticsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.UpdateSegmentStatisticsRequest
|
||||
func (_e *MockDataCoord_Expecter) UpdateSegmentStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_UpdateSegmentStatistics_Call {
|
||||
return &MockDataCoord_UpdateSegmentStatistics_Call{Call: _e.mock.On("UpdateSegmentStatistics", _a0, _a1)}
|
||||
}
|
||||
|
@ -3125,8 +2960,8 @@ type MockDataCoord_WatchChannels_Call struct {
|
|||
}
|
||||
|
||||
// WatchChannels is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.WatchChannelsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.WatchChannelsRequest
|
||||
func (_e *MockDataCoord_Expecter) WatchChannels(_a0 interface{}, _a1 interface{}) *MockDataCoord_WatchChannels_Call {
|
||||
return &MockDataCoord_WatchChannels_Call{Call: _e.mock.On("WatchChannels", _a0, _a1)}
|
||||
}
|
||||
|
|
|
@ -72,9 +72,9 @@ type MockDataCoordClient_AlterIndex_Call struct {
|
|||
}
|
||||
|
||||
// AlterIndex is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.AlterIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.AlterIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) AlterIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_AlterIndex_Call {
|
||||
return &MockDataCoordClient_AlterIndex_Call{Call: _e.mock.On("AlterIndex",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -142,9 +142,9 @@ type MockDataCoordClient_AssignSegmentID_Call struct {
|
|||
}
|
||||
|
||||
// AssignSegmentID is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.AssignSegmentIDRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.AssignSegmentIDRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) AssignSegmentID(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_AssignSegmentID_Call {
|
||||
return &MockDataCoordClient_AssignSegmentID_Call{Call: _e.mock.On("AssignSegmentID",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -212,9 +212,9 @@ type MockDataCoordClient_BroadcastAlteredCollection_Call struct {
|
|||
}
|
||||
|
||||
// BroadcastAlteredCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.AlterCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.AlterCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) BroadcastAlteredCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_BroadcastAlteredCollection_Call {
|
||||
return &MockDataCoordClient_BroadcastAlteredCollection_Call{Call: _e.mock.On("BroadcastAlteredCollection",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -282,9 +282,9 @@ type MockDataCoordClient_CheckHealth_Call struct {
|
|||
}
|
||||
|
||||
// CheckHealth is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CheckHealthRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CheckHealthRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) CheckHealth(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_CheckHealth_Call {
|
||||
return &MockDataCoordClient_CheckHealth_Call{Call: _e.mock.On("CheckHealth",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -393,9 +393,9 @@ type MockDataCoordClient_CreateIndex_Call struct {
|
|||
}
|
||||
|
||||
// CreateIndex is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.CreateIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.CreateIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) CreateIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_CreateIndex_Call {
|
||||
return &MockDataCoordClient_CreateIndex_Call{Call: _e.mock.On("CreateIndex",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -463,9 +463,9 @@ type MockDataCoordClient_DescribeIndex_Call struct {
|
|||
}
|
||||
|
||||
// DescribeIndex is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.DescribeIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.DescribeIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) DescribeIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_DescribeIndex_Call {
|
||||
return &MockDataCoordClient_DescribeIndex_Call{Call: _e.mock.On("DescribeIndex",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -533,9 +533,9 @@ type MockDataCoordClient_DropIndex_Call struct {
|
|||
}
|
||||
|
||||
// DropIndex is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.DropIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.DropIndexRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) DropIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_DropIndex_Call {
|
||||
return &MockDataCoordClient_DropIndex_Call{Call: _e.mock.On("DropIndex",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -603,9 +603,9 @@ type MockDataCoordClient_DropVirtualChannel_Call struct {
|
|||
}
|
||||
|
||||
// DropVirtualChannel is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.DropVirtualChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.DropVirtualChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) DropVirtualChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_DropVirtualChannel_Call {
|
||||
return &MockDataCoordClient_DropVirtualChannel_Call{Call: _e.mock.On("DropVirtualChannel",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -673,9 +673,9 @@ type MockDataCoordClient_Flush_Call struct {
|
|||
}
|
||||
|
||||
// Flush is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.FlushRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.FlushRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) Flush(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_Flush_Call {
|
||||
return &MockDataCoordClient_Flush_Call{Call: _e.mock.On("Flush",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -743,9 +743,9 @@ type MockDataCoordClient_GcConfirm_Call struct {
|
|||
}
|
||||
|
||||
// GcConfirm is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GcConfirmRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GcConfirmRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GcConfirm(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GcConfirm_Call {
|
||||
return &MockDataCoordClient_GcConfirm_Call{Call: _e.mock.On("GcConfirm",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -813,9 +813,9 @@ type MockDataCoordClient_GcControl_Call struct {
|
|||
}
|
||||
|
||||
// GcControl is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GcControlRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GcControlRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GcControl(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GcControl_Call {
|
||||
return &MockDataCoordClient_GcControl_Call{Call: _e.mock.On("GcControl",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -883,9 +883,9 @@ type MockDataCoordClient_GetCollectionStatistics_Call struct {
|
|||
}
|
||||
|
||||
// GetCollectionStatistics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetCollectionStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetCollectionStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetCollectionStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetCollectionStatistics_Call {
|
||||
return &MockDataCoordClient_GetCollectionStatistics_Call{Call: _e.mock.On("GetCollectionStatistics",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -953,9 +953,9 @@ type MockDataCoordClient_GetCompactionState_Call struct {
|
|||
}
|
||||
|
||||
// GetCompactionState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetCompactionStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetCompactionStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetCompactionState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetCompactionState_Call {
|
||||
return &MockDataCoordClient_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1023,9 +1023,9 @@ type MockDataCoordClient_GetCompactionStateWithPlans_Call struct {
|
|||
}
|
||||
|
||||
// GetCompactionStateWithPlans is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetCompactionPlansRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetCompactionPlansRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetCompactionStateWithPlans(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetCompactionStateWithPlans_Call {
|
||||
return &MockDataCoordClient_GetCompactionStateWithPlans_Call{Call: _e.mock.On("GetCompactionStateWithPlans",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1093,9 +1093,9 @@ type MockDataCoordClient_GetComponentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetComponentStates is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetComponentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetComponentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetComponentStates_Call {
|
||||
return &MockDataCoordClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1163,9 +1163,9 @@ type MockDataCoordClient_GetFlushAllState_Call struct {
|
|||
}
|
||||
|
||||
// GetFlushAllState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetFlushAllStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetFlushAllStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetFlushAllState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetFlushAllState_Call {
|
||||
return &MockDataCoordClient_GetFlushAllState_Call{Call: _e.mock.On("GetFlushAllState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1233,9 +1233,9 @@ type MockDataCoordClient_GetFlushState_Call struct {
|
|||
}
|
||||
|
||||
// GetFlushState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetFlushStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetFlushStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetFlushState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetFlushState_Call {
|
||||
return &MockDataCoordClient_GetFlushState_Call{Call: _e.mock.On("GetFlushState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1303,9 +1303,9 @@ type MockDataCoordClient_GetFlushedSegments_Call struct {
|
|||
}
|
||||
|
||||
// GetFlushedSegments is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetFlushedSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetFlushedSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetFlushedSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetFlushedSegments_Call {
|
||||
return &MockDataCoordClient_GetFlushedSegments_Call{Call: _e.mock.On("GetFlushedSegments",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1373,9 +1373,9 @@ type MockDataCoordClient_GetImportProgress_Call struct {
|
|||
}
|
||||
|
||||
// GetImportProgress is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetImportProgressRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetImportProgressRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetImportProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetImportProgress_Call {
|
||||
return &MockDataCoordClient_GetImportProgress_Call{Call: _e.mock.On("GetImportProgress",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1443,9 +1443,9 @@ type MockDataCoordClient_GetIndexBuildProgress_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexBuildProgress is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexBuildProgressRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexBuildProgressRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetIndexBuildProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexBuildProgress_Call {
|
||||
return &MockDataCoordClient_GetIndexBuildProgress_Call{Call: _e.mock.On("GetIndexBuildProgress",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1513,9 +1513,9 @@ type MockDataCoordClient_GetIndexInfos_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexInfos is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexInfoRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexInfoRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetIndexInfos(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexInfos_Call {
|
||||
return &MockDataCoordClient_GetIndexInfos_Call{Call: _e.mock.On("GetIndexInfos",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1583,9 +1583,9 @@ type MockDataCoordClient_GetIndexState_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetIndexState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexState_Call {
|
||||
return &MockDataCoordClient_GetIndexState_Call{Call: _e.mock.On("GetIndexState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1653,9 +1653,9 @@ type MockDataCoordClient_GetIndexStatistics_Call struct {
|
|||
}
|
||||
|
||||
// GetIndexStatistics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetIndexStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetIndexStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexStatistics_Call {
|
||||
return &MockDataCoordClient_GetIndexStatistics_Call{Call: _e.mock.On("GetIndexStatistics",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1723,9 +1723,9 @@ type MockDataCoordClient_GetInsertBinlogPaths_Call struct {
|
|||
}
|
||||
|
||||
// GetInsertBinlogPaths is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetInsertBinlogPathsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetInsertBinlogPathsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetInsertBinlogPaths(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetInsertBinlogPaths_Call {
|
||||
return &MockDataCoordClient_GetInsertBinlogPaths_Call{Call: _e.mock.On("GetInsertBinlogPaths",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1793,9 +1793,9 @@ type MockDataCoordClient_GetMetrics_Call struct {
|
|||
}
|
||||
|
||||
// GetMetrics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetMetricsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetMetricsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetMetrics_Call {
|
||||
return &MockDataCoordClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1863,9 +1863,9 @@ type MockDataCoordClient_GetPartitionStatistics_Call struct {
|
|||
}
|
||||
|
||||
// GetPartitionStatistics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetPartitionStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetPartitionStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetPartitionStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetPartitionStatistics_Call {
|
||||
return &MockDataCoordClient_GetPartitionStatistics_Call{Call: _e.mock.On("GetPartitionStatistics",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1933,9 +1933,9 @@ type MockDataCoordClient_GetRecoveryInfo_Call struct {
|
|||
}
|
||||
|
||||
// GetRecoveryInfo is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetRecoveryInfoRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetRecoveryInfoRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetRecoveryInfo(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetRecoveryInfo_Call {
|
||||
return &MockDataCoordClient_GetRecoveryInfo_Call{Call: _e.mock.On("GetRecoveryInfo",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2003,9 +2003,9 @@ type MockDataCoordClient_GetRecoveryInfoV2_Call struct {
|
|||
}
|
||||
|
||||
// GetRecoveryInfoV2 is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetRecoveryInfoRequestV2
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetRecoveryInfoRequestV2
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetRecoveryInfoV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetRecoveryInfoV2_Call {
|
||||
return &MockDataCoordClient_GetRecoveryInfoV2_Call{Call: _e.mock.On("GetRecoveryInfoV2",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2073,9 +2073,9 @@ type MockDataCoordClient_GetSegmentIndexState_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentIndexState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetSegmentIndexStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.GetSegmentIndexStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetSegmentIndexState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentIndexState_Call {
|
||||
return &MockDataCoordClient_GetSegmentIndexState_Call{Call: _e.mock.On("GetSegmentIndexState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2143,9 +2143,9 @@ type MockDataCoordClient_GetSegmentInfo_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentInfo is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentInfoRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentInfoRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetSegmentInfo(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentInfo_Call {
|
||||
return &MockDataCoordClient_GetSegmentInfo_Call{Call: _e.mock.On("GetSegmentInfo",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2213,9 +2213,9 @@ type MockDataCoordClient_GetSegmentInfoChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentInfoChannel is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentInfoChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentInfoChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetSegmentInfoChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentInfoChannel_Call {
|
||||
return &MockDataCoordClient_GetSegmentInfoChannel_Call{Call: _e.mock.On("GetSegmentInfoChannel",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2283,9 +2283,9 @@ type MockDataCoordClient_GetSegmentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentStates is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetSegmentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentStates_Call {
|
||||
return &MockDataCoordClient_GetSegmentStates_Call{Call: _e.mock.On("GetSegmentStates",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2353,9 +2353,9 @@ type MockDataCoordClient_GetSegmentsByStates_Call struct {
|
|||
}
|
||||
|
||||
// GetSegmentsByStates is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentsByStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.GetSegmentsByStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetSegmentsByStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentsByStates_Call {
|
||||
return &MockDataCoordClient_GetSegmentsByStates_Call{Call: _e.mock.On("GetSegmentsByStates",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2423,9 +2423,9 @@ type MockDataCoordClient_GetStatisticsChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetStatisticsChannel is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetStatisticsChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetStatisticsChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetStatisticsChannel_Call {
|
||||
return &MockDataCoordClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2493,9 +2493,9 @@ type MockDataCoordClient_GetTimeTickChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetTimeTickChannel is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetTimeTickChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetTimeTickChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) GetTimeTickChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetTimeTickChannel_Call {
|
||||
return &MockDataCoordClient_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2524,76 +2524,6 @@ func (_c *MockDataCoordClient_GetTimeTickChannel_Call) RunAndReturn(run func(con
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataCoordClient) Import(ctx context.Context, in *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *datapb.ImportTaskResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) (*datapb.ImportTaskResponse, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) *datapb.ImportTaskResponse); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*datapb.ImportTaskResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataCoordClient_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type MockDataCoordClient_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ImportTaskRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) Import(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_Import_Call {
|
||||
return &MockDataCoordClient_Import_Call{Call: _e.mock.On("Import",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_Import_Call) Run(run func(ctx context.Context, in *datapb.ImportTaskRequest, opts ...grpc.CallOption)) *MockDataCoordClient_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*datapb.ImportTaskRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_Import_Call) Return(_a0 *datapb.ImportTaskResponse, _a1 error) *MockDataCoordClient_Import_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_Import_Call) RunAndReturn(run func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) (*datapb.ImportTaskResponse, error)) *MockDataCoordClient_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ImportV2 provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataCoordClient) ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal, opts ...grpc.CallOption) (*internalpb.ImportResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -2633,9 +2563,9 @@ type MockDataCoordClient_ImportV2_Call struct {
|
|||
}
|
||||
|
||||
// ImportV2 is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ImportRequestInternal
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ImportRequestInternal
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) ImportV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ImportV2_Call {
|
||||
return &MockDataCoordClient_ImportV2_Call{Call: _e.mock.On("ImportV2",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2703,9 +2633,9 @@ type MockDataCoordClient_ListImports_Call struct {
|
|||
}
|
||||
|
||||
// ListImports is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ListImportsRequestInternal
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ListImportsRequestInternal
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) ListImports(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ListImports_Call {
|
||||
return &MockDataCoordClient_ListImports_Call{Call: _e.mock.On("ListImports",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2773,9 +2703,9 @@ type MockDataCoordClient_ListIndexes_Call struct {
|
|||
}
|
||||
|
||||
// ListIndexes is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.ListIndexesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *indexpb.ListIndexesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) ListIndexes(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ListIndexes_Call {
|
||||
return &MockDataCoordClient_ListIndexes_Call{Call: _e.mock.On("ListIndexes",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2843,9 +2773,9 @@ type MockDataCoordClient_ManualCompaction_Call struct {
|
|||
}
|
||||
|
||||
// ManualCompaction is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ManualCompactionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ManualCompactionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) ManualCompaction(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ManualCompaction_Call {
|
||||
return &MockDataCoordClient_ManualCompaction_Call{Call: _e.mock.On("ManualCompaction",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2913,9 +2843,9 @@ type MockDataCoordClient_MarkSegmentsDropped_Call struct {
|
|||
}
|
||||
|
||||
// MarkSegmentsDropped is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.MarkSegmentsDroppedRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.MarkSegmentsDroppedRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) MarkSegmentsDropped(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_MarkSegmentsDropped_Call {
|
||||
return &MockDataCoordClient_MarkSegmentsDropped_Call{Call: _e.mock.On("MarkSegmentsDropped",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2983,9 +2913,9 @@ type MockDataCoordClient_ReportDataNodeTtMsgs_Call struct {
|
|||
}
|
||||
|
||||
// ReportDataNodeTtMsgs is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ReportDataNodeTtMsgsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ReportDataNodeTtMsgsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) ReportDataNodeTtMsgs(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ReportDataNodeTtMsgs_Call {
|
||||
return &MockDataCoordClient_ReportDataNodeTtMsgs_Call{Call: _e.mock.On("ReportDataNodeTtMsgs",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3053,9 +2983,9 @@ type MockDataCoordClient_SaveBinlogPaths_Call struct {
|
|||
}
|
||||
|
||||
// SaveBinlogPaths is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.SaveBinlogPathsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.SaveBinlogPathsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) SaveBinlogPaths(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_SaveBinlogPaths_Call {
|
||||
return &MockDataCoordClient_SaveBinlogPaths_Call{Call: _e.mock.On("SaveBinlogPaths",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3084,76 +3014,6 @@ func (_c *MockDataCoordClient_SaveBinlogPaths_Call) RunAndReturn(run func(contex
|
|||
return _c
|
||||
}
|
||||
|
||||
// SaveImportSegment provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataCoordClient) SaveImportSegment(ctx context.Context, in *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SaveImportSegmentRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SaveImportSegmentRequest, ...grpc.CallOption) *commonpb.Status); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.SaveImportSegmentRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataCoordClient_SaveImportSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveImportSegment'
|
||||
type MockDataCoordClient_SaveImportSegment_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SaveImportSegment is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.SaveImportSegmentRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) SaveImportSegment(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_SaveImportSegment_Call {
|
||||
return &MockDataCoordClient_SaveImportSegment_Call{Call: _e.mock.On("SaveImportSegment",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_SaveImportSegment_Call) Run(run func(ctx context.Context, in *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption)) *MockDataCoordClient_SaveImportSegment_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*datapb.SaveImportSegmentRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_SaveImportSegment_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataCoordClient_SaveImportSegment_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_SaveImportSegment_Call) RunAndReturn(run func(context.Context, *datapb.SaveImportSegmentRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockDataCoordClient_SaveImportSegment_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetSegmentState provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataCoordClient) SetSegmentState(ctx context.Context, in *datapb.SetSegmentStateRequest, opts ...grpc.CallOption) (*datapb.SetSegmentStateResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -3193,9 +3053,9 @@ type MockDataCoordClient_SetSegmentState_Call struct {
|
|||
}
|
||||
|
||||
// SetSegmentState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.SetSegmentStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.SetSegmentStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) SetSegmentState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_SetSegmentState_Call {
|
||||
return &MockDataCoordClient_SetSegmentState_Call{Call: _e.mock.On("SetSegmentState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3263,9 +3123,9 @@ type MockDataCoordClient_ShowConfigurations_Call struct {
|
|||
}
|
||||
|
||||
// ShowConfigurations is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ShowConfigurationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ShowConfigurationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ShowConfigurations_Call {
|
||||
return &MockDataCoordClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3294,76 +3154,6 @@ func (_c *MockDataCoordClient_ShowConfigurations_Call) RunAndReturn(run func(con
|
|||
return _c
|
||||
}
|
||||
|
||||
// UnsetIsImportingState provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataCoordClient) UnsetIsImportingState(ctx context.Context, in *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.UnsetIsImportingStateRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.UnsetIsImportingStateRequest, ...grpc.CallOption) *commonpb.Status); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.UnsetIsImportingStateRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataCoordClient_UnsetIsImportingState_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UnsetIsImportingState'
|
||||
type MockDataCoordClient_UnsetIsImportingState_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// UnsetIsImportingState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.UnsetIsImportingStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) UnsetIsImportingState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_UnsetIsImportingState_Call {
|
||||
return &MockDataCoordClient_UnsetIsImportingState_Call{Call: _e.mock.On("UnsetIsImportingState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_UnsetIsImportingState_Call) Run(run func(ctx context.Context, in *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption)) *MockDataCoordClient_UnsetIsImportingState_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*datapb.UnsetIsImportingStateRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_UnsetIsImportingState_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataCoordClient_UnsetIsImportingState_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataCoordClient_UnsetIsImportingState_Call) RunAndReturn(run func(context.Context, *datapb.UnsetIsImportingStateRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockDataCoordClient_UnsetIsImportingState_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// UpdateChannelCheckpoint provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataCoordClient) UpdateChannelCheckpoint(ctx context.Context, in *datapb.UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -3403,9 +3193,9 @@ type MockDataCoordClient_UpdateChannelCheckpoint_Call struct {
|
|||
}
|
||||
|
||||
// UpdateChannelCheckpoint is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.UpdateChannelCheckpointRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.UpdateChannelCheckpointRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) UpdateChannelCheckpoint(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_UpdateChannelCheckpoint_Call {
|
||||
return &MockDataCoordClient_UpdateChannelCheckpoint_Call{Call: _e.mock.On("UpdateChannelCheckpoint",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3473,9 +3263,9 @@ type MockDataCoordClient_UpdateSegmentStatistics_Call struct {
|
|||
}
|
||||
|
||||
// UpdateSegmentStatistics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.UpdateSegmentStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.UpdateSegmentStatisticsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) UpdateSegmentStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_UpdateSegmentStatistics_Call {
|
||||
return &MockDataCoordClient_UpdateSegmentStatistics_Call{Call: _e.mock.On("UpdateSegmentStatistics",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3543,9 +3333,9 @@ type MockDataCoordClient_WatchChannels_Call struct {
|
|||
}
|
||||
|
||||
// WatchChannels is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.WatchChannelsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.WatchChannelsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataCoordClient_Expecter) WatchChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_WatchChannels_Call {
|
||||
return &MockDataCoordClient_WatchChannels_Call{Call: _e.mock.On("WatchChannels",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
|
|
@ -32,61 +32,6 @@ func (_m *MockDataNode) EXPECT() *MockDataNode_Expecter {
|
|||
return &MockDataNode_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// AddImportSegment provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataNode) AddImportSegment(_a0 context.Context, _a1 *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *datapb.AddImportSegmentResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.AddImportSegmentRequest) *datapb.AddImportSegmentResponse); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*datapb.AddImportSegmentResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.AddImportSegmentRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataNode_AddImportSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AddImportSegment'
|
||||
type MockDataNode_AddImportSegment_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// AddImportSegment is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.AddImportSegmentRequest
|
||||
func (_e *MockDataNode_Expecter) AddImportSegment(_a0 interface{}, _a1 interface{}) *MockDataNode_AddImportSegment_Call {
|
||||
return &MockDataNode_AddImportSegment_Call{Call: _e.mock.On("AddImportSegment", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *MockDataNode_AddImportSegment_Call) Run(run func(_a0 context.Context, _a1 *datapb.AddImportSegmentRequest)) *MockDataNode_AddImportSegment_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*datapb.AddImportSegmentRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNode_AddImportSegment_Call) Return(_a0 *datapb.AddImportSegmentResponse, _a1 error) *MockDataNode_AddImportSegment_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNode_AddImportSegment_Call) RunAndReturn(run func(context.Context, *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error)) *MockDataNode_AddImportSegment_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// CheckChannelOperationProgress provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataNode) CheckChannelOperationProgress(_a0 context.Context, _a1 *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
@ -119,8 +64,8 @@ type MockDataNode_CheckChannelOperationProgress_Call struct {
|
|||
}
|
||||
|
||||
// CheckChannelOperationProgress is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ChannelWatchInfo
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ChannelWatchInfo
|
||||
func (_e *MockDataNode_Expecter) CheckChannelOperationProgress(_a0 interface{}, _a1 interface{}) *MockDataNode_CheckChannelOperationProgress_Call {
|
||||
return &MockDataNode_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", _a0, _a1)}
|
||||
}
|
||||
|
@ -174,8 +119,8 @@ type MockDataNode_Compaction_Call struct {
|
|||
}
|
||||
|
||||
// Compaction is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.CompactionPlan
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.CompactionPlan
|
||||
func (_e *MockDataNode_Expecter) Compaction(_a0 interface{}, _a1 interface{}) *MockDataNode_Compaction_Call {
|
||||
return &MockDataNode_Compaction_Call{Call: _e.mock.On("Compaction", _a0, _a1)}
|
||||
}
|
||||
|
@ -229,8 +174,8 @@ type MockDataNode_DropImport_Call struct {
|
|||
}
|
||||
|
||||
// DropImport is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.DropImportRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.DropImportRequest
|
||||
func (_e *MockDataNode_Expecter) DropImport(_a0 interface{}, _a1 interface{}) *MockDataNode_DropImport_Call {
|
||||
return &MockDataNode_DropImport_Call{Call: _e.mock.On("DropImport", _a0, _a1)}
|
||||
}
|
||||
|
@ -284,8 +229,8 @@ type MockDataNode_FlushChannels_Call struct {
|
|||
}
|
||||
|
||||
// FlushChannels is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.FlushChannelsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.FlushChannelsRequest
|
||||
func (_e *MockDataNode_Expecter) FlushChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushChannels_Call {
|
||||
return &MockDataNode_FlushChannels_Call{Call: _e.mock.On("FlushChannels", _a0, _a1)}
|
||||
}
|
||||
|
@ -339,8 +284,8 @@ type MockDataNode_FlushSegments_Call struct {
|
|||
}
|
||||
|
||||
// FlushSegments is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.FlushSegmentsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.FlushSegmentsRequest
|
||||
func (_e *MockDataNode_Expecter) FlushSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushSegments_Call {
|
||||
return &MockDataNode_FlushSegments_Call{Call: _e.mock.On("FlushSegments", _a0, _a1)}
|
||||
}
|
||||
|
@ -435,8 +380,8 @@ type MockDataNode_GetCompactionState_Call struct {
|
|||
}
|
||||
|
||||
// GetCompactionState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.CompactionStateRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.CompactionStateRequest
|
||||
func (_e *MockDataNode_Expecter) GetCompactionState(_a0 interface{}, _a1 interface{}) *MockDataNode_GetCompactionState_Call {
|
||||
return &MockDataNode_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", _a0, _a1)}
|
||||
}
|
||||
|
@ -490,8 +435,8 @@ type MockDataNode_GetComponentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetComponentStates is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetComponentStatesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetComponentStatesRequest
|
||||
func (_e *MockDataNode_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *MockDataNode_GetComponentStates_Call {
|
||||
return &MockDataNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)}
|
||||
}
|
||||
|
@ -545,8 +490,8 @@ type MockDataNode_GetMetrics_Call struct {
|
|||
}
|
||||
|
||||
// GetMetrics is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetMetricsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetMetricsRequest
|
||||
func (_e *MockDataNode_Expecter) GetMetrics(_a0 interface{}, _a1 interface{}) *MockDataNode_GetMetrics_Call {
|
||||
return &MockDataNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", _a0, _a1)}
|
||||
}
|
||||
|
@ -682,8 +627,8 @@ type MockDataNode_GetStatisticsChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetStatisticsChannel is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetStatisticsChannelRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetStatisticsChannelRequest
|
||||
func (_e *MockDataNode_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *MockDataNode_GetStatisticsChannel_Call {
|
||||
return &MockDataNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)}
|
||||
}
|
||||
|
@ -705,61 +650,6 @@ func (_c *MockDataNode_GetStatisticsChannel_Call) RunAndReturn(run func(context.
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataNode) Import(_a0 context.Context, _a1 *datapb.ImportTaskRequest) (*commonpb.Status, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest) (*commonpb.Status, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest) *commonpb.Status); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ImportTaskRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataNode_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type MockDataNode_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ImportTaskRequest
|
||||
func (_e *MockDataNode_Expecter) Import(_a0 interface{}, _a1 interface{}) *MockDataNode_Import_Call {
|
||||
return &MockDataNode_Import_Call{Call: _e.mock.On("Import", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *MockDataNode_Import_Call) Run(run func(_a0 context.Context, _a1 *datapb.ImportTaskRequest)) *MockDataNode_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*datapb.ImportTaskRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNode_Import_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNode_Import_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNode_Import_Call) RunAndReturn(run func(context.Context, *datapb.ImportTaskRequest) (*commonpb.Status, error)) *MockDataNode_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ImportV2 provides a mock function with given fields: _a0, _a1
|
||||
func (_m *MockDataNode) ImportV2(_a0 context.Context, _a1 *datapb.ImportRequest) (*commonpb.Status, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
@ -792,8 +682,8 @@ type MockDataNode_ImportV2_Call struct {
|
|||
}
|
||||
|
||||
// ImportV2 is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ImportRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ImportRequest
|
||||
func (_e *MockDataNode_Expecter) ImportV2(_a0 interface{}, _a1 interface{}) *MockDataNode_ImportV2_Call {
|
||||
return &MockDataNode_ImportV2_Call{Call: _e.mock.On("ImportV2", _a0, _a1)}
|
||||
}
|
||||
|
@ -888,8 +778,8 @@ type MockDataNode_NotifyChannelOperation_Call struct {
|
|||
}
|
||||
|
||||
// NotifyChannelOperation is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ChannelOperationsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ChannelOperationsRequest
|
||||
func (_e *MockDataNode_Expecter) NotifyChannelOperation(_a0 interface{}, _a1 interface{}) *MockDataNode_NotifyChannelOperation_Call {
|
||||
return &MockDataNode_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", _a0, _a1)}
|
||||
}
|
||||
|
@ -943,8 +833,8 @@ type MockDataNode_PreImport_Call struct {
|
|||
}
|
||||
|
||||
// PreImport is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.PreImportRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.PreImportRequest
|
||||
func (_e *MockDataNode_Expecter) PreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_PreImport_Call {
|
||||
return &MockDataNode_PreImport_Call{Call: _e.mock.On("PreImport", _a0, _a1)}
|
||||
}
|
||||
|
@ -998,8 +888,8 @@ type MockDataNode_QueryImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryImport is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.QueryImportRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.QueryImportRequest
|
||||
func (_e *MockDataNode_Expecter) QueryImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryImport_Call {
|
||||
return &MockDataNode_QueryImport_Call{Call: _e.mock.On("QueryImport", _a0, _a1)}
|
||||
}
|
||||
|
@ -1053,8 +943,8 @@ type MockDataNode_QueryPreImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryPreImport is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.QueryPreImportRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.QueryPreImportRequest
|
||||
func (_e *MockDataNode_Expecter) QueryPreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryPreImport_Call {
|
||||
return &MockDataNode_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", _a0, _a1)}
|
||||
}
|
||||
|
@ -1149,8 +1039,8 @@ type MockDataNode_ResendSegmentStats_Call struct {
|
|||
}
|
||||
|
||||
// ResendSegmentStats is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ResendSegmentStatsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.ResendSegmentStatsRequest
|
||||
func (_e *MockDataNode_Expecter) ResendSegmentStats(_a0 interface{}, _a1 interface{}) *MockDataNode_ResendSegmentStats_Call {
|
||||
return &MockDataNode_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", _a0, _a1)}
|
||||
}
|
||||
|
@ -1183,7 +1073,7 @@ type MockDataNode_SetAddress_Call struct {
|
|||
}
|
||||
|
||||
// SetAddress is a helper method to define mock.On call
|
||||
// - address string
|
||||
// - address string
|
||||
func (_e *MockDataNode_Expecter) SetAddress(address interface{}) *MockDataNode_SetAddress_Call {
|
||||
return &MockDataNode_SetAddress_Call{Call: _e.mock.On("SetAddress", address)}
|
||||
}
|
||||
|
@ -1225,7 +1115,7 @@ type MockDataNode_SetDataCoordClient_Call struct {
|
|||
}
|
||||
|
||||
// SetDataCoordClient is a helper method to define mock.On call
|
||||
// - dataCoord types.DataCoordClient
|
||||
// - dataCoord types.DataCoordClient
|
||||
func (_e *MockDataNode_Expecter) SetDataCoordClient(dataCoord interface{}) *MockDataNode_SetDataCoordClient_Call {
|
||||
return &MockDataNode_SetDataCoordClient_Call{Call: _e.mock.On("SetDataCoordClient", dataCoord)}
|
||||
}
|
||||
|
@ -1258,7 +1148,7 @@ type MockDataNode_SetEtcdClient_Call struct {
|
|||
}
|
||||
|
||||
// SetEtcdClient is a helper method to define mock.On call
|
||||
// - etcdClient *clientv3.Client
|
||||
// - etcdClient *clientv3.Client
|
||||
func (_e *MockDataNode_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataNode_SetEtcdClient_Call {
|
||||
return &MockDataNode_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)}
|
||||
}
|
||||
|
@ -1300,7 +1190,7 @@ type MockDataNode_SetRootCoordClient_Call struct {
|
|||
}
|
||||
|
||||
// SetRootCoordClient is a helper method to define mock.On call
|
||||
// - rootCoord types.RootCoordClient
|
||||
// - rootCoord types.RootCoordClient
|
||||
func (_e *MockDataNode_Expecter) SetRootCoordClient(rootCoord interface{}) *MockDataNode_SetRootCoordClient_Call {
|
||||
return &MockDataNode_SetRootCoordClient_Call{Call: _e.mock.On("SetRootCoordClient", rootCoord)}
|
||||
}
|
||||
|
@ -1354,8 +1244,8 @@ type MockDataNode_ShowConfigurations_Call struct {
|
|||
}
|
||||
|
||||
// ShowConfigurations is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ShowConfigurationsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ShowConfigurationsRequest
|
||||
func (_e *MockDataNode_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *MockDataNode_ShowConfigurations_Call {
|
||||
return &MockDataNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)}
|
||||
}
|
||||
|
@ -1491,8 +1381,8 @@ type MockDataNode_SyncSegments_Call struct {
|
|||
}
|
||||
|
||||
// SyncSegments is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.SyncSegmentsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.SyncSegmentsRequest
|
||||
func (_e *MockDataNode_Expecter) SyncSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_SyncSegments_Call {
|
||||
return &MockDataNode_SyncSegments_Call{Call: _e.mock.On("SyncSegments", _a0, _a1)}
|
||||
}
|
||||
|
@ -1525,7 +1415,7 @@ type MockDataNode_UpdateStateCode_Call struct {
|
|||
}
|
||||
|
||||
// UpdateStateCode is a helper method to define mock.On call
|
||||
// - stateCode commonpb.StateCode
|
||||
// - stateCode commonpb.StateCode
|
||||
func (_e *MockDataNode_Expecter) UpdateStateCode(stateCode interface{}) *MockDataNode_UpdateStateCode_Call {
|
||||
return &MockDataNode_UpdateStateCode_Call{Call: _e.mock.On("UpdateStateCode", stateCode)}
|
||||
}
|
||||
|
@ -1579,8 +1469,8 @@ type MockDataNode_WatchDmChannels_Call struct {
|
|||
}
|
||||
|
||||
// WatchDmChannels is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.WatchDmChannelsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *datapb.WatchDmChannelsRequest
|
||||
func (_e *MockDataNode_Expecter) WatchDmChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_WatchDmChannels_Call {
|
||||
return &MockDataNode_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", _a0, _a1)}
|
||||
}
|
||||
|
|
|
@ -31,76 +31,6 @@ func (_m *MockDataNodeClient) EXPECT() *MockDataNodeClient_Expecter {
|
|||
return &MockDataNodeClient_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// AddImportSegment provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataNodeClient) AddImportSegment(ctx context.Context, in *datapb.AddImportSegmentRequest, opts ...grpc.CallOption) (*datapb.AddImportSegmentResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *datapb.AddImportSegmentResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.AddImportSegmentRequest, ...grpc.CallOption) (*datapb.AddImportSegmentResponse, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.AddImportSegmentRequest, ...grpc.CallOption) *datapb.AddImportSegmentResponse); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*datapb.AddImportSegmentResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.AddImportSegmentRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataNodeClient_AddImportSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AddImportSegment'
|
||||
type MockDataNodeClient_AddImportSegment_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// AddImportSegment is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.AddImportSegmentRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) AddImportSegment(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_AddImportSegment_Call {
|
||||
return &MockDataNodeClient_AddImportSegment_Call{Call: _e.mock.On("AddImportSegment",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockDataNodeClient_AddImportSegment_Call) Run(run func(ctx context.Context, in *datapb.AddImportSegmentRequest, opts ...grpc.CallOption)) *MockDataNodeClient_AddImportSegment_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*datapb.AddImportSegmentRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNodeClient_AddImportSegment_Call) Return(_a0 *datapb.AddImportSegmentResponse, _a1 error) *MockDataNodeClient_AddImportSegment_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNodeClient_AddImportSegment_Call) RunAndReturn(run func(context.Context, *datapb.AddImportSegmentRequest, ...grpc.CallOption) (*datapb.AddImportSegmentResponse, error)) *MockDataNodeClient_AddImportSegment_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// CheckChannelOperationProgress provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataNodeClient) CheckChannelOperationProgress(ctx context.Context, in *datapb.ChannelWatchInfo, opts ...grpc.CallOption) (*datapb.ChannelOperationProgressResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -140,9 +70,9 @@ type MockDataNodeClient_CheckChannelOperationProgress_Call struct {
|
|||
}
|
||||
|
||||
// CheckChannelOperationProgress is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ChannelWatchInfo
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ChannelWatchInfo
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) CheckChannelOperationProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_CheckChannelOperationProgress_Call {
|
||||
return &MockDataNodeClient_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -251,9 +181,9 @@ type MockDataNodeClient_Compaction_Call struct {
|
|||
}
|
||||
|
||||
// Compaction is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.CompactionPlan
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.CompactionPlan
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) Compaction(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_Compaction_Call {
|
||||
return &MockDataNodeClient_Compaction_Call{Call: _e.mock.On("Compaction",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -321,9 +251,9 @@ type MockDataNodeClient_DropImport_Call struct {
|
|||
}
|
||||
|
||||
// DropImport is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.DropImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.DropImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) DropImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_DropImport_Call {
|
||||
return &MockDataNodeClient_DropImport_Call{Call: _e.mock.On("DropImport",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -391,9 +321,9 @@ type MockDataNodeClient_FlushChannels_Call struct {
|
|||
}
|
||||
|
||||
// FlushChannels is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.FlushChannelsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.FlushChannelsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) FlushChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushChannels_Call {
|
||||
return &MockDataNodeClient_FlushChannels_Call{Call: _e.mock.On("FlushChannels",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -461,9 +391,9 @@ type MockDataNodeClient_FlushSegments_Call struct {
|
|||
}
|
||||
|
||||
// FlushSegments is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.FlushSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.FlushSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) FlushSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushSegments_Call {
|
||||
return &MockDataNodeClient_FlushSegments_Call{Call: _e.mock.On("FlushSegments",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -531,9 +461,9 @@ type MockDataNodeClient_GetCompactionState_Call struct {
|
|||
}
|
||||
|
||||
// GetCompactionState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.CompactionStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.CompactionStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) GetCompactionState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetCompactionState_Call {
|
||||
return &MockDataNodeClient_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -601,9 +531,9 @@ type MockDataNodeClient_GetComponentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetComponentStates is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetComponentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetComponentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetComponentStates_Call {
|
||||
return &MockDataNodeClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -671,9 +601,9 @@ type MockDataNodeClient_GetMetrics_Call struct {
|
|||
}
|
||||
|
||||
// GetMetrics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetMetricsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetMetricsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetMetrics_Call {
|
||||
return &MockDataNodeClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -741,9 +671,9 @@ type MockDataNodeClient_GetStatisticsChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetStatisticsChannel is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetStatisticsChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetStatisticsChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetStatisticsChannel_Call {
|
||||
return &MockDataNodeClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -772,76 +702,6 @@ func (_c *MockDataNodeClient_GetStatisticsChannel_Call) RunAndReturn(run func(co
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) *commonpb.Status); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockDataNodeClient_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type MockDataNodeClient_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ImportTaskRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) Import(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_Import_Call {
|
||||
return &MockDataNodeClient_Import_Call{Call: _e.mock.On("Import",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockDataNodeClient_Import_Call) Run(run func(ctx context.Context, in *datapb.ImportTaskRequest, opts ...grpc.CallOption)) *MockDataNodeClient_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*datapb.ImportTaskRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNodeClient_Import_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNodeClient_Import_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockDataNodeClient_Import_Call) RunAndReturn(run func(context.Context, *datapb.ImportTaskRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockDataNodeClient_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ImportV2 provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockDataNodeClient) ImportV2(ctx context.Context, in *datapb.ImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -881,9 +741,9 @@ type MockDataNodeClient_ImportV2_Call struct {
|
|||
}
|
||||
|
||||
// ImportV2 is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) ImportV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ImportV2_Call {
|
||||
return &MockDataNodeClient_ImportV2_Call{Call: _e.mock.On("ImportV2",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -951,9 +811,9 @@ type MockDataNodeClient_NotifyChannelOperation_Call struct {
|
|||
}
|
||||
|
||||
// NotifyChannelOperation is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ChannelOperationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ChannelOperationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) NotifyChannelOperation(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_NotifyChannelOperation_Call {
|
||||
return &MockDataNodeClient_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1021,9 +881,9 @@ type MockDataNodeClient_PreImport_Call struct {
|
|||
}
|
||||
|
||||
// PreImport is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.PreImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.PreImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) PreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_PreImport_Call {
|
||||
return &MockDataNodeClient_PreImport_Call{Call: _e.mock.On("PreImport",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1091,9 +951,9 @@ type MockDataNodeClient_QueryImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryImport is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.QueryImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.QueryImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) QueryImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryImport_Call {
|
||||
return &MockDataNodeClient_QueryImport_Call{Call: _e.mock.On("QueryImport",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1161,9 +1021,9 @@ type MockDataNodeClient_QueryPreImport_Call struct {
|
|||
}
|
||||
|
||||
// QueryPreImport is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.QueryPreImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.QueryPreImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) QueryPreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryPreImport_Call {
|
||||
return &MockDataNodeClient_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1231,9 +1091,9 @@ type MockDataNodeClient_ResendSegmentStats_Call struct {
|
|||
}
|
||||
|
||||
// ResendSegmentStats is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ResendSegmentStatsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.ResendSegmentStatsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) ResendSegmentStats(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ResendSegmentStats_Call {
|
||||
return &MockDataNodeClient_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1301,9 +1161,9 @@ type MockDataNodeClient_ShowConfigurations_Call struct {
|
|||
}
|
||||
|
||||
// ShowConfigurations is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ShowConfigurationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ShowConfigurationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ShowConfigurations_Call {
|
||||
return &MockDataNodeClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1371,9 +1231,9 @@ type MockDataNodeClient_SyncSegments_Call struct {
|
|||
}
|
||||
|
||||
// SyncSegments is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.SyncSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.SyncSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) SyncSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_SyncSegments_Call {
|
||||
return &MockDataNodeClient_SyncSegments_Call{Call: _e.mock.On("SyncSegments",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1441,9 +1301,9 @@ type MockDataNodeClient_WatchDmChannels_Call struct {
|
|||
}
|
||||
|
||||
// WatchDmChannels is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *datapb.WatchDmChannelsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *datapb.WatchDmChannelsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockDataNodeClient_Expecter) WatchDmChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_WatchDmChannels_Call {
|
||||
return &MockDataNodeClient_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
|
|
@ -68,8 +68,8 @@ type RootCoord_AllocID_Call struct {
|
|||
}
|
||||
|
||||
// AllocID is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *rootcoordpb.AllocIDRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *rootcoordpb.AllocIDRequest
|
||||
func (_e *RootCoord_Expecter) AllocID(_a0 interface{}, _a1 interface{}) *RootCoord_AllocID_Call {
|
||||
return &RootCoord_AllocID_Call{Call: _e.mock.On("AllocID", _a0, _a1)}
|
||||
}
|
||||
|
@ -123,8 +123,8 @@ type RootCoord_AllocTimestamp_Call struct {
|
|||
}
|
||||
|
||||
// AllocTimestamp is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *rootcoordpb.AllocTimestampRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *rootcoordpb.AllocTimestampRequest
|
||||
func (_e *RootCoord_Expecter) AllocTimestamp(_a0 interface{}, _a1 interface{}) *RootCoord_AllocTimestamp_Call {
|
||||
return &RootCoord_AllocTimestamp_Call{Call: _e.mock.On("AllocTimestamp", _a0, _a1)}
|
||||
}
|
||||
|
@ -178,8 +178,8 @@ type RootCoord_AlterAlias_Call struct {
|
|||
}
|
||||
|
||||
// AlterAlias is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.AlterAliasRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.AlterAliasRequest
|
||||
func (_e *RootCoord_Expecter) AlterAlias(_a0 interface{}, _a1 interface{}) *RootCoord_AlterAlias_Call {
|
||||
return &RootCoord_AlterAlias_Call{Call: _e.mock.On("AlterAlias", _a0, _a1)}
|
||||
}
|
||||
|
@ -233,8 +233,8 @@ type RootCoord_AlterCollection_Call struct {
|
|||
}
|
||||
|
||||
// AlterCollection is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.AlterCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.AlterCollectionRequest
|
||||
func (_e *RootCoord_Expecter) AlterCollection(_a0 interface{}, _a1 interface{}) *RootCoord_AlterCollection_Call {
|
||||
return &RootCoord_AlterCollection_Call{Call: _e.mock.On("AlterCollection", _a0, _a1)}
|
||||
}
|
||||
|
@ -288,8 +288,8 @@ type RootCoord_CheckHealth_Call struct {
|
|||
}
|
||||
|
||||
// CheckHealth is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CheckHealthRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CheckHealthRequest
|
||||
func (_e *RootCoord_Expecter) CheckHealth(_a0 interface{}, _a1 interface{}) *RootCoord_CheckHealth_Call {
|
||||
return &RootCoord_CheckHealth_Call{Call: _e.mock.On("CheckHealth", _a0, _a1)}
|
||||
}
|
||||
|
@ -343,8 +343,8 @@ type RootCoord_CreateAlias_Call struct {
|
|||
}
|
||||
|
||||
// CreateAlias is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateAliasRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateAliasRequest
|
||||
func (_e *RootCoord_Expecter) CreateAlias(_a0 interface{}, _a1 interface{}) *RootCoord_CreateAlias_Call {
|
||||
return &RootCoord_CreateAlias_Call{Call: _e.mock.On("CreateAlias", _a0, _a1)}
|
||||
}
|
||||
|
@ -398,8 +398,8 @@ type RootCoord_CreateCollection_Call struct {
|
|||
}
|
||||
|
||||
// CreateCollection is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateCollectionRequest
|
||||
func (_e *RootCoord_Expecter) CreateCollection(_a0 interface{}, _a1 interface{}) *RootCoord_CreateCollection_Call {
|
||||
return &RootCoord_CreateCollection_Call{Call: _e.mock.On("CreateCollection", _a0, _a1)}
|
||||
}
|
||||
|
@ -453,8 +453,8 @@ type RootCoord_CreateCredential_Call struct {
|
|||
}
|
||||
|
||||
// CreateCredential is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.CredentialInfo
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.CredentialInfo
|
||||
func (_e *RootCoord_Expecter) CreateCredential(_a0 interface{}, _a1 interface{}) *RootCoord_CreateCredential_Call {
|
||||
return &RootCoord_CreateCredential_Call{Call: _e.mock.On("CreateCredential", _a0, _a1)}
|
||||
}
|
||||
|
@ -508,8 +508,8 @@ type RootCoord_CreateDatabase_Call struct {
|
|||
}
|
||||
|
||||
// CreateDatabase is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateDatabaseRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateDatabaseRequest
|
||||
func (_e *RootCoord_Expecter) CreateDatabase(_a0 interface{}, _a1 interface{}) *RootCoord_CreateDatabase_Call {
|
||||
return &RootCoord_CreateDatabase_Call{Call: _e.mock.On("CreateDatabase", _a0, _a1)}
|
||||
}
|
||||
|
@ -563,8 +563,8 @@ type RootCoord_CreatePartition_Call struct {
|
|||
}
|
||||
|
||||
// CreatePartition is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreatePartitionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreatePartitionRequest
|
||||
func (_e *RootCoord_Expecter) CreatePartition(_a0 interface{}, _a1 interface{}) *RootCoord_CreatePartition_Call {
|
||||
return &RootCoord_CreatePartition_Call{Call: _e.mock.On("CreatePartition", _a0, _a1)}
|
||||
}
|
||||
|
@ -618,8 +618,8 @@ type RootCoord_CreateRole_Call struct {
|
|||
}
|
||||
|
||||
// CreateRole is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateRoleRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.CreateRoleRequest
|
||||
func (_e *RootCoord_Expecter) CreateRole(_a0 interface{}, _a1 interface{}) *RootCoord_CreateRole_Call {
|
||||
return &RootCoord_CreateRole_Call{Call: _e.mock.On("CreateRole", _a0, _a1)}
|
||||
}
|
||||
|
@ -673,8 +673,8 @@ type RootCoord_DeleteCredential_Call struct {
|
|||
}
|
||||
|
||||
// DeleteCredential is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DeleteCredentialRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DeleteCredentialRequest
|
||||
func (_e *RootCoord_Expecter) DeleteCredential(_a0 interface{}, _a1 interface{}) *RootCoord_DeleteCredential_Call {
|
||||
return &RootCoord_DeleteCredential_Call{Call: _e.mock.On("DeleteCredential", _a0, _a1)}
|
||||
}
|
||||
|
@ -728,8 +728,8 @@ type RootCoord_DescribeAlias_Call struct {
|
|||
}
|
||||
|
||||
// DescribeAlias is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DescribeAliasRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DescribeAliasRequest
|
||||
func (_e *RootCoord_Expecter) DescribeAlias(_a0 interface{}, _a1 interface{}) *RootCoord_DescribeAlias_Call {
|
||||
return &RootCoord_DescribeAlias_Call{Call: _e.mock.On("DescribeAlias", _a0, _a1)}
|
||||
}
|
||||
|
@ -783,8 +783,8 @@ type RootCoord_DescribeCollection_Call struct {
|
|||
}
|
||||
|
||||
// DescribeCollection is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DescribeCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DescribeCollectionRequest
|
||||
func (_e *RootCoord_Expecter) DescribeCollection(_a0 interface{}, _a1 interface{}) *RootCoord_DescribeCollection_Call {
|
||||
return &RootCoord_DescribeCollection_Call{Call: _e.mock.On("DescribeCollection", _a0, _a1)}
|
||||
}
|
||||
|
@ -838,8 +838,8 @@ type RootCoord_DescribeCollectionInternal_Call struct {
|
|||
}
|
||||
|
||||
// DescribeCollectionInternal is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DescribeCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DescribeCollectionRequest
|
||||
func (_e *RootCoord_Expecter) DescribeCollectionInternal(_a0 interface{}, _a1 interface{}) *RootCoord_DescribeCollectionInternal_Call {
|
||||
return &RootCoord_DescribeCollectionInternal_Call{Call: _e.mock.On("DescribeCollectionInternal", _a0, _a1)}
|
||||
}
|
||||
|
@ -893,8 +893,8 @@ type RootCoord_DropAlias_Call struct {
|
|||
}
|
||||
|
||||
// DropAlias is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropAliasRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropAliasRequest
|
||||
func (_e *RootCoord_Expecter) DropAlias(_a0 interface{}, _a1 interface{}) *RootCoord_DropAlias_Call {
|
||||
return &RootCoord_DropAlias_Call{Call: _e.mock.On("DropAlias", _a0, _a1)}
|
||||
}
|
||||
|
@ -948,8 +948,8 @@ type RootCoord_DropCollection_Call struct {
|
|||
}
|
||||
|
||||
// DropCollection is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropCollectionRequest
|
||||
func (_e *RootCoord_Expecter) DropCollection(_a0 interface{}, _a1 interface{}) *RootCoord_DropCollection_Call {
|
||||
return &RootCoord_DropCollection_Call{Call: _e.mock.On("DropCollection", _a0, _a1)}
|
||||
}
|
||||
|
@ -1003,8 +1003,8 @@ type RootCoord_DropDatabase_Call struct {
|
|||
}
|
||||
|
||||
// DropDatabase is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropDatabaseRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropDatabaseRequest
|
||||
func (_e *RootCoord_Expecter) DropDatabase(_a0 interface{}, _a1 interface{}) *RootCoord_DropDatabase_Call {
|
||||
return &RootCoord_DropDatabase_Call{Call: _e.mock.On("DropDatabase", _a0, _a1)}
|
||||
}
|
||||
|
@ -1058,8 +1058,8 @@ type RootCoord_DropPartition_Call struct {
|
|||
}
|
||||
|
||||
// DropPartition is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropPartitionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropPartitionRequest
|
||||
func (_e *RootCoord_Expecter) DropPartition(_a0 interface{}, _a1 interface{}) *RootCoord_DropPartition_Call {
|
||||
return &RootCoord_DropPartition_Call{Call: _e.mock.On("DropPartition", _a0, _a1)}
|
||||
}
|
||||
|
@ -1113,8 +1113,8 @@ type RootCoord_DropRole_Call struct {
|
|||
}
|
||||
|
||||
// DropRole is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropRoleRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.DropRoleRequest
|
||||
func (_e *RootCoord_Expecter) DropRole(_a0 interface{}, _a1 interface{}) *RootCoord_DropRole_Call {
|
||||
return &RootCoord_DropRole_Call{Call: _e.mock.On("DropRole", _a0, _a1)}
|
||||
}
|
||||
|
@ -1168,8 +1168,8 @@ type RootCoord_GetComponentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetComponentStates is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetComponentStatesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetComponentStatesRequest
|
||||
func (_e *RootCoord_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *RootCoord_GetComponentStates_Call {
|
||||
return &RootCoord_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)}
|
||||
}
|
||||
|
@ -1223,8 +1223,8 @@ type RootCoord_GetCredential_Call struct {
|
|||
}
|
||||
|
||||
// GetCredential is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *rootcoordpb.GetCredentialRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *rootcoordpb.GetCredentialRequest
|
||||
func (_e *RootCoord_Expecter) GetCredential(_a0 interface{}, _a1 interface{}) *RootCoord_GetCredential_Call {
|
||||
return &RootCoord_GetCredential_Call{Call: _e.mock.On("GetCredential", _a0, _a1)}
|
||||
}
|
||||
|
@ -1246,61 +1246,6 @@ func (_c *RootCoord_GetCredential_Call) RunAndReturn(run func(context.Context, *
|
|||
return _c
|
||||
}
|
||||
|
||||
// GetImportState provides a mock function with given fields: _a0, _a1
|
||||
func (_m *RootCoord) GetImportState(_a0 context.Context, _a1 *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *milvuspb.GetImportStateResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetImportStateRequest) *milvuspb.GetImportStateResponse); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*milvuspb.GetImportStateResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetImportStateRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// RootCoord_GetImportState_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetImportState'
|
||||
type RootCoord_GetImportState_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// GetImportState is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.GetImportStateRequest
|
||||
func (_e *RootCoord_Expecter) GetImportState(_a0 interface{}, _a1 interface{}) *RootCoord_GetImportState_Call {
|
||||
return &RootCoord_GetImportState_Call{Call: _e.mock.On("GetImportState", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *RootCoord_GetImportState_Call) Run(run func(_a0 context.Context, _a1 *milvuspb.GetImportStateRequest)) *RootCoord_GetImportState_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*milvuspb.GetImportStateRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_GetImportState_Call) Return(_a0 *milvuspb.GetImportStateResponse, _a1 error) *RootCoord_GetImportState_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_GetImportState_Call) RunAndReturn(run func(context.Context, *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error)) *RootCoord_GetImportState_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// GetMetrics provides a mock function with given fields: ctx, req
|
||||
func (_m *RootCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
||||
ret := _m.Called(ctx, req)
|
||||
|
@ -1333,8 +1278,8 @@ type RootCoord_GetMetrics_Call struct {
|
|||
}
|
||||
|
||||
// GetMetrics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - req *milvuspb.GetMetricsRequest
|
||||
// - ctx context.Context
|
||||
// - req *milvuspb.GetMetricsRequest
|
||||
func (_e *RootCoord_Expecter) GetMetrics(ctx interface{}, req interface{}) *RootCoord_GetMetrics_Call {
|
||||
return &RootCoord_GetMetrics_Call{Call: _e.mock.On("GetMetrics", ctx, req)}
|
||||
}
|
||||
|
@ -1388,8 +1333,8 @@ type RootCoord_GetStatisticsChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetStatisticsChannel is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetStatisticsChannelRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetStatisticsChannelRequest
|
||||
func (_e *RootCoord_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *RootCoord_GetStatisticsChannel_Call {
|
||||
return &RootCoord_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)}
|
||||
}
|
||||
|
@ -1443,8 +1388,8 @@ type RootCoord_GetTimeTickChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetTimeTickChannel is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetTimeTickChannelRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.GetTimeTickChannelRequest
|
||||
func (_e *RootCoord_Expecter) GetTimeTickChannel(_a0 interface{}, _a1 interface{}) *RootCoord_GetTimeTickChannel_Call {
|
||||
return &RootCoord_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel", _a0, _a1)}
|
||||
}
|
||||
|
@ -1498,8 +1443,8 @@ type RootCoord_HasCollection_Call struct {
|
|||
}
|
||||
|
||||
// HasCollection is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.HasCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.HasCollectionRequest
|
||||
func (_e *RootCoord_Expecter) HasCollection(_a0 interface{}, _a1 interface{}) *RootCoord_HasCollection_Call {
|
||||
return &RootCoord_HasCollection_Call{Call: _e.mock.On("HasCollection", _a0, _a1)}
|
||||
}
|
||||
|
@ -1553,8 +1498,8 @@ type RootCoord_HasPartition_Call struct {
|
|||
}
|
||||
|
||||
// HasPartition is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.HasPartitionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.HasPartitionRequest
|
||||
func (_e *RootCoord_Expecter) HasPartition(_a0 interface{}, _a1 interface{}) *RootCoord_HasPartition_Call {
|
||||
return &RootCoord_HasPartition_Call{Call: _e.mock.On("HasPartition", _a0, _a1)}
|
||||
}
|
||||
|
@ -1576,61 +1521,6 @@ func (_c *RootCoord_HasPartition_Call) RunAndReturn(run func(context.Context, *m
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: _a0, _a1
|
||||
func (_m *RootCoord) Import(_a0 context.Context, _a1 *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *milvuspb.ImportResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ImportRequest) *milvuspb.ImportResponse); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*milvuspb.ImportResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.ImportRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// RootCoord_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type RootCoord_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ImportRequest
|
||||
func (_e *RootCoord_Expecter) Import(_a0 interface{}, _a1 interface{}) *RootCoord_Import_Call {
|
||||
return &RootCoord_Import_Call{Call: _e.mock.On("Import", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *RootCoord_Import_Call) Run(run func(_a0 context.Context, _a1 *milvuspb.ImportRequest)) *RootCoord_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*milvuspb.ImportRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_Import_Call) Return(_a0 *milvuspb.ImportResponse, _a1 error) *RootCoord_Import_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_Import_Call) RunAndReturn(run func(context.Context, *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error)) *RootCoord_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// Init provides a mock function with given fields:
|
||||
func (_m *RootCoord) Init() error {
|
||||
ret := _m.Called()
|
||||
|
@ -1704,8 +1594,8 @@ type RootCoord_InvalidateCollectionMetaCache_Call struct {
|
|||
}
|
||||
|
||||
// InvalidateCollectionMetaCache is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *proxypb.InvalidateCollMetaCacheRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *proxypb.InvalidateCollMetaCacheRequest
|
||||
func (_e *RootCoord_Expecter) InvalidateCollectionMetaCache(_a0 interface{}, _a1 interface{}) *RootCoord_InvalidateCollectionMetaCache_Call {
|
||||
return &RootCoord_InvalidateCollectionMetaCache_Call{Call: _e.mock.On("InvalidateCollectionMetaCache", _a0, _a1)}
|
||||
}
|
||||
|
@ -1759,8 +1649,8 @@ type RootCoord_ListAliases_Call struct {
|
|||
}
|
||||
|
||||
// ListAliases is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ListAliasesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ListAliasesRequest
|
||||
func (_e *RootCoord_Expecter) ListAliases(_a0 interface{}, _a1 interface{}) *RootCoord_ListAliases_Call {
|
||||
return &RootCoord_ListAliases_Call{Call: _e.mock.On("ListAliases", _a0, _a1)}
|
||||
}
|
||||
|
@ -1814,8 +1704,8 @@ type RootCoord_ListCredUsers_Call struct {
|
|||
}
|
||||
|
||||
// ListCredUsers is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ListCredUsersRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ListCredUsersRequest
|
||||
func (_e *RootCoord_Expecter) ListCredUsers(_a0 interface{}, _a1 interface{}) *RootCoord_ListCredUsers_Call {
|
||||
return &RootCoord_ListCredUsers_Call{Call: _e.mock.On("ListCredUsers", _a0, _a1)}
|
||||
}
|
||||
|
@ -1869,8 +1759,8 @@ type RootCoord_ListDatabases_Call struct {
|
|||
}
|
||||
|
||||
// ListDatabases is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ListDatabasesRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ListDatabasesRequest
|
||||
func (_e *RootCoord_Expecter) ListDatabases(_a0 interface{}, _a1 interface{}) *RootCoord_ListDatabases_Call {
|
||||
return &RootCoord_ListDatabases_Call{Call: _e.mock.On("ListDatabases", _a0, _a1)}
|
||||
}
|
||||
|
@ -1892,61 +1782,6 @@ func (_c *RootCoord_ListDatabases_Call) RunAndReturn(run func(context.Context, *
|
|||
return _c
|
||||
}
|
||||
|
||||
// ListImportTasks provides a mock function with given fields: _a0, _a1
|
||||
func (_m *RootCoord) ListImportTasks(_a0 context.Context, _a1 *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *milvuspb.ListImportTasksResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ListImportTasksRequest) *milvuspb.ListImportTasksResponse); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*milvuspb.ListImportTasksResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.ListImportTasksRequest) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// RootCoord_ListImportTasks_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListImportTasks'
|
||||
type RootCoord_ListImportTasks_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// ListImportTasks is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ListImportTasksRequest
|
||||
func (_e *RootCoord_Expecter) ListImportTasks(_a0 interface{}, _a1 interface{}) *RootCoord_ListImportTasks_Call {
|
||||
return &RootCoord_ListImportTasks_Call{Call: _e.mock.On("ListImportTasks", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *RootCoord_ListImportTasks_Call) Run(run func(_a0 context.Context, _a1 *milvuspb.ListImportTasksRequest)) *RootCoord_ListImportTasks_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*milvuspb.ListImportTasksRequest))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_ListImportTasks_Call) Return(_a0 *milvuspb.ListImportTasksResponse, _a1 error) *RootCoord_ListImportTasks_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_ListImportTasks_Call) RunAndReturn(run func(context.Context, *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error)) *RootCoord_ListImportTasks_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ListPolicy provides a mock function with given fields: _a0, _a1
|
||||
func (_m *RootCoord) ListPolicy(_a0 context.Context, _a1 *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
@ -1979,8 +1814,8 @@ type RootCoord_ListPolicy_Call struct {
|
|||
}
|
||||
|
||||
// ListPolicy is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ListPolicyRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ListPolicyRequest
|
||||
func (_e *RootCoord_Expecter) ListPolicy(_a0 interface{}, _a1 interface{}) *RootCoord_ListPolicy_Call {
|
||||
return &RootCoord_ListPolicy_Call{Call: _e.mock.On("ListPolicy", _a0, _a1)}
|
||||
}
|
||||
|
@ -2034,8 +1869,8 @@ type RootCoord_OperatePrivilege_Call struct {
|
|||
}
|
||||
|
||||
// OperatePrivilege is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.OperatePrivilegeRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.OperatePrivilegeRequest
|
||||
func (_e *RootCoord_Expecter) OperatePrivilege(_a0 interface{}, _a1 interface{}) *RootCoord_OperatePrivilege_Call {
|
||||
return &RootCoord_OperatePrivilege_Call{Call: _e.mock.On("OperatePrivilege", _a0, _a1)}
|
||||
}
|
||||
|
@ -2089,8 +1924,8 @@ type RootCoord_OperateUserRole_Call struct {
|
|||
}
|
||||
|
||||
// OperateUserRole is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.OperateUserRoleRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.OperateUserRoleRequest
|
||||
func (_e *RootCoord_Expecter) OperateUserRole(_a0 interface{}, _a1 interface{}) *RootCoord_OperateUserRole_Call {
|
||||
return &RootCoord_OperateUserRole_Call{Call: _e.mock.On("OperateUserRole", _a0, _a1)}
|
||||
}
|
||||
|
@ -2185,8 +2020,8 @@ type RootCoord_RenameCollection_Call struct {
|
|||
}
|
||||
|
||||
// RenameCollection is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.RenameCollectionRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.RenameCollectionRequest
|
||||
func (_e *RootCoord_Expecter) RenameCollection(_a0 interface{}, _a1 interface{}) *RootCoord_RenameCollection_Call {
|
||||
return &RootCoord_RenameCollection_Call{Call: _e.mock.On("RenameCollection", _a0, _a1)}
|
||||
}
|
||||
|
@ -2208,61 +2043,6 @@ func (_c *RootCoord_RenameCollection_Call) RunAndReturn(run func(context.Context
|
|||
return _c
|
||||
}
|
||||
|
||||
// ReportImport provides a mock function with given fields: _a0, _a1
|
||||
func (_m *RootCoord) ReportImport(_a0 context.Context, _a1 *rootcoordpb.ImportResult) (*commonpb.Status, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ImportResult) (*commonpb.Status, error)); ok {
|
||||
return rf(_a0, _a1)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ImportResult) *commonpb.Status); ok {
|
||||
r0 = rf(_a0, _a1)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *rootcoordpb.ImportResult) error); ok {
|
||||
r1 = rf(_a0, _a1)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// RootCoord_ReportImport_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ReportImport'
|
||||
type RootCoord_ReportImport_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// ReportImport is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *rootcoordpb.ImportResult
|
||||
func (_e *RootCoord_Expecter) ReportImport(_a0 interface{}, _a1 interface{}) *RootCoord_ReportImport_Call {
|
||||
return &RootCoord_ReportImport_Call{Call: _e.mock.On("ReportImport", _a0, _a1)}
|
||||
}
|
||||
|
||||
func (_c *RootCoord_ReportImport_Call) Run(run func(_a0 context.Context, _a1 *rootcoordpb.ImportResult)) *RootCoord_ReportImport_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].(*rootcoordpb.ImportResult))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_ReportImport_Call) Return(_a0 *commonpb.Status, _a1 error) *RootCoord_ReportImport_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *RootCoord_ReportImport_Call) RunAndReturn(run func(context.Context, *rootcoordpb.ImportResult) (*commonpb.Status, error)) *RootCoord_ReportImport_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SelectGrant provides a mock function with given fields: _a0, _a1
|
||||
func (_m *RootCoord) SelectGrant(_a0 context.Context, _a1 *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) {
|
||||
ret := _m.Called(_a0, _a1)
|
||||
|
@ -2295,8 +2075,8 @@ type RootCoord_SelectGrant_Call struct {
|
|||
}
|
||||
|
||||
// SelectGrant is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.SelectGrantRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.SelectGrantRequest
|
||||
func (_e *RootCoord_Expecter) SelectGrant(_a0 interface{}, _a1 interface{}) *RootCoord_SelectGrant_Call {
|
||||
return &RootCoord_SelectGrant_Call{Call: _e.mock.On("SelectGrant", _a0, _a1)}
|
||||
}
|
||||
|
@ -2350,8 +2130,8 @@ type RootCoord_SelectRole_Call struct {
|
|||
}
|
||||
|
||||
// SelectRole is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.SelectRoleRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.SelectRoleRequest
|
||||
func (_e *RootCoord_Expecter) SelectRole(_a0 interface{}, _a1 interface{}) *RootCoord_SelectRole_Call {
|
||||
return &RootCoord_SelectRole_Call{Call: _e.mock.On("SelectRole", _a0, _a1)}
|
||||
}
|
||||
|
@ -2405,8 +2185,8 @@ type RootCoord_SelectUser_Call struct {
|
|||
}
|
||||
|
||||
// SelectUser is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.SelectUserRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.SelectUserRequest
|
||||
func (_e *RootCoord_Expecter) SelectUser(_a0 interface{}, _a1 interface{}) *RootCoord_SelectUser_Call {
|
||||
return &RootCoord_SelectUser_Call{Call: _e.mock.On("SelectUser", _a0, _a1)}
|
||||
}
|
||||
|
@ -2439,7 +2219,7 @@ type RootCoord_SetAddress_Call struct {
|
|||
}
|
||||
|
||||
// SetAddress is a helper method to define mock.On call
|
||||
// - address string
|
||||
// - address string
|
||||
func (_e *RootCoord_Expecter) SetAddress(address interface{}) *RootCoord_SetAddress_Call {
|
||||
return &RootCoord_SetAddress_Call{Call: _e.mock.On("SetAddress", address)}
|
||||
}
|
||||
|
@ -2481,7 +2261,7 @@ type RootCoord_SetDataCoordClient_Call struct {
|
|||
}
|
||||
|
||||
// SetDataCoordClient is a helper method to define mock.On call
|
||||
// - dataCoord types.DataCoordClient
|
||||
// - dataCoord types.DataCoordClient
|
||||
func (_e *RootCoord_Expecter) SetDataCoordClient(dataCoord interface{}) *RootCoord_SetDataCoordClient_Call {
|
||||
return &RootCoord_SetDataCoordClient_Call{Call: _e.mock.On("SetDataCoordClient", dataCoord)}
|
||||
}
|
||||
|
@ -2514,7 +2294,7 @@ type RootCoord_SetEtcdClient_Call struct {
|
|||
}
|
||||
|
||||
// SetEtcdClient is a helper method to define mock.On call
|
||||
// - etcdClient *clientv3.Client
|
||||
// - etcdClient *clientv3.Client
|
||||
func (_e *RootCoord_Expecter) SetEtcdClient(etcdClient interface{}) *RootCoord_SetEtcdClient_Call {
|
||||
return &RootCoord_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)}
|
||||
}
|
||||
|
@ -2547,7 +2327,7 @@ type RootCoord_SetProxyCreator_Call struct {
|
|||
}
|
||||
|
||||
// SetProxyCreator is a helper method to define mock.On call
|
||||
// - _a0 func(context.Context , string , int64)(types.ProxyClient , error)
|
||||
// - _a0 func(context.Context , string , int64)(types.ProxyClient , error)
|
||||
func (_e *RootCoord_Expecter) SetProxyCreator(_a0 interface{}) *RootCoord_SetProxyCreator_Call {
|
||||
return &RootCoord_SetProxyCreator_Call{Call: _e.mock.On("SetProxyCreator", _a0)}
|
||||
}
|
||||
|
@ -2589,7 +2369,7 @@ type RootCoord_SetQueryCoordClient_Call struct {
|
|||
}
|
||||
|
||||
// SetQueryCoordClient is a helper method to define mock.On call
|
||||
// - queryCoord types.QueryCoordClient
|
||||
// - queryCoord types.QueryCoordClient
|
||||
func (_e *RootCoord_Expecter) SetQueryCoordClient(queryCoord interface{}) *RootCoord_SetQueryCoordClient_Call {
|
||||
return &RootCoord_SetQueryCoordClient_Call{Call: _e.mock.On("SetQueryCoordClient", queryCoord)}
|
||||
}
|
||||
|
@ -2622,7 +2402,7 @@ type RootCoord_SetTiKVClient_Call struct {
|
|||
}
|
||||
|
||||
// SetTiKVClient is a helper method to define mock.On call
|
||||
// - client *txnkv.Client
|
||||
// - client *txnkv.Client
|
||||
func (_e *RootCoord_Expecter) SetTiKVClient(client interface{}) *RootCoord_SetTiKVClient_Call {
|
||||
return &RootCoord_SetTiKVClient_Call{Call: _e.mock.On("SetTiKVClient", client)}
|
||||
}
|
||||
|
@ -2676,8 +2456,8 @@ type RootCoord_ShowCollections_Call struct {
|
|||
}
|
||||
|
||||
// ShowCollections is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowCollectionsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowCollectionsRequest
|
||||
func (_e *RootCoord_Expecter) ShowCollections(_a0 interface{}, _a1 interface{}) *RootCoord_ShowCollections_Call {
|
||||
return &RootCoord_ShowCollections_Call{Call: _e.mock.On("ShowCollections", _a0, _a1)}
|
||||
}
|
||||
|
@ -2731,8 +2511,8 @@ type RootCoord_ShowConfigurations_Call struct {
|
|||
}
|
||||
|
||||
// ShowConfigurations is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ShowConfigurationsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ShowConfigurationsRequest
|
||||
func (_e *RootCoord_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *RootCoord_ShowConfigurations_Call {
|
||||
return &RootCoord_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)}
|
||||
}
|
||||
|
@ -2786,8 +2566,8 @@ type RootCoord_ShowPartitions_Call struct {
|
|||
}
|
||||
|
||||
// ShowPartitions is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowPartitionsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowPartitionsRequest
|
||||
func (_e *RootCoord_Expecter) ShowPartitions(_a0 interface{}, _a1 interface{}) *RootCoord_ShowPartitions_Call {
|
||||
return &RootCoord_ShowPartitions_Call{Call: _e.mock.On("ShowPartitions", _a0, _a1)}
|
||||
}
|
||||
|
@ -2841,8 +2621,8 @@ type RootCoord_ShowPartitionsInternal_Call struct {
|
|||
}
|
||||
|
||||
// ShowPartitionsInternal is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowPartitionsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowPartitionsRequest
|
||||
func (_e *RootCoord_Expecter) ShowPartitionsInternal(_a0 interface{}, _a1 interface{}) *RootCoord_ShowPartitionsInternal_Call {
|
||||
return &RootCoord_ShowPartitionsInternal_Call{Call: _e.mock.On("ShowPartitionsInternal", _a0, _a1)}
|
||||
}
|
||||
|
@ -2896,8 +2676,8 @@ type RootCoord_ShowSegments_Call struct {
|
|||
}
|
||||
|
||||
// ShowSegments is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowSegmentsRequest
|
||||
// - _a0 context.Context
|
||||
// - _a1 *milvuspb.ShowSegmentsRequest
|
||||
func (_e *RootCoord_Expecter) ShowSegments(_a0 interface{}, _a1 interface{}) *RootCoord_ShowSegments_Call {
|
||||
return &RootCoord_ShowSegments_Call{Call: _e.mock.On("ShowSegments", _a0, _a1)}
|
||||
}
|
||||
|
@ -3033,8 +2813,8 @@ type RootCoord_UpdateChannelTimeTick_Call struct {
|
|||
}
|
||||
|
||||
// UpdateChannelTimeTick is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ChannelTimeTickMsg
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.ChannelTimeTickMsg
|
||||
func (_e *RootCoord_Expecter) UpdateChannelTimeTick(_a0 interface{}, _a1 interface{}) *RootCoord_UpdateChannelTimeTick_Call {
|
||||
return &RootCoord_UpdateChannelTimeTick_Call{Call: _e.mock.On("UpdateChannelTimeTick", _a0, _a1)}
|
||||
}
|
||||
|
@ -3088,8 +2868,8 @@ type RootCoord_UpdateCredential_Call struct {
|
|||
}
|
||||
|
||||
// UpdateCredential is a helper method to define mock.On call
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.CredentialInfo
|
||||
// - _a0 context.Context
|
||||
// - _a1 *internalpb.CredentialInfo
|
||||
func (_e *RootCoord_Expecter) UpdateCredential(_a0 interface{}, _a1 interface{}) *RootCoord_UpdateCredential_Call {
|
||||
return &RootCoord_UpdateCredential_Call{Call: _e.mock.On("UpdateCredential", _a0, _a1)}
|
||||
}
|
||||
|
@ -3122,7 +2902,7 @@ type RootCoord_UpdateStateCode_Call struct {
|
|||
}
|
||||
|
||||
// UpdateStateCode is a helper method to define mock.On call
|
||||
// - _a0 commonpb.StateCode
|
||||
// - _a0 commonpb.StateCode
|
||||
func (_e *RootCoord_Expecter) UpdateStateCode(_a0 interface{}) *RootCoord_UpdateStateCode_Call {
|
||||
return &RootCoord_UpdateStateCode_Call{Call: _e.mock.On("UpdateStateCode", _a0)}
|
||||
}
|
||||
|
|
|
@ -72,9 +72,9 @@ type MockRootCoordClient_AllocID_Call struct {
|
|||
}
|
||||
|
||||
// AllocID is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *rootcoordpb.AllocIDRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *rootcoordpb.AllocIDRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) AllocID(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_AllocID_Call {
|
||||
return &MockRootCoordClient_AllocID_Call{Call: _e.mock.On("AllocID",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -142,9 +142,9 @@ type MockRootCoordClient_AllocTimestamp_Call struct {
|
|||
}
|
||||
|
||||
// AllocTimestamp is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *rootcoordpb.AllocTimestampRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *rootcoordpb.AllocTimestampRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) AllocTimestamp(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_AllocTimestamp_Call {
|
||||
return &MockRootCoordClient_AllocTimestamp_Call{Call: _e.mock.On("AllocTimestamp",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -212,9 +212,9 @@ type MockRootCoordClient_AlterAlias_Call struct {
|
|||
}
|
||||
|
||||
// AlterAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.AlterAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.AlterAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) AlterAlias(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_AlterAlias_Call {
|
||||
return &MockRootCoordClient_AlterAlias_Call{Call: _e.mock.On("AlterAlias",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -282,9 +282,9 @@ type MockRootCoordClient_AlterCollection_Call struct {
|
|||
}
|
||||
|
||||
// AlterCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.AlterCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.AlterCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) AlterCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_AlterCollection_Call {
|
||||
return &MockRootCoordClient_AlterCollection_Call{Call: _e.mock.On("AlterCollection",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -352,9 +352,9 @@ type MockRootCoordClient_CheckHealth_Call struct {
|
|||
}
|
||||
|
||||
// CheckHealth is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CheckHealthRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CheckHealthRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) CheckHealth(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_CheckHealth_Call {
|
||||
return &MockRootCoordClient_CheckHealth_Call{Call: _e.mock.On("CheckHealth",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -463,9 +463,9 @@ type MockRootCoordClient_CreateAlias_Call struct {
|
|||
}
|
||||
|
||||
// CreateAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) CreateAlias(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_CreateAlias_Call {
|
||||
return &MockRootCoordClient_CreateAlias_Call{Call: _e.mock.On("CreateAlias",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -533,9 +533,9 @@ type MockRootCoordClient_CreateCollection_Call struct {
|
|||
}
|
||||
|
||||
// CreateCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) CreateCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_CreateCollection_Call {
|
||||
return &MockRootCoordClient_CreateCollection_Call{Call: _e.mock.On("CreateCollection",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -603,9 +603,9 @@ type MockRootCoordClient_CreateCredential_Call struct {
|
|||
}
|
||||
|
||||
// CreateCredential is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.CredentialInfo
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.CredentialInfo
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) CreateCredential(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_CreateCredential_Call {
|
||||
return &MockRootCoordClient_CreateCredential_Call{Call: _e.mock.On("CreateCredential",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -673,9 +673,9 @@ type MockRootCoordClient_CreateDatabase_Call struct {
|
|||
}
|
||||
|
||||
// CreateDatabase is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateDatabaseRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateDatabaseRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) CreateDatabase(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_CreateDatabase_Call {
|
||||
return &MockRootCoordClient_CreateDatabase_Call{Call: _e.mock.On("CreateDatabase",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -743,9 +743,9 @@ type MockRootCoordClient_CreatePartition_Call struct {
|
|||
}
|
||||
|
||||
// CreatePartition is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreatePartitionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreatePartitionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) CreatePartition(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_CreatePartition_Call {
|
||||
return &MockRootCoordClient_CreatePartition_Call{Call: _e.mock.On("CreatePartition",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -813,9 +813,9 @@ type MockRootCoordClient_CreateRole_Call struct {
|
|||
}
|
||||
|
||||
// CreateRole is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.CreateRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) CreateRole(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_CreateRole_Call {
|
||||
return &MockRootCoordClient_CreateRole_Call{Call: _e.mock.On("CreateRole",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -883,9 +883,9 @@ type MockRootCoordClient_DeleteCredential_Call struct {
|
|||
}
|
||||
|
||||
// DeleteCredential is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DeleteCredentialRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DeleteCredentialRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DeleteCredential(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DeleteCredential_Call {
|
||||
return &MockRootCoordClient_DeleteCredential_Call{Call: _e.mock.On("DeleteCredential",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -953,9 +953,9 @@ type MockRootCoordClient_DescribeAlias_Call struct {
|
|||
}
|
||||
|
||||
// DescribeAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DescribeAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DescribeAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DescribeAlias(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DescribeAlias_Call {
|
||||
return &MockRootCoordClient_DescribeAlias_Call{Call: _e.mock.On("DescribeAlias",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1023,9 +1023,9 @@ type MockRootCoordClient_DescribeCollection_Call struct {
|
|||
}
|
||||
|
||||
// DescribeCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DescribeCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DescribeCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DescribeCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DescribeCollection_Call {
|
||||
return &MockRootCoordClient_DescribeCollection_Call{Call: _e.mock.On("DescribeCollection",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1093,9 +1093,9 @@ type MockRootCoordClient_DescribeCollectionInternal_Call struct {
|
|||
}
|
||||
|
||||
// DescribeCollectionInternal is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DescribeCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DescribeCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DescribeCollectionInternal(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DescribeCollectionInternal_Call {
|
||||
return &MockRootCoordClient_DescribeCollectionInternal_Call{Call: _e.mock.On("DescribeCollectionInternal",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1163,9 +1163,9 @@ type MockRootCoordClient_DropAlias_Call struct {
|
|||
}
|
||||
|
||||
// DropAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropAliasRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DropAlias(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DropAlias_Call {
|
||||
return &MockRootCoordClient_DropAlias_Call{Call: _e.mock.On("DropAlias",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1233,9 +1233,9 @@ type MockRootCoordClient_DropCollection_Call struct {
|
|||
}
|
||||
|
||||
// DropCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DropCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DropCollection_Call {
|
||||
return &MockRootCoordClient_DropCollection_Call{Call: _e.mock.On("DropCollection",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1303,9 +1303,9 @@ type MockRootCoordClient_DropDatabase_Call struct {
|
|||
}
|
||||
|
||||
// DropDatabase is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropDatabaseRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropDatabaseRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DropDatabase(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DropDatabase_Call {
|
||||
return &MockRootCoordClient_DropDatabase_Call{Call: _e.mock.On("DropDatabase",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1373,9 +1373,9 @@ type MockRootCoordClient_DropPartition_Call struct {
|
|||
}
|
||||
|
||||
// DropPartition is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropPartitionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropPartitionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DropPartition(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DropPartition_Call {
|
||||
return &MockRootCoordClient_DropPartition_Call{Call: _e.mock.On("DropPartition",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1443,9 +1443,9 @@ type MockRootCoordClient_DropRole_Call struct {
|
|||
}
|
||||
|
||||
// DropRole is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.DropRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) DropRole(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_DropRole_Call {
|
||||
return &MockRootCoordClient_DropRole_Call{Call: _e.mock.On("DropRole",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1513,9 +1513,9 @@ type MockRootCoordClient_GetComponentStates_Call struct {
|
|||
}
|
||||
|
||||
// GetComponentStates is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetComponentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetComponentStatesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_GetComponentStates_Call {
|
||||
return &MockRootCoordClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1583,9 +1583,9 @@ type MockRootCoordClient_GetCredential_Call struct {
|
|||
}
|
||||
|
||||
// GetCredential is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *rootcoordpb.GetCredentialRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *rootcoordpb.GetCredentialRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) GetCredential(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_GetCredential_Call {
|
||||
return &MockRootCoordClient_GetCredential_Call{Call: _e.mock.On("GetCredential",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1614,76 +1614,6 @@ func (_c *MockRootCoordClient_GetCredential_Call) RunAndReturn(run func(context.
|
|||
return _c
|
||||
}
|
||||
|
||||
// GetImportState provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) GetImportState(ctx context.Context, in *milvuspb.GetImportStateRequest, opts ...grpc.CallOption) (*milvuspb.GetImportStateResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *milvuspb.GetImportStateResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetImportStateRequest, ...grpc.CallOption) (*milvuspb.GetImportStateResponse, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetImportStateRequest, ...grpc.CallOption) *milvuspb.GetImportStateResponse); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*milvuspb.GetImportStateResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetImportStateRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockRootCoordClient_GetImportState_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetImportState'
|
||||
type MockRootCoordClient_GetImportState_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// GetImportState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetImportStateRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) GetImportState(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_GetImportState_Call {
|
||||
return &MockRootCoordClient_GetImportState_Call{Call: _e.mock.On("GetImportState",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_GetImportState_Call) Run(run func(ctx context.Context, in *milvuspb.GetImportStateRequest, opts ...grpc.CallOption)) *MockRootCoordClient_GetImportState_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*milvuspb.GetImportStateRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_GetImportState_Call) Return(_a0 *milvuspb.GetImportStateResponse, _a1 error) *MockRootCoordClient_GetImportState_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_GetImportState_Call) RunAndReturn(run func(context.Context, *milvuspb.GetImportStateRequest, ...grpc.CallOption) (*milvuspb.GetImportStateResponse, error)) *MockRootCoordClient_GetImportState_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// GetMetrics provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -1723,9 +1653,9 @@ type MockRootCoordClient_GetMetrics_Call struct {
|
|||
}
|
||||
|
||||
// GetMetrics is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetMetricsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.GetMetricsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_GetMetrics_Call {
|
||||
return &MockRootCoordClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1793,9 +1723,9 @@ type MockRootCoordClient_GetStatisticsChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetStatisticsChannel is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetStatisticsChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetStatisticsChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_GetStatisticsChannel_Call {
|
||||
return &MockRootCoordClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1863,9 +1793,9 @@ type MockRootCoordClient_GetTimeTickChannel_Call struct {
|
|||
}
|
||||
|
||||
// GetTimeTickChannel is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetTimeTickChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.GetTimeTickChannelRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) GetTimeTickChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_GetTimeTickChannel_Call {
|
||||
return &MockRootCoordClient_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -1933,9 +1863,9 @@ type MockRootCoordClient_HasCollection_Call struct {
|
|||
}
|
||||
|
||||
// HasCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.HasCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.HasCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) HasCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_HasCollection_Call {
|
||||
return &MockRootCoordClient_HasCollection_Call{Call: _e.mock.On("HasCollection",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2003,9 +1933,9 @@ type MockRootCoordClient_HasPartition_Call struct {
|
|||
}
|
||||
|
||||
// HasPartition is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.HasPartitionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.HasPartitionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) HasPartition(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_HasPartition_Call {
|
||||
return &MockRootCoordClient_HasPartition_Call{Call: _e.mock.On("HasPartition",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2034,76 +1964,6 @@ func (_c *MockRootCoordClient_HasPartition_Call) RunAndReturn(run func(context.C
|
|||
return _c
|
||||
}
|
||||
|
||||
// Import provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) Import(ctx context.Context, in *milvuspb.ImportRequest, opts ...grpc.CallOption) (*milvuspb.ImportResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *milvuspb.ImportResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ImportRequest, ...grpc.CallOption) (*milvuspb.ImportResponse, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ImportRequest, ...grpc.CallOption) *milvuspb.ImportResponse); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*milvuspb.ImportResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.ImportRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockRootCoordClient_Import_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Import'
|
||||
type MockRootCoordClient_Import_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// Import is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ImportRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) Import(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_Import_Call {
|
||||
return &MockRootCoordClient_Import_Call{Call: _e.mock.On("Import",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_Import_Call) Run(run func(ctx context.Context, in *milvuspb.ImportRequest, opts ...grpc.CallOption)) *MockRootCoordClient_Import_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*milvuspb.ImportRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_Import_Call) Return(_a0 *milvuspb.ImportResponse, _a1 error) *MockRootCoordClient_Import_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_Import_Call) RunAndReturn(run func(context.Context, *milvuspb.ImportRequest, ...grpc.CallOption) (*milvuspb.ImportResponse, error)) *MockRootCoordClient_Import_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// InvalidateCollectionMetaCache provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -2143,9 +2003,9 @@ type MockRootCoordClient_InvalidateCollectionMetaCache_Call struct {
|
|||
}
|
||||
|
||||
// InvalidateCollectionMetaCache is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *proxypb.InvalidateCollMetaCacheRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *proxypb.InvalidateCollMetaCacheRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) InvalidateCollectionMetaCache(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_InvalidateCollectionMetaCache_Call {
|
||||
return &MockRootCoordClient_InvalidateCollectionMetaCache_Call{Call: _e.mock.On("InvalidateCollectionMetaCache",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2213,9 +2073,9 @@ type MockRootCoordClient_ListAliases_Call struct {
|
|||
}
|
||||
|
||||
// ListAliases is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ListAliasesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ListAliasesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ListAliases(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ListAliases_Call {
|
||||
return &MockRootCoordClient_ListAliases_Call{Call: _e.mock.On("ListAliases",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2283,9 +2143,9 @@ type MockRootCoordClient_ListCredUsers_Call struct {
|
|||
}
|
||||
|
||||
// ListCredUsers is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ListCredUsersRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ListCredUsersRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ListCredUsers(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ListCredUsers_Call {
|
||||
return &MockRootCoordClient_ListCredUsers_Call{Call: _e.mock.On("ListCredUsers",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2353,9 +2213,9 @@ type MockRootCoordClient_ListDatabases_Call struct {
|
|||
}
|
||||
|
||||
// ListDatabases is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ListDatabasesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ListDatabasesRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ListDatabases(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ListDatabases_Call {
|
||||
return &MockRootCoordClient_ListDatabases_Call{Call: _e.mock.On("ListDatabases",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2384,76 +2244,6 @@ func (_c *MockRootCoordClient_ListDatabases_Call) RunAndReturn(run func(context.
|
|||
return _c
|
||||
}
|
||||
|
||||
// ListImportTasks provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) ListImportTasks(ctx context.Context, in *milvuspb.ListImportTasksRequest, opts ...grpc.CallOption) (*milvuspb.ListImportTasksResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *milvuspb.ListImportTasksResponse
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ListImportTasksRequest, ...grpc.CallOption) (*milvuspb.ListImportTasksResponse, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.ListImportTasksRequest, ...grpc.CallOption) *milvuspb.ListImportTasksResponse); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*milvuspb.ListImportTasksResponse)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.ListImportTasksRequest, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockRootCoordClient_ListImportTasks_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListImportTasks'
|
||||
type MockRootCoordClient_ListImportTasks_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// ListImportTasks is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ListImportTasksRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ListImportTasks(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ListImportTasks_Call {
|
||||
return &MockRootCoordClient_ListImportTasks_Call{Call: _e.mock.On("ListImportTasks",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_ListImportTasks_Call) Run(run func(ctx context.Context, in *milvuspb.ListImportTasksRequest, opts ...grpc.CallOption)) *MockRootCoordClient_ListImportTasks_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*milvuspb.ListImportTasksRequest), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_ListImportTasks_Call) Return(_a0 *milvuspb.ListImportTasksResponse, _a1 error) *MockRootCoordClient_ListImportTasks_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_ListImportTasks_Call) RunAndReturn(run func(context.Context, *milvuspb.ListImportTasksRequest, ...grpc.CallOption) (*milvuspb.ListImportTasksResponse, error)) *MockRootCoordClient_ListImportTasks_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// ListPolicy provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest, opts ...grpc.CallOption) (*internalpb.ListPolicyResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -2493,9 +2283,9 @@ type MockRootCoordClient_ListPolicy_Call struct {
|
|||
}
|
||||
|
||||
// ListPolicy is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ListPolicyRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ListPolicyRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ListPolicy(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ListPolicy_Call {
|
||||
return &MockRootCoordClient_ListPolicy_Call{Call: _e.mock.On("ListPolicy",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2563,9 +2353,9 @@ type MockRootCoordClient_OperatePrivilege_Call struct {
|
|||
}
|
||||
|
||||
// OperatePrivilege is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.OperatePrivilegeRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.OperatePrivilegeRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) OperatePrivilege(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_OperatePrivilege_Call {
|
||||
return &MockRootCoordClient_OperatePrivilege_Call{Call: _e.mock.On("OperatePrivilege",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2633,9 +2423,9 @@ type MockRootCoordClient_OperateUserRole_Call struct {
|
|||
}
|
||||
|
||||
// OperateUserRole is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.OperateUserRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.OperateUserRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) OperateUserRole(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_OperateUserRole_Call {
|
||||
return &MockRootCoordClient_OperateUserRole_Call{Call: _e.mock.On("OperateUserRole",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2703,9 +2493,9 @@ type MockRootCoordClient_RenameCollection_Call struct {
|
|||
}
|
||||
|
||||
// RenameCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.RenameCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.RenameCollectionRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) RenameCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_RenameCollection_Call {
|
||||
return &MockRootCoordClient_RenameCollection_Call{Call: _e.mock.On("RenameCollection",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2734,76 +2524,6 @@ func (_c *MockRootCoordClient_RenameCollection_Call) RunAndReturn(run func(conte
|
|||
return _c
|
||||
}
|
||||
|
||||
// ReportImport provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) ReportImport(ctx context.Context, in *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
for _i := range opts {
|
||||
_va[_i] = opts[_i]
|
||||
}
|
||||
var _ca []interface{}
|
||||
_ca = append(_ca, ctx, in)
|
||||
_ca = append(_ca, _va...)
|
||||
ret := _m.Called(_ca...)
|
||||
|
||||
var r0 *commonpb.Status
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ImportResult, ...grpc.CallOption) (*commonpb.Status, error)); ok {
|
||||
return rf(ctx, in, opts...)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ImportResult, ...grpc.CallOption) *commonpb.Status); ok {
|
||||
r0 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*commonpb.Status)
|
||||
}
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, *rootcoordpb.ImportResult, ...grpc.CallOption) error); ok {
|
||||
r1 = rf(ctx, in, opts...)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockRootCoordClient_ReportImport_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ReportImport'
|
||||
type MockRootCoordClient_ReportImport_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// ReportImport is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *rootcoordpb.ImportResult
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ReportImport(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ReportImport_Call {
|
||||
return &MockRootCoordClient_ReportImport_Call{Call: _e.mock.On("ReportImport",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_ReportImport_Call) Run(run func(ctx context.Context, in *rootcoordpb.ImportResult, opts ...grpc.CallOption)) *MockRootCoordClient_ReportImport_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
variadicArgs := make([]grpc.CallOption, len(args)-2)
|
||||
for i, a := range args[2:] {
|
||||
if a != nil {
|
||||
variadicArgs[i] = a.(grpc.CallOption)
|
||||
}
|
||||
}
|
||||
run(args[0].(context.Context), args[1].(*rootcoordpb.ImportResult), variadicArgs...)
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_ReportImport_Call) Return(_a0 *commonpb.Status, _a1 error) *MockRootCoordClient_ReportImport_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockRootCoordClient_ReportImport_Call) RunAndReturn(run func(context.Context, *rootcoordpb.ImportResult, ...grpc.CallOption) (*commonpb.Status, error)) *MockRootCoordClient_ReportImport_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SelectGrant provides a mock function with given fields: ctx, in, opts
|
||||
func (_m *MockRootCoordClient) SelectGrant(ctx context.Context, in *milvuspb.SelectGrantRequest, opts ...grpc.CallOption) (*milvuspb.SelectGrantResponse, error) {
|
||||
_va := make([]interface{}, len(opts))
|
||||
|
@ -2843,9 +2563,9 @@ type MockRootCoordClient_SelectGrant_Call struct {
|
|||
}
|
||||
|
||||
// SelectGrant is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.SelectGrantRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.SelectGrantRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) SelectGrant(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_SelectGrant_Call {
|
||||
return &MockRootCoordClient_SelectGrant_Call{Call: _e.mock.On("SelectGrant",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2913,9 +2633,9 @@ type MockRootCoordClient_SelectRole_Call struct {
|
|||
}
|
||||
|
||||
// SelectRole is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.SelectRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.SelectRoleRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) SelectRole(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_SelectRole_Call {
|
||||
return &MockRootCoordClient_SelectRole_Call{Call: _e.mock.On("SelectRole",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -2983,9 +2703,9 @@ type MockRootCoordClient_SelectUser_Call struct {
|
|||
}
|
||||
|
||||
// SelectUser is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.SelectUserRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.SelectUserRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) SelectUser(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_SelectUser_Call {
|
||||
return &MockRootCoordClient_SelectUser_Call{Call: _e.mock.On("SelectUser",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3053,9 +2773,9 @@ type MockRootCoordClient_ShowCollections_Call struct {
|
|||
}
|
||||
|
||||
// ShowCollections is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowCollectionsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowCollectionsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ShowCollections(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ShowCollections_Call {
|
||||
return &MockRootCoordClient_ShowCollections_Call{Call: _e.mock.On("ShowCollections",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3123,9 +2843,9 @@ type MockRootCoordClient_ShowConfigurations_Call struct {
|
|||
}
|
||||
|
||||
// ShowConfigurations is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ShowConfigurationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ShowConfigurationsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ShowConfigurations_Call {
|
||||
return &MockRootCoordClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3193,9 +2913,9 @@ type MockRootCoordClient_ShowPartitions_Call struct {
|
|||
}
|
||||
|
||||
// ShowPartitions is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowPartitionsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowPartitionsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ShowPartitions(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ShowPartitions_Call {
|
||||
return &MockRootCoordClient_ShowPartitions_Call{Call: _e.mock.On("ShowPartitions",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3263,9 +2983,9 @@ type MockRootCoordClient_ShowPartitionsInternal_Call struct {
|
|||
}
|
||||
|
||||
// ShowPartitionsInternal is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowPartitionsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowPartitionsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ShowPartitionsInternal(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ShowPartitionsInternal_Call {
|
||||
return &MockRootCoordClient_ShowPartitionsInternal_Call{Call: _e.mock.On("ShowPartitionsInternal",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3333,9 +3053,9 @@ type MockRootCoordClient_ShowSegments_Call struct {
|
|||
}
|
||||
|
||||
// ShowSegments is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *milvuspb.ShowSegmentsRequest
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) ShowSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ShowSegments_Call {
|
||||
return &MockRootCoordClient_ShowSegments_Call{Call: _e.mock.On("ShowSegments",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3403,9 +3123,9 @@ type MockRootCoordClient_UpdateChannelTimeTick_Call struct {
|
|||
}
|
||||
|
||||
// UpdateChannelTimeTick is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ChannelTimeTickMsg
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.ChannelTimeTickMsg
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) UpdateChannelTimeTick(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_UpdateChannelTimeTick_Call {
|
||||
return &MockRootCoordClient_UpdateChannelTimeTick_Call{Call: _e.mock.On("UpdateChannelTimeTick",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
@ -3473,9 +3193,9 @@ type MockRootCoordClient_UpdateCredential_Call struct {
|
|||
}
|
||||
|
||||
// UpdateCredential is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.CredentialInfo
|
||||
// - opts ...grpc.CallOption
|
||||
// - ctx context.Context
|
||||
// - in *internalpb.CredentialInfo
|
||||
// - opts ...grpc.CallOption
|
||||
func (_e *MockRootCoordClient_Expecter) UpdateCredential(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_UpdateCredential_Call {
|
||||
return &MockRootCoordClient_UpdateCredential_Call{Call: _e.mock.On("UpdateCredential",
|
||||
append([]interface{}{ctx, in}, opts...)...)}
|
||||
|
|
|
@ -65,13 +65,9 @@ service DataCoord {
|
|||
rpc DropVirtualChannel(DropVirtualChannelRequest) returns (DropVirtualChannelResponse) {}
|
||||
|
||||
rpc SetSegmentState(SetSegmentStateRequest) returns (SetSegmentStateResponse) {}
|
||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
||||
rpc Import(ImportTaskRequest) returns (ImportTaskResponse) {}
|
||||
rpc UpdateSegmentStatistics(UpdateSegmentStatisticsRequest) returns (common.Status) {}
|
||||
rpc UpdateChannelCheckpoint(UpdateChannelCheckpointRequest) returns (common.Status) {}
|
||||
|
||||
rpc SaveImportSegment(SaveImportSegmentRequest) returns(common.Status) {}
|
||||
rpc UnsetIsImportingState(UnsetIsImportingStateRequest) returns(common.Status) {}
|
||||
rpc MarkSegmentsDropped(MarkSegmentsDroppedRequest) returns(common.Status) {}
|
||||
|
||||
rpc BroadcastAlteredCollection(AlterCollectionRequest) returns (common.Status) {}
|
||||
|
@ -118,14 +114,9 @@ service DataNode {
|
|||
rpc GetCompactionState(CompactionStateRequest) returns (CompactionStateResponse) {}
|
||||
rpc SyncSegments(SyncSegmentsRequest) returns (common.Status) {}
|
||||
|
||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
||||
rpc Import(ImportTaskRequest) returns(common.Status) {}
|
||||
|
||||
// Deprecated
|
||||
rpc ResendSegmentStats(ResendSegmentStatsRequest) returns(ResendSegmentStatsResponse) {}
|
||||
|
||||
rpc AddImportSegment(AddImportSegmentRequest) returns(AddImportSegmentResponse) {}
|
||||
|
||||
rpc FlushChannels(FlushChannelsRequest) returns(common.Status) {}
|
||||
rpc NotifyChannelOperation(ChannelOperationsRequest) returns(common.Status) {}
|
||||
rpc CheckChannelOperationProgress(ChannelWatchInfo) returns(ChannelOperationProgressResponse) {}
|
||||
|
@ -143,7 +134,7 @@ message FlushRequest {
|
|||
int64 dbID = 2;
|
||||
repeated int64 segmentIDs = 3;
|
||||
int64 collectionID = 4;
|
||||
bool isImport = 5;
|
||||
bool isImport = 5; // deprecated
|
||||
}
|
||||
|
||||
message FlushResponse {
|
||||
|
@ -167,8 +158,8 @@ message SegmentIDRequest {
|
|||
string channel_name = 2;
|
||||
int64 collectionID = 3;
|
||||
int64 partitionID = 4;
|
||||
bool isImport = 5; // Indicate whether this request comes from a bulk insert task.
|
||||
int64 importTaskID = 6; // Needed for segment lock.
|
||||
bool isImport = 5; // deprecated
|
||||
int64 importTaskID = 6; // deprecated
|
||||
SegmentLevel level = 7;
|
||||
}
|
||||
|
||||
|
@ -347,7 +338,7 @@ message SaveBinlogPathsRequest {
|
|||
repeated FieldBinlog field2StatslogPaths = 8;
|
||||
repeated FieldBinlog deltalogs = 9;
|
||||
bool dropped = 10;
|
||||
bool importing = 11;
|
||||
bool importing = 11; // deprecated
|
||||
string channel = 12; // report channel name for verification
|
||||
SegmentLevel seg_level =13;
|
||||
int64 partitionID =14; // report partitionID for create L0 segment
|
||||
|
@ -604,56 +595,6 @@ message DropVirtualChannelResponse {
|
|||
common.Status status = 1;
|
||||
}
|
||||
|
||||
message ImportTask {
|
||||
common.Status status = 1;
|
||||
int64 collection_id = 2; // target collection ID
|
||||
int64 partition_id = 3; // target partition ID
|
||||
repeated string channel_names = 4; // target channel names of the collection.
|
||||
bool row_based = 5; // the file is row-based or column-based
|
||||
int64 task_id = 6; // id of the task
|
||||
repeated string files = 7; // file paths to be imported
|
||||
repeated common.KeyValuePair infos = 8; // extra information about the task, bucket, etc.
|
||||
string database_name = 16; // Database name
|
||||
}
|
||||
|
||||
message ImportTaskState {
|
||||
common.ImportState stateCode = 1; // Import state code.
|
||||
repeated int64 segments = 2; // Ids of segments created in import task.
|
||||
repeated int64 row_ids = 3; // Row IDs for the newly inserted rows.
|
||||
int64 row_count = 4; // # of rows added in the import task.
|
||||
string error_message = 5; // Error message for the failed task.
|
||||
}
|
||||
|
||||
message ImportTaskInfo {
|
||||
int64 id = 1; // Task ID.
|
||||
int64 request_id = 2 [deprecated = true]; // Request ID of the import task.
|
||||
int64 datanode_id = 3; // ID of DataNode that processes the task.
|
||||
int64 collection_id = 4; // Collection ID for the import task.
|
||||
int64 partition_id = 5; // Partition ID for the import task.
|
||||
repeated string channel_names = 6; // Names of channels for the collection.
|
||||
string bucket = 7; // Bucket for the import task.
|
||||
bool row_based = 8; // Boolean indicating whether import files are row-based or column-based.
|
||||
repeated string files = 9; // A list of files to import.
|
||||
int64 create_ts = 10; // Timestamp when the import task is created.
|
||||
ImportTaskState state = 11; // State of the import task.
|
||||
string collection_name = 12; // Collection name for the import task.
|
||||
string partition_name = 13; // Partition name for the import task.
|
||||
repeated common.KeyValuePair infos = 14; // extra information about the task, bucket, etc.
|
||||
int64 start_ts = 15; // Timestamp when the import task is sent to datanode to execute.
|
||||
string database_name = 16; // Database name
|
||||
}
|
||||
|
||||
message ImportTaskResponse {
|
||||
common.Status status = 1;
|
||||
int64 datanode_id = 2; // which datanode takes this task
|
||||
}
|
||||
|
||||
message ImportTaskRequest {
|
||||
common.MsgBase base = 1;
|
||||
ImportTask import_task = 2; // Target import task.
|
||||
repeated int64 working_nodes = 3; // DataNodes that are currently working.
|
||||
}
|
||||
|
||||
message UpdateSegmentStatisticsRequest {
|
||||
common.MsgBase base = 1;
|
||||
repeated common.SegmentStats stats = 2;
|
||||
|
@ -675,37 +616,6 @@ message ResendSegmentStatsResponse {
|
|||
repeated int64 seg_resent = 2;
|
||||
}
|
||||
|
||||
message AddImportSegmentRequest {
|
||||
common.MsgBase base = 1;
|
||||
int64 segment_id = 2;
|
||||
string channel_name = 3;
|
||||
int64 collection_id = 4;
|
||||
int64 partition_id = 5;
|
||||
int64 row_num = 6;
|
||||
repeated FieldBinlog stats_log = 7;
|
||||
}
|
||||
|
||||
message AddImportSegmentResponse {
|
||||
common.Status status = 1;
|
||||
bytes channel_pos = 2; // deprecated
|
||||
}
|
||||
|
||||
message SaveImportSegmentRequest {
|
||||
common.MsgBase base = 1;
|
||||
int64 segment_id = 2;
|
||||
string channel_name = 3;
|
||||
int64 collection_id = 4;
|
||||
int64 partition_id = 5;
|
||||
int64 row_num = 6;
|
||||
SaveBinlogPathsRequest save_binlog_path_req = 7;
|
||||
bytes dml_position_id = 8;
|
||||
}
|
||||
|
||||
message UnsetIsImportingStateRequest {
|
||||
common.MsgBase base = 1;
|
||||
repeated int64 segment_ids = 2; // IDs of segments whose `isImport` states need to be unset.
|
||||
}
|
||||
|
||||
message MarkSegmentsDroppedRequest {
|
||||
common.MsgBase base = 1;
|
||||
repeated int64 segment_ids = 2; // IDs of segments that needs to be marked as `dropped`.
|
||||
|
|
|
@ -7,7 +7,6 @@ import "common.proto";
|
|||
import "milvus.proto";
|
||||
import "internal.proto";
|
||||
import "proxy.proto";
|
||||
//import "data_coord.proto";
|
||||
import "etcd_meta.proto";
|
||||
|
||||
service RootCoord {
|
||||
|
@ -116,12 +115,6 @@ service RootCoord {
|
|||
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
|
||||
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
|
||||
|
||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load
|
||||
rpc Import(milvus.ImportRequest) returns (milvus.ImportResponse) {}
|
||||
rpc GetImportState(milvus.GetImportStateRequest) returns (milvus.GetImportStateResponse) {}
|
||||
rpc ListImportTasks(milvus.ListImportTasksRequest) returns (milvus.ListImportTasksResponse) {}
|
||||
rpc ReportImport(ImportResult) returns (common.Status) {}
|
||||
|
||||
// https://wiki.lfaidata.foundation/display/MIL/MEP+27+--+Support+Basic+Authentication
|
||||
rpc CreateCredential(internal.CredentialInfo) returns (common.Status) {}
|
||||
rpc UpdateCredential(internal.CredentialInfo) returns (common.Status) {}
|
||||
|
@ -171,17 +164,6 @@ message AllocIDResponse {
|
|||
uint32 count = 3;
|
||||
}
|
||||
|
||||
message ImportResult {
|
||||
common.Status status = 1;
|
||||
int64 task_id = 2; // id of the task
|
||||
int64 datanode_id = 3; // id of the datanode which takes this task
|
||||
common.ImportState state = 4; // state of the task
|
||||
repeated int64 segments = 5; // id array of new sealed segments
|
||||
repeated int64 auto_ids = 6; // auto-generated ids for auto-id primary key
|
||||
int64 row_count = 7; // how many rows are imported by this task
|
||||
repeated common.KeyValuePair infos = 8; // more informations about the task, file path, failed reason, etc.
|
||||
}
|
||||
|
||||
// TODO: find a proper place for these segment-related messages.
|
||||
|
||||
message DescribeSegmentsRequest {
|
||||
|
|
|
@ -98,14 +98,6 @@ func (coord *DataCoordMock) Flush(ctx context.Context, req *datapb.FlushRequest,
|
|||
panic("implement me")
|
||||
}
|
||||
|
||||
func (coord *DataCoordMock) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (coord *DataCoordMock) UnsetIsImportingState(ctx context.Context, in *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func (coord *DataCoordMock) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
panic("implement me")
|
||||
}
|
||||
|
@ -247,10 +239,6 @@ func (coord *DataCoordMock) SetSegmentState(ctx context.Context, req *datapb.Set
|
|||
return &datapb.SetSegmentStateResponse{}, nil
|
||||
}
|
||||
|
||||
func (coord *DataCoordMock) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
|
||||
return &datapb.ImportTaskResponse{}, nil
|
||||
}
|
||||
|
||||
func (coord *DataCoordMock) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
|
|
@ -42,7 +42,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/proxy/connection"
|
||||
"github.com/milvus-io/milvus/internal/util/importutil"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
|
@ -4237,6 +4236,10 @@ func convertToV2GetImportRequest(req *milvuspb.GetImportStateRequest) *internalp
|
|||
}
|
||||
|
||||
func convertToV1GetImportResponse(rsp *internalpb.GetImportProgressResponse) *milvuspb.GetImportStateResponse {
|
||||
const (
|
||||
failedReason = "failed_reason"
|
||||
progressPercent = "progress_percent"
|
||||
)
|
||||
if rsp.GetStatus().GetCode() != 0 {
|
||||
return &milvuspb.GetImportStateResponse{
|
||||
Status: rsp.GetStatus(),
|
||||
|
@ -4257,11 +4260,11 @@ func convertToV1GetImportResponse(rsp *internalpb.GetImportProgressResponse) *mi
|
|||
}
|
||||
infos := make([]*commonpb.KeyValuePair, 0)
|
||||
infos = append(infos, &commonpb.KeyValuePair{
|
||||
Key: importutil.FailedReason,
|
||||
Key: failedReason,
|
||||
Value: rsp.GetReason(),
|
||||
})
|
||||
infos = append(infos, &commonpb.KeyValuePair{
|
||||
Key: importutil.ProgressPercent,
|
||||
Key: progressPercent,
|
||||
Value: strconv.FormatInt(rsp.GetProgress(), 10),
|
||||
})
|
||||
return &milvuspb.GetImportStateResponse{
|
||||
|
|
|
@ -1012,17 +1012,6 @@ func (coord *RootCoordMock) ListImportTasks(ctx context.Context, in *milvuspb.Li
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (coord *RootCoordMock) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
code := coord.state.Load().(commonpb.StateCode)
|
||||
if code != commonpb.StateCode_Healthy {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: fmt.Sprintf("state code = %s", commonpb.StateCode_name[int32(code)]),
|
||||
}, nil
|
||||
}
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
func NewRootCoordMock(opts ...RootCoordMockOption) *RootCoordMock {
|
||||
rc := &RootCoordMock{
|
||||
nodeID: typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
|
||||
|
|
|
@ -1426,7 +1426,6 @@ func (t *flushTask) Execute(ctx context.Context) error {
|
|||
commonpbutil.WithMsgType(commonpb.MsgType_Flush),
|
||||
),
|
||||
CollectionID: collID,
|
||||
IsImport: false,
|
||||
}
|
||||
resp, err := t.dataCoord.Flush(ctx, flushReq)
|
||||
if err != nil {
|
||||
|
|
|
@ -54,9 +54,6 @@ type Broker interface {
|
|||
|
||||
WatchChannels(ctx context.Context, info *watchInfo) error
|
||||
UnwatchChannels(ctx context.Context, info *watchInfo) error
|
||||
Flush(ctx context.Context, cID int64, segIDs []int64) error
|
||||
Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
|
||||
UnsetIsImportingState(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)
|
||||
GetSegmentStates(context.Context, *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error)
|
||||
GcConfirm(ctx context.Context, collectionID, partitionID UniqueID) bool
|
||||
|
||||
|
@ -188,35 +185,6 @@ func (b *ServerBroker) UnwatchChannels(ctx context.Context, info *watchInfo) err
|
|||
return nil
|
||||
}
|
||||
|
||||
func (b *ServerBroker) Flush(ctx context.Context, cID int64, segIDs []int64) error {
|
||||
resp, err := b.s.dataCoord.Flush(ctx, &datapb.FlushRequest{
|
||||
Base: commonpbutil.NewMsgBase(
|
||||
commonpbutil.WithMsgType(commonpb.MsgType_Flush),
|
||||
commonpbutil.WithSourceID(b.s.session.ServerID),
|
||||
),
|
||||
DbID: 0,
|
||||
SegmentIDs: segIDs,
|
||||
CollectionID: cID,
|
||||
IsImport: true,
|
||||
})
|
||||
if err != nil {
|
||||
return errors.New("failed to call flush to data coordinator: " + err.Error())
|
||||
}
|
||||
if resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
return merr.Error(resp.GetStatus())
|
||||
}
|
||||
log.Info("flush on collection succeed", zap.Int64("collectionID", cID))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *ServerBroker) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
return b.s.dataCoord.Import(ctx, req)
|
||||
}
|
||||
|
||||
func (b *ServerBroker) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
return b.s.dataCoord.UnsetIsImportingState(ctx, req)
|
||||
}
|
||||
|
||||
func (b *ServerBroker) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
|
||||
return b.s.dataCoord.GetSegmentStates(ctx, req)
|
||||
}
|
||||
|
|
|
@ -127,61 +127,6 @@ func TestServerBroker_UnwatchChannels(t *testing.T) {
|
|||
b.UnwatchChannels(ctx, &watchInfo{})
|
||||
}
|
||||
|
||||
func TestServerBroker_Flush(t *testing.T) {
|
||||
t.Run("failed to execute", func(t *testing.T) {
|
||||
c := newTestCore(withInvalidDataCoord())
|
||||
b := newServerBroker(c)
|
||||
ctx := context.Background()
|
||||
err := b.Flush(ctx, 1, []int64{1, 2})
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("non success error code on execute", func(t *testing.T) {
|
||||
c := newTestCore(withFailedDataCoord())
|
||||
b := newServerBroker(c)
|
||||
ctx := context.Background()
|
||||
err := b.Flush(ctx, 1, []int64{1, 2})
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("success", func(t *testing.T) {
|
||||
c := newTestCore(withValidDataCoord())
|
||||
b := newServerBroker(c)
|
||||
ctx := context.Background()
|
||||
err := b.Flush(ctx, 1, []int64{1, 2})
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func TestServerBroker_Import(t *testing.T) {
|
||||
t.Run("failed to execute", func(t *testing.T) {
|
||||
c := newTestCore(withInvalidDataCoord())
|
||||
b := newServerBroker(c)
|
||||
ctx := context.Background()
|
||||
resp, err := b.Import(ctx, &datapb.ImportTaskRequest{})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("non success error code on execute", func(t *testing.T) {
|
||||
c := newTestCore(withFailedDataCoord())
|
||||
b := newServerBroker(c)
|
||||
ctx := context.Background()
|
||||
resp, err := b.Import(ctx, &datapb.ImportTaskRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetStatus().GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("success", func(t *testing.T) {
|
||||
c := newTestCore(withValidDataCoord())
|
||||
b := newServerBroker(c)
|
||||
ctx := context.Background()
|
||||
resp, err := b.Import(ctx, &datapb.ImportTaskRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
})
|
||||
}
|
||||
|
||||
func TestServerBroker_DropCollectionIndex(t *testing.T) {
|
||||
t.Run("failed to execute", func(t *testing.T) {
|
||||
c := newTestCore(withInvalidDataCoord())
|
||||
|
|
|
@ -1,142 +0,0 @@
|
|||
// 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 rootcoord
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type GetCollectionNameFunc func(dbName string, collID, partitionID UniqueID) (string, string, error)
|
||||
|
||||
type IDAllocator func(count uint32) (UniqueID, UniqueID, error)
|
||||
|
||||
type ImportFunc func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
|
||||
|
||||
type GetSegmentStatesFunc func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error)
|
||||
|
||||
type DescribeIndexFunc func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error)
|
||||
|
||||
type GetSegmentIndexStateFunc func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error)
|
||||
|
||||
type UnsetIsImportingStateFunc func(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)
|
||||
|
||||
type ImportFactory interface {
|
||||
NewGetCollectionNameFunc() GetCollectionNameFunc
|
||||
NewIDAllocator() IDAllocator
|
||||
NewImportFunc() ImportFunc
|
||||
NewGetSegmentStatesFunc() GetSegmentStatesFunc
|
||||
NewDescribeIndexFunc() DescribeIndexFunc
|
||||
NewGetSegmentIndexStateFunc() GetSegmentIndexStateFunc
|
||||
NewUnsetIsImportingStateFunc() UnsetIsImportingStateFunc
|
||||
}
|
||||
|
||||
type ImportFactoryImpl struct {
|
||||
c *Core
|
||||
}
|
||||
|
||||
func (f ImportFactoryImpl) NewGetCollectionNameFunc() GetCollectionNameFunc {
|
||||
return GetCollectionNameWithCore(f.c)
|
||||
}
|
||||
|
||||
func (f ImportFactoryImpl) NewIDAllocator() IDAllocator {
|
||||
return IDAllocatorWithCore(f.c)
|
||||
}
|
||||
|
||||
func (f ImportFactoryImpl) NewImportFunc() ImportFunc {
|
||||
return ImportFuncWithCore(f.c)
|
||||
}
|
||||
|
||||
func (f ImportFactoryImpl) NewGetSegmentStatesFunc() GetSegmentStatesFunc {
|
||||
return GetSegmentStatesWithCore(f.c)
|
||||
}
|
||||
|
||||
func (f ImportFactoryImpl) NewDescribeIndexFunc() DescribeIndexFunc {
|
||||
return DescribeIndexWithCore(f.c)
|
||||
}
|
||||
|
||||
func (f ImportFactoryImpl) NewGetSegmentIndexStateFunc() GetSegmentIndexStateFunc {
|
||||
return GetSegmentIndexStateWithCore(f.c)
|
||||
}
|
||||
|
||||
func (f ImportFactoryImpl) NewUnsetIsImportingStateFunc() UnsetIsImportingStateFunc {
|
||||
return UnsetIsImportingStateWithCore(f.c)
|
||||
}
|
||||
|
||||
func NewImportFactory(c *Core) ImportFactory {
|
||||
return &ImportFactoryImpl{c: c}
|
||||
}
|
||||
|
||||
func GetCollectionNameWithCore(c *Core) GetCollectionNameFunc {
|
||||
return func(dbName string, collID, partitionID UniqueID) (string, string, error) {
|
||||
colInfo, err := c.meta.GetCollectionByID(c.ctx, dbName, collID, typeutil.MaxTimestamp, false)
|
||||
if err != nil {
|
||||
log.Error("Core failed to get collection name by id", zap.Int64("ID", collID), zap.Error(err))
|
||||
return "", "", err
|
||||
}
|
||||
partName, err := c.meta.GetPartitionNameByID(collID, partitionID, 0)
|
||||
if err != nil {
|
||||
log.Error("Core failed to get partition name by id", zap.Int64("ID", partitionID), zap.Error(err))
|
||||
return colInfo.Name, "", err
|
||||
}
|
||||
|
||||
return colInfo.Name, partName, nil
|
||||
}
|
||||
}
|
||||
|
||||
func IDAllocatorWithCore(c *Core) IDAllocator {
|
||||
return func(count uint32) (UniqueID, UniqueID, error) {
|
||||
return c.idAllocator.Alloc(count)
|
||||
}
|
||||
}
|
||||
|
||||
func ImportFuncWithCore(c *Core) ImportFunc {
|
||||
return func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
return c.broker.Import(ctx, req)
|
||||
}
|
||||
}
|
||||
|
||||
func GetSegmentStatesWithCore(c *Core) GetSegmentStatesFunc {
|
||||
return func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
|
||||
return c.broker.GetSegmentStates(ctx, req)
|
||||
}
|
||||
}
|
||||
|
||||
func DescribeIndexWithCore(c *Core) DescribeIndexFunc {
|
||||
return func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error) {
|
||||
return c.broker.DescribeIndex(ctx, colID)
|
||||
}
|
||||
}
|
||||
|
||||
func GetSegmentIndexStateWithCore(c *Core) GetSegmentIndexStateFunc {
|
||||
return func(ctx context.Context, collID UniqueID, indexName string, segIDs []UniqueID) ([]*indexpb.SegmentIndexState, error) {
|
||||
return c.broker.GetSegmentIndexState(ctx, collID, indexName, segIDs)
|
||||
}
|
||||
}
|
||||
|
||||
func UnsetIsImportingStateWithCore(c *Core) UnsetIsImportingStateFunc {
|
||||
return func(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
return c.broker.UnsetIsImportingState(ctx, req)
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -74,10 +74,6 @@ type IMetaTable interface {
|
|||
IsAlias(db, name string) bool
|
||||
ListAliasesByID(collID UniqueID) []string
|
||||
|
||||
// TODO: better to accept ctx.
|
||||
GetPartitionNameByID(collID UniqueID, partitionID UniqueID, ts Timestamp) (string, error) // serve for bulk insert.
|
||||
GetPartitionByName(collID UniqueID, partitionName string, ts Timestamp) (UniqueID, error) // serve for bulk insert.
|
||||
|
||||
// TODO: better to accept ctx.
|
||||
AddCredential(credInfo *internalpb.CredentialInfo) error
|
||||
GetCredential(username string) (*internalpb.CredentialInfo, error)
|
||||
|
@ -1147,74 +1143,6 @@ func (mt *MetaTable) ListAliasesByID(collID UniqueID) []string {
|
|||
return mt.listAliasesByID(collID)
|
||||
}
|
||||
|
||||
// GetPartitionNameByID serve for bulk insert.
|
||||
func (mt *MetaTable) GetPartitionNameByID(collID UniqueID, partitionID UniqueID, ts Timestamp) (string, error) {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
|
||||
coll, ok := mt.collID2Meta[collID]
|
||||
if ok && coll.Available() && coll.CreateTime <= ts {
|
||||
// cache hit.
|
||||
for _, partition := range coll.Partitions {
|
||||
if partition.Available() && partition.PartitionID == partitionID && partition.PartitionCreatedTimestamp <= ts {
|
||||
// cache hit.
|
||||
return partition.PartitionName, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
// cache miss, get from catalog anyway.
|
||||
coll, err := mt.catalog.GetCollectionByID(mt.ctx, coll.DBID, ts, collID)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
if !coll.Available() {
|
||||
return "", fmt.Errorf("collection not exist: %d", collID)
|
||||
}
|
||||
for _, partition := range coll.Partitions {
|
||||
// no need to check time travel logic again, since catalog already did.
|
||||
if partition.Available() && partition.PartitionID == partitionID {
|
||||
return partition.PartitionName, nil
|
||||
}
|
||||
}
|
||||
return "", merr.WrapErrPartitionNotFound(partitionID)
|
||||
}
|
||||
|
||||
// GetPartitionByName serve for bulk insert.
|
||||
func (mt *MetaTable) GetPartitionByName(collID UniqueID, partitionName string, ts Timestamp) (UniqueID, error) {
|
||||
mt.ddLock.RLock()
|
||||
defer mt.ddLock.RUnlock()
|
||||
|
||||
coll, ok := mt.collID2Meta[collID]
|
||||
if ok && coll.Available() && coll.CreateTime <= ts {
|
||||
// cache hit.
|
||||
for _, partition := range coll.Partitions {
|
||||
if partition.Available() && partition.PartitionName == partitionName && partition.PartitionCreatedTimestamp <= ts {
|
||||
// cache hit.
|
||||
return partition.PartitionID, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
// cache miss, get from catalog anyway.
|
||||
coll, err := mt.catalog.GetCollectionByID(mt.ctx, coll.DBID, ts, collID)
|
||||
if err != nil {
|
||||
return common.InvalidPartitionID, err
|
||||
}
|
||||
if !coll.Available() {
|
||||
return common.InvalidPartitionID, merr.WrapErrCollectionNotFoundWithDB(coll.DBID, collID)
|
||||
}
|
||||
for _, partition := range coll.Partitions {
|
||||
// no need to check time travel logic again, since catalog already did.
|
||||
if partition.Available() && partition.PartitionName == partitionName {
|
||||
return partition.PartitionID, nil
|
||||
}
|
||||
}
|
||||
|
||||
log.Error("partition ID not found for partition name", zap.String("partitionName", partitionName),
|
||||
zap.Int64("collectionID", collID), zap.String("collectionName", coll.Name))
|
||||
return common.InvalidPartitionID, fmt.Errorf("partition ID not found for partition name '%s' in collection '%s'",
|
||||
partitionName, coll.Name)
|
||||
}
|
||||
|
||||
// AddCredential add credential
|
||||
func (mt *MetaTable) AddCredential(credInfo *internalpb.CredentialInfo) error {
|
||||
if credInfo.Username == "" {
|
||||
|
|
|
@ -254,8 +254,6 @@ type mockDataCoord struct {
|
|||
GetComponentStatesFunc func(ctx context.Context) (*milvuspb.ComponentStates, error)
|
||||
WatchChannelsFunc func(ctx context.Context, req *datapb.WatchChannelsRequest) (*datapb.WatchChannelsResponse, error)
|
||||
FlushFunc func(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error)
|
||||
ImportFunc func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
|
||||
UnsetIsImportingStateFunc func(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error)
|
||||
broadCastAlteredCollectionFunc func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error)
|
||||
GetSegmentIndexStateFunc func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error)
|
||||
DropIndexFunc func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error)
|
||||
|
@ -277,14 +275,6 @@ func (m *mockDataCoord) Flush(ctx context.Context, req *datapb.FlushRequest, opt
|
|||
return m.FlushFunc(ctx, req)
|
||||
}
|
||||
|
||||
func (m *mockDataCoord) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
|
||||
return m.ImportFunc(ctx, req)
|
||||
}
|
||||
|
||||
func (m *mockDataCoord) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return m.UnsetIsImportingStateFunc(ctx, req)
|
||||
}
|
||||
|
||||
func (m *mockDataCoord) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return m.broadCastAlteredCollectionFunc(ctx, req)
|
||||
}
|
||||
|
@ -708,12 +698,6 @@ func withInvalidDataCoord() Opt {
|
|||
dc.FlushFunc = func(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
|
||||
return nil, errors.New("error mock Flush")
|
||||
}
|
||||
dc.ImportFunc = func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
return nil, errors.New("error mock Import")
|
||||
}
|
||||
dc.UnsetIsImportingStateFunc = func(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
return nil, errors.New("error mock UnsetIsImportingState")
|
||||
}
|
||||
dc.broadCastAlteredCollectionFunc = func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
|
||||
return nil, errors.New("error mock broadCastAlteredCollection")
|
||||
}
|
||||
|
@ -745,17 +729,6 @@ func withFailedDataCoord() Opt {
|
|||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
dc.ImportFunc = func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
return &datapb.ImportTaskResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
dc.UnsetIsImportingStateFunc = func(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: "mock UnsetIsImportingState error",
|
||||
}, nil
|
||||
}
|
||||
dc.broadCastAlteredCollectionFunc = func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
@ -788,14 +761,6 @@ func withValidDataCoord() Opt {
|
|||
Status: merr.Success(),
|
||||
}, nil
|
||||
}
|
||||
dc.ImportFunc = func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
return &datapb.ImportTaskResponse{
|
||||
Status: merr.Success(),
|
||||
}, nil
|
||||
}
|
||||
dc.UnsetIsImportingStateFunc = func(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
dc.broadCastAlteredCollectionFunc = func(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
@ -924,7 +889,6 @@ type mockBroker struct {
|
|||
AddSegRefLockFunc func(ctx context.Context, taskID int64, segIDs []int64) error
|
||||
ReleaseSegRefLockFunc func(ctx context.Context, taskID int64, segIDs []int64) error
|
||||
FlushFunc func(ctx context.Context, cID int64, segIDs []int64) error
|
||||
ImportFunc func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error)
|
||||
|
||||
DropCollectionIndexFunc func(ctx context.Context, collID UniqueID, partIDs []UniqueID) error
|
||||
DescribeIndexFunc func(ctx context.Context, colID UniqueID) (*indexpb.DescribeIndexResponse, error)
|
||||
|
|
|
@ -52,8 +52,8 @@ type GarbageCollector_GcCollectionData_Call struct {
|
|||
}
|
||||
|
||||
// GcCollectionData is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - coll *model.Collection
|
||||
// - ctx context.Context
|
||||
// - coll *model.Collection
|
||||
func (_e *GarbageCollector_Expecter) GcCollectionData(ctx interface{}, coll interface{}) *GarbageCollector_GcCollectionData_Call {
|
||||
return &GarbageCollector_GcCollectionData_Call{Call: _e.mock.On("GcCollectionData", ctx, coll)}
|
||||
}
|
||||
|
@ -105,9 +105,9 @@ type GarbageCollector_GcPartitionData_Call struct {
|
|||
}
|
||||
|
||||
// GcPartitionData is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - pChannels []string
|
||||
// - partition *model.Partition
|
||||
// - ctx context.Context
|
||||
// - pChannels []string
|
||||
// - partition *model.Partition
|
||||
func (_e *GarbageCollector_Expecter) GcPartitionData(ctx interface{}, pChannels interface{}, partition interface{}) *GarbageCollector_GcPartitionData_Call {
|
||||
return &GarbageCollector_GcPartitionData_Call{Call: _e.mock.On("GcPartitionData", ctx, pChannels, partition)}
|
||||
}
|
||||
|
@ -140,8 +140,8 @@ type GarbageCollector_ReDropCollection_Call struct {
|
|||
}
|
||||
|
||||
// ReDropCollection is a helper method to define mock.On call
|
||||
// - collMeta *model.Collection
|
||||
// - ts uint64
|
||||
// - collMeta *model.Collection
|
||||
// - ts uint64
|
||||
func (_e *GarbageCollector_Expecter) ReDropCollection(collMeta interface{}, ts interface{}) *GarbageCollector_ReDropCollection_Call {
|
||||
return &GarbageCollector_ReDropCollection_Call{Call: _e.mock.On("ReDropCollection", collMeta, ts)}
|
||||
}
|
||||
|
@ -174,10 +174,10 @@ type GarbageCollector_ReDropPartition_Call struct {
|
|||
}
|
||||
|
||||
// ReDropPartition is a helper method to define mock.On call
|
||||
// - dbID int64
|
||||
// - pChannels []string
|
||||
// - partition *model.Partition
|
||||
// - ts uint64
|
||||
// - dbID int64
|
||||
// - pChannels []string
|
||||
// - partition *model.Partition
|
||||
// - ts uint64
|
||||
func (_e *GarbageCollector_Expecter) ReDropPartition(dbID interface{}, pChannels interface{}, partition interface{}, ts interface{}) *GarbageCollector_ReDropPartition_Call {
|
||||
return &GarbageCollector_ReDropPartition_Call{Call: _e.mock.On("ReDropPartition", dbID, pChannels, partition, ts)}
|
||||
}
|
||||
|
@ -210,7 +210,7 @@ type GarbageCollector_RemoveCreatingCollection_Call struct {
|
|||
}
|
||||
|
||||
// RemoveCreatingCollection is a helper method to define mock.On call
|
||||
// - collMeta *model.Collection
|
||||
// - collMeta *model.Collection
|
||||
func (_e *GarbageCollector_Expecter) RemoveCreatingCollection(collMeta interface{}) *GarbageCollector_RemoveCreatingCollection_Call {
|
||||
return &GarbageCollector_RemoveCreatingCollection_Call{Call: _e.mock.On("RemoveCreatingCollection", collMeta)}
|
||||
}
|
||||
|
@ -243,9 +243,9 @@ type GarbageCollector_RemoveCreatingPartition_Call struct {
|
|||
}
|
||||
|
||||
// RemoveCreatingPartition is a helper method to define mock.On call
|
||||
// - dbID int64
|
||||
// - partition *model.Partition
|
||||
// - ts uint64
|
||||
// - dbID int64
|
||||
// - partition *model.Partition
|
||||
// - ts uint64
|
||||
func (_e *GarbageCollector_Expecter) RemoveCreatingPartition(dbID interface{}, partition interface{}, ts interface{}) *GarbageCollector_RemoveCreatingPartition_Call {
|
||||
return &GarbageCollector_RemoveCreatingPartition_Call{Call: _e.mock.On("RemoveCreatingPartition", dbID, partition, ts)}
|
||||
}
|
||||
|
|
|
@ -48,8 +48,8 @@ type IMetaTable_AddCollection_Call struct {
|
|||
}
|
||||
|
||||
// AddCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - coll *model.Collection
|
||||
// - ctx context.Context
|
||||
// - coll *model.Collection
|
||||
func (_e *IMetaTable_Expecter) AddCollection(ctx interface{}, coll interface{}) *IMetaTable_AddCollection_Call {
|
||||
return &IMetaTable_AddCollection_Call{Call: _e.mock.On("AddCollection", ctx, coll)}
|
||||
}
|
||||
|
@ -91,7 +91,7 @@ type IMetaTable_AddCredential_Call struct {
|
|||
}
|
||||
|
||||
// AddCredential is a helper method to define mock.On call
|
||||
// - credInfo *internalpb.CredentialInfo
|
||||
// - credInfo *internalpb.CredentialInfo
|
||||
func (_e *IMetaTable_Expecter) AddCredential(credInfo interface{}) *IMetaTable_AddCredential_Call {
|
||||
return &IMetaTable_AddCredential_Call{Call: _e.mock.On("AddCredential", credInfo)}
|
||||
}
|
||||
|
@ -133,8 +133,8 @@ type IMetaTable_AddPartition_Call struct {
|
|||
}
|
||||
|
||||
// AddPartition is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - partition *model.Partition
|
||||
// - ctx context.Context
|
||||
// - partition *model.Partition
|
||||
func (_e *IMetaTable_Expecter) AddPartition(ctx interface{}, partition interface{}) *IMetaTable_AddPartition_Call {
|
||||
return &IMetaTable_AddPartition_Call{Call: _e.mock.On("AddPartition", ctx, partition)}
|
||||
}
|
||||
|
@ -176,11 +176,11 @@ type IMetaTable_AlterAlias_Call struct {
|
|||
}
|
||||
|
||||
// AlterAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) AlterAlias(ctx interface{}, dbName interface{}, alias interface{}, collectionName interface{}, ts interface{}) *IMetaTable_AlterAlias_Call {
|
||||
return &IMetaTable_AlterAlias_Call{Call: _e.mock.On("AlterAlias", ctx, dbName, alias, collectionName, ts)}
|
||||
}
|
||||
|
@ -222,10 +222,10 @@ type IMetaTable_AlterCollection_Call struct {
|
|||
}
|
||||
|
||||
// AlterCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - oldColl *model.Collection
|
||||
// - newColl *model.Collection
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - oldColl *model.Collection
|
||||
// - newColl *model.Collection
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) AlterCollection(ctx interface{}, oldColl interface{}, newColl interface{}, ts interface{}) *IMetaTable_AlterCollection_Call {
|
||||
return &IMetaTable_AlterCollection_Call{Call: _e.mock.On("AlterCollection", ctx, oldColl, newColl, ts)}
|
||||
}
|
||||
|
@ -267,7 +267,7 @@ type IMetaTable_AlterCredential_Call struct {
|
|||
}
|
||||
|
||||
// AlterCredential is a helper method to define mock.On call
|
||||
// - credInfo *internalpb.CredentialInfo
|
||||
// - credInfo *internalpb.CredentialInfo
|
||||
func (_e *IMetaTable_Expecter) AlterCredential(credInfo interface{}) *IMetaTable_AlterCredential_Call {
|
||||
return &IMetaTable_AlterCredential_Call{Call: _e.mock.On("AlterCredential", credInfo)}
|
||||
}
|
||||
|
@ -309,10 +309,10 @@ type IMetaTable_ChangeCollectionState_Call struct {
|
|||
}
|
||||
|
||||
// ChangeCollectionState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - collectionID int64
|
||||
// - state etcdpb.CollectionState
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - collectionID int64
|
||||
// - state etcdpb.CollectionState
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) ChangeCollectionState(ctx interface{}, collectionID interface{}, state interface{}, ts interface{}) *IMetaTable_ChangeCollectionState_Call {
|
||||
return &IMetaTable_ChangeCollectionState_Call{Call: _e.mock.On("ChangeCollectionState", ctx, collectionID, state, ts)}
|
||||
}
|
||||
|
@ -354,11 +354,11 @@ type IMetaTable_ChangePartitionState_Call struct {
|
|||
}
|
||||
|
||||
// ChangePartitionState is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - collectionID int64
|
||||
// - partitionID int64
|
||||
// - state etcdpb.PartitionState
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - collectionID int64
|
||||
// - partitionID int64
|
||||
// - state etcdpb.PartitionState
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) ChangePartitionState(ctx interface{}, collectionID interface{}, partitionID interface{}, state interface{}, ts interface{}) *IMetaTable_ChangePartitionState_Call {
|
||||
return &IMetaTable_ChangePartitionState_Call{Call: _e.mock.On("ChangePartitionState", ctx, collectionID, partitionID, state, ts)}
|
||||
}
|
||||
|
@ -400,11 +400,11 @@ type IMetaTable_CreateAlias_Call struct {
|
|||
}
|
||||
|
||||
// CreateAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) CreateAlias(ctx interface{}, dbName interface{}, alias interface{}, collectionName interface{}, ts interface{}) *IMetaTable_CreateAlias_Call {
|
||||
return &IMetaTable_CreateAlias_Call{Call: _e.mock.On("CreateAlias", ctx, dbName, alias, collectionName, ts)}
|
||||
}
|
||||
|
@ -446,9 +446,9 @@ type IMetaTable_CreateDatabase_Call struct {
|
|||
}
|
||||
|
||||
// CreateDatabase is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - db *model.Database
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - db *model.Database
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) CreateDatabase(ctx interface{}, db interface{}, ts interface{}) *IMetaTable_CreateDatabase_Call {
|
||||
return &IMetaTable_CreateDatabase_Call{Call: _e.mock.On("CreateDatabase", ctx, db, ts)}
|
||||
}
|
||||
|
@ -490,8 +490,8 @@ type IMetaTable_CreateRole_Call struct {
|
|||
}
|
||||
|
||||
// CreateRole is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - entity *milvuspb.RoleEntity
|
||||
// - tenant string
|
||||
// - entity *milvuspb.RoleEntity
|
||||
func (_e *IMetaTable_Expecter) CreateRole(tenant interface{}, entity interface{}) *IMetaTable_CreateRole_Call {
|
||||
return &IMetaTable_CreateRole_Call{Call: _e.mock.On("CreateRole", tenant, entity)}
|
||||
}
|
||||
|
@ -533,7 +533,7 @@ type IMetaTable_DeleteCredential_Call struct {
|
|||
}
|
||||
|
||||
// DeleteCredential is a helper method to define mock.On call
|
||||
// - username string
|
||||
// - username string
|
||||
func (_e *IMetaTable_Expecter) DeleteCredential(username interface{}) *IMetaTable_DeleteCredential_Call {
|
||||
return &IMetaTable_DeleteCredential_Call{Call: _e.mock.On("DeleteCredential", username)}
|
||||
}
|
||||
|
@ -585,10 +585,10 @@ type IMetaTable_DescribeAlias_Call struct {
|
|||
}
|
||||
|
||||
// DescribeAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) DescribeAlias(ctx interface{}, dbName interface{}, alias interface{}, ts interface{}) *IMetaTable_DescribeAlias_Call {
|
||||
return &IMetaTable_DescribeAlias_Call{Call: _e.mock.On("DescribeAlias", ctx, dbName, alias, ts)}
|
||||
}
|
||||
|
@ -630,10 +630,10 @@ type IMetaTable_DropAlias_Call struct {
|
|||
}
|
||||
|
||||
// DropAlias is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - alias string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) DropAlias(ctx interface{}, dbName interface{}, alias interface{}, ts interface{}) *IMetaTable_DropAlias_Call {
|
||||
return &IMetaTable_DropAlias_Call{Call: _e.mock.On("DropAlias", ctx, dbName, alias, ts)}
|
||||
}
|
||||
|
@ -675,9 +675,9 @@ type IMetaTable_DropDatabase_Call struct {
|
|||
}
|
||||
|
||||
// DropDatabase is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) DropDatabase(ctx interface{}, dbName interface{}, ts interface{}) *IMetaTable_DropDatabase_Call {
|
||||
return &IMetaTable_DropDatabase_Call{Call: _e.mock.On("DropDatabase", ctx, dbName, ts)}
|
||||
}
|
||||
|
@ -719,8 +719,8 @@ type IMetaTable_DropGrant_Call struct {
|
|||
}
|
||||
|
||||
// DropGrant is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - role *milvuspb.RoleEntity
|
||||
// - tenant string
|
||||
// - role *milvuspb.RoleEntity
|
||||
func (_e *IMetaTable_Expecter) DropGrant(tenant interface{}, role interface{}) *IMetaTable_DropGrant_Call {
|
||||
return &IMetaTable_DropGrant_Call{Call: _e.mock.On("DropGrant", tenant, role)}
|
||||
}
|
||||
|
@ -762,8 +762,8 @@ type IMetaTable_DropRole_Call struct {
|
|||
}
|
||||
|
||||
// DropRole is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - roleName string
|
||||
// - tenant string
|
||||
// - roleName string
|
||||
func (_e *IMetaTable_Expecter) DropRole(tenant interface{}, roleName interface{}) *IMetaTable_DropRole_Call {
|
||||
return &IMetaTable_DropRole_Call{Call: _e.mock.On("DropRole", tenant, roleName)}
|
||||
}
|
||||
|
@ -817,11 +817,11 @@ type IMetaTable_GetCollectionByID_Call struct {
|
|||
}
|
||||
|
||||
// GetCollectionByID is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - collectionID int64
|
||||
// - ts uint64
|
||||
// - allowUnavailable bool
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - collectionID int64
|
||||
// - ts uint64
|
||||
// - allowUnavailable bool
|
||||
func (_e *IMetaTable_Expecter) GetCollectionByID(ctx interface{}, dbName interface{}, collectionID interface{}, ts interface{}, allowUnavailable interface{}) *IMetaTable_GetCollectionByID_Call {
|
||||
return &IMetaTable_GetCollectionByID_Call{Call: _e.mock.On("GetCollectionByID", ctx, dbName, collectionID, ts, allowUnavailable)}
|
||||
}
|
||||
|
@ -875,10 +875,10 @@ type IMetaTable_GetCollectionByName_Call struct {
|
|||
}
|
||||
|
||||
// GetCollectionByName is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) GetCollectionByName(ctx interface{}, dbName interface{}, collectionName interface{}, ts interface{}) *IMetaTable_GetCollectionByName_Call {
|
||||
return &IMetaTable_GetCollectionByName_Call{Call: _e.mock.On("GetCollectionByName", ctx, dbName, collectionName, ts)}
|
||||
}
|
||||
|
@ -922,7 +922,7 @@ type IMetaTable_GetCollectionVirtualChannels_Call struct {
|
|||
}
|
||||
|
||||
// GetCollectionVirtualChannels is a helper method to define mock.On call
|
||||
// - colID int64
|
||||
// - colID int64
|
||||
func (_e *IMetaTable_Expecter) GetCollectionVirtualChannels(colID interface{}) *IMetaTable_GetCollectionVirtualChannels_Call {
|
||||
return &IMetaTable_GetCollectionVirtualChannels_Call{Call: _e.mock.On("GetCollectionVirtualChannels", colID)}
|
||||
}
|
||||
|
@ -976,7 +976,7 @@ type IMetaTable_GetCredential_Call struct {
|
|||
}
|
||||
|
||||
// GetCredential is a helper method to define mock.On call
|
||||
// - username string
|
||||
// - username string
|
||||
func (_e *IMetaTable_Expecter) GetCredential(username interface{}) *IMetaTable_GetCredential_Call {
|
||||
return &IMetaTable_GetCredential_Call{Call: _e.mock.On("GetCredential", username)}
|
||||
}
|
||||
|
@ -1030,9 +1030,9 @@ type IMetaTable_GetDatabaseByID_Call struct {
|
|||
}
|
||||
|
||||
// GetDatabaseByID is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbID int64
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbID int64
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) GetDatabaseByID(ctx interface{}, dbID interface{}, ts interface{}) *IMetaTable_GetDatabaseByID_Call {
|
||||
return &IMetaTable_GetDatabaseByID_Call{Call: _e.mock.On("GetDatabaseByID", ctx, dbID, ts)}
|
||||
}
|
||||
|
@ -1086,9 +1086,9 @@ type IMetaTable_GetDatabaseByName_Call struct {
|
|||
}
|
||||
|
||||
// GetDatabaseByName is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) GetDatabaseByName(ctx interface{}, dbName interface{}, ts interface{}) *IMetaTable_GetDatabaseByName_Call {
|
||||
return &IMetaTable_GetDatabaseByName_Call{Call: _e.mock.On("GetDatabaseByName", ctx, dbName, ts)}
|
||||
}
|
||||
|
@ -1110,114 +1110,6 @@ func (_c *IMetaTable_GetDatabaseByName_Call) RunAndReturn(run func(context.Conte
|
|||
return _c
|
||||
}
|
||||
|
||||
// GetPartitionByName provides a mock function with given fields: collID, partitionName, ts
|
||||
func (_m *IMetaTable) GetPartitionByName(collID int64, partitionName string, ts uint64) (int64, error) {
|
||||
ret := _m.Called(collID, partitionName, ts)
|
||||
|
||||
var r0 int64
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(int64, string, uint64) (int64, error)); ok {
|
||||
return rf(collID, partitionName, ts)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(int64, string, uint64) int64); ok {
|
||||
r0 = rf(collID, partitionName, ts)
|
||||
} else {
|
||||
r0 = ret.Get(0).(int64)
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(int64, string, uint64) error); ok {
|
||||
r1 = rf(collID, partitionName, ts)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// IMetaTable_GetPartitionByName_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetPartitionByName'
|
||||
type IMetaTable_GetPartitionByName_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// GetPartitionByName is a helper method to define mock.On call
|
||||
// - collID int64
|
||||
// - partitionName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) GetPartitionByName(collID interface{}, partitionName interface{}, ts interface{}) *IMetaTable_GetPartitionByName_Call {
|
||||
return &IMetaTable_GetPartitionByName_Call{Call: _e.mock.On("GetPartitionByName", collID, partitionName, ts)}
|
||||
}
|
||||
|
||||
func (_c *IMetaTable_GetPartitionByName_Call) Run(run func(collID int64, partitionName string, ts uint64)) *IMetaTable_GetPartitionByName_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(int64), args[1].(string), args[2].(uint64))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *IMetaTable_GetPartitionByName_Call) Return(_a0 int64, _a1 error) *IMetaTable_GetPartitionByName_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *IMetaTable_GetPartitionByName_Call) RunAndReturn(run func(int64, string, uint64) (int64, error)) *IMetaTable_GetPartitionByName_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// GetPartitionNameByID provides a mock function with given fields: collID, partitionID, ts
|
||||
func (_m *IMetaTable) GetPartitionNameByID(collID int64, partitionID int64, ts uint64) (string, error) {
|
||||
ret := _m.Called(collID, partitionID, ts)
|
||||
|
||||
var r0 string
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(0).(func(int64, int64, uint64) (string, error)); ok {
|
||||
return rf(collID, partitionID, ts)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(int64, int64, uint64) string); ok {
|
||||
r0 = rf(collID, partitionID, ts)
|
||||
} else {
|
||||
r0 = ret.Get(0).(string)
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(int64, int64, uint64) error); ok {
|
||||
r1 = rf(collID, partitionID, ts)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// IMetaTable_GetPartitionNameByID_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetPartitionNameByID'
|
||||
type IMetaTable_GetPartitionNameByID_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// GetPartitionNameByID is a helper method to define mock.On call
|
||||
// - collID int64
|
||||
// - partitionID int64
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) GetPartitionNameByID(collID interface{}, partitionID interface{}, ts interface{}) *IMetaTable_GetPartitionNameByID_Call {
|
||||
return &IMetaTable_GetPartitionNameByID_Call{Call: _e.mock.On("GetPartitionNameByID", collID, partitionID, ts)}
|
||||
}
|
||||
|
||||
func (_c *IMetaTable_GetPartitionNameByID_Call) Run(run func(collID int64, partitionID int64, ts uint64)) *IMetaTable_GetPartitionNameByID_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(int64), args[1].(int64), args[2].(uint64))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *IMetaTable_GetPartitionNameByID_Call) Return(_a0 string, _a1 error) *IMetaTable_GetPartitionNameByID_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *IMetaTable_GetPartitionNameByID_Call) RunAndReturn(run func(int64, int64, uint64) (string, error)) *IMetaTable_GetPartitionNameByID_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// IsAlias provides a mock function with given fields: db, name
|
||||
func (_m *IMetaTable) IsAlias(db string, name string) bool {
|
||||
ret := _m.Called(db, name)
|
||||
|
@ -1238,8 +1130,8 @@ type IMetaTable_IsAlias_Call struct {
|
|||
}
|
||||
|
||||
// IsAlias is a helper method to define mock.On call
|
||||
// - db string
|
||||
// - name string
|
||||
// - db string
|
||||
// - name string
|
||||
func (_e *IMetaTable_Expecter) IsAlias(db interface{}, name interface{}) *IMetaTable_IsAlias_Call {
|
||||
return &IMetaTable_IsAlias_Call{Call: _e.mock.On("IsAlias", db, name)}
|
||||
}
|
||||
|
@ -1293,10 +1185,10 @@ type IMetaTable_ListAliases_Call struct {
|
|||
}
|
||||
|
||||
// ListAliases is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - collectionName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) ListAliases(ctx interface{}, dbName interface{}, collectionName interface{}, ts interface{}) *IMetaTable_ListAliases_Call {
|
||||
return &IMetaTable_ListAliases_Call{Call: _e.mock.On("ListAliases", ctx, dbName, collectionName, ts)}
|
||||
}
|
||||
|
@ -1340,7 +1232,7 @@ type IMetaTable_ListAliasesByID_Call struct {
|
|||
}
|
||||
|
||||
// ListAliasesByID is a helper method to define mock.On call
|
||||
// - collID int64
|
||||
// - collID int64
|
||||
func (_e *IMetaTable_Expecter) ListAliasesByID(collID interface{}) *IMetaTable_ListAliasesByID_Call {
|
||||
return &IMetaTable_ListAliasesByID_Call{Call: _e.mock.On("ListAliasesByID", collID)}
|
||||
}
|
||||
|
@ -1384,7 +1276,7 @@ type IMetaTable_ListAllAvailCollections_Call struct {
|
|||
}
|
||||
|
||||
// ListAllAvailCollections is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - ctx context.Context
|
||||
func (_e *IMetaTable_Expecter) ListAllAvailCollections(ctx interface{}) *IMetaTable_ListAllAvailCollections_Call {
|
||||
return &IMetaTable_ListAllAvailCollections_Call{Call: _e.mock.On("ListAllAvailCollections", ctx)}
|
||||
}
|
||||
|
@ -1481,10 +1373,10 @@ type IMetaTable_ListCollections_Call struct {
|
|||
}
|
||||
|
||||
// ListCollections is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - ts uint64
|
||||
// - onlyAvail bool
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - ts uint64
|
||||
// - onlyAvail bool
|
||||
func (_e *IMetaTable_Expecter) ListCollections(ctx interface{}, dbName interface{}, ts interface{}, onlyAvail interface{}) *IMetaTable_ListCollections_Call {
|
||||
return &IMetaTable_ListCollections_Call{Call: _e.mock.On("ListCollections", ctx, dbName, ts, onlyAvail)}
|
||||
}
|
||||
|
@ -1591,8 +1483,8 @@ type IMetaTable_ListDatabases_Call struct {
|
|||
}
|
||||
|
||||
// ListDatabases is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) ListDatabases(ctx interface{}, ts interface{}) *IMetaTable_ListDatabases_Call {
|
||||
return &IMetaTable_ListDatabases_Call{Call: _e.mock.On("ListDatabases", ctx, ts)}
|
||||
}
|
||||
|
@ -1646,7 +1538,7 @@ type IMetaTable_ListPolicy_Call struct {
|
|||
}
|
||||
|
||||
// ListPolicy is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - tenant string
|
||||
func (_e *IMetaTable_Expecter) ListPolicy(tenant interface{}) *IMetaTable_ListPolicy_Call {
|
||||
return &IMetaTable_ListPolicy_Call{Call: _e.mock.On("ListPolicy", tenant)}
|
||||
}
|
||||
|
@ -1700,7 +1592,7 @@ type IMetaTable_ListUserRole_Call struct {
|
|||
}
|
||||
|
||||
// ListUserRole is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - tenant string
|
||||
func (_e *IMetaTable_Expecter) ListUserRole(tenant interface{}) *IMetaTable_ListUserRole_Call {
|
||||
return &IMetaTable_ListUserRole_Call{Call: _e.mock.On("ListUserRole", tenant)}
|
||||
}
|
||||
|
@ -1742,9 +1634,9 @@ type IMetaTable_OperatePrivilege_Call struct {
|
|||
}
|
||||
|
||||
// OperatePrivilege is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - entity *milvuspb.GrantEntity
|
||||
// - operateType milvuspb.OperatePrivilegeType
|
||||
// - tenant string
|
||||
// - entity *milvuspb.GrantEntity
|
||||
// - operateType milvuspb.OperatePrivilegeType
|
||||
func (_e *IMetaTable_Expecter) OperatePrivilege(tenant interface{}, entity interface{}, operateType interface{}) *IMetaTable_OperatePrivilege_Call {
|
||||
return &IMetaTable_OperatePrivilege_Call{Call: _e.mock.On("OperatePrivilege", tenant, entity, operateType)}
|
||||
}
|
||||
|
@ -1786,10 +1678,10 @@ type IMetaTable_OperateUserRole_Call struct {
|
|||
}
|
||||
|
||||
// OperateUserRole is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - userEntity *milvuspb.UserEntity
|
||||
// - roleEntity *milvuspb.RoleEntity
|
||||
// - operateType milvuspb.OperateUserRoleType
|
||||
// - tenant string
|
||||
// - userEntity *milvuspb.UserEntity
|
||||
// - roleEntity *milvuspb.RoleEntity
|
||||
// - operateType milvuspb.OperateUserRoleType
|
||||
func (_e *IMetaTable_Expecter) OperateUserRole(tenant interface{}, userEntity interface{}, roleEntity interface{}, operateType interface{}) *IMetaTable_OperateUserRole_Call {
|
||||
return &IMetaTable_OperateUserRole_Call{Call: _e.mock.On("OperateUserRole", tenant, userEntity, roleEntity, operateType)}
|
||||
}
|
||||
|
@ -1831,9 +1723,9 @@ type IMetaTable_RemoveCollection_Call struct {
|
|||
}
|
||||
|
||||
// RemoveCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - collectionID int64
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - collectionID int64
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) RemoveCollection(ctx interface{}, collectionID interface{}, ts interface{}) *IMetaTable_RemoveCollection_Call {
|
||||
return &IMetaTable_RemoveCollection_Call{Call: _e.mock.On("RemoveCollection", ctx, collectionID, ts)}
|
||||
}
|
||||
|
@ -1875,11 +1767,11 @@ type IMetaTable_RemovePartition_Call struct {
|
|||
}
|
||||
|
||||
// RemovePartition is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbID int64
|
||||
// - collectionID int64
|
||||
// - partitionID int64
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbID int64
|
||||
// - collectionID int64
|
||||
// - partitionID int64
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) RemovePartition(ctx interface{}, dbID interface{}, collectionID interface{}, partitionID interface{}, ts interface{}) *IMetaTable_RemovePartition_Call {
|
||||
return &IMetaTable_RemovePartition_Call{Call: _e.mock.On("RemovePartition", ctx, dbID, collectionID, partitionID, ts)}
|
||||
}
|
||||
|
@ -1921,12 +1813,12 @@ type IMetaTable_RenameCollection_Call struct {
|
|||
}
|
||||
|
||||
// RenameCollection is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - oldName string
|
||||
// - newDBName string
|
||||
// - newName string
|
||||
// - ts uint64
|
||||
// - ctx context.Context
|
||||
// - dbName string
|
||||
// - oldName string
|
||||
// - newDBName string
|
||||
// - newName string
|
||||
// - ts uint64
|
||||
func (_e *IMetaTable_Expecter) RenameCollection(ctx interface{}, dbName interface{}, oldName interface{}, newDBName interface{}, newName interface{}, ts interface{}) *IMetaTable_RenameCollection_Call {
|
||||
return &IMetaTable_RenameCollection_Call{Call: _e.mock.On("RenameCollection", ctx, dbName, oldName, newDBName, newName, ts)}
|
||||
}
|
||||
|
@ -1980,8 +1872,8 @@ type IMetaTable_SelectGrant_Call struct {
|
|||
}
|
||||
|
||||
// SelectGrant is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - entity *milvuspb.GrantEntity
|
||||
// - tenant string
|
||||
// - entity *milvuspb.GrantEntity
|
||||
func (_e *IMetaTable_Expecter) SelectGrant(tenant interface{}, entity interface{}) *IMetaTable_SelectGrant_Call {
|
||||
return &IMetaTable_SelectGrant_Call{Call: _e.mock.On("SelectGrant", tenant, entity)}
|
||||
}
|
||||
|
@ -2035,9 +1927,9 @@ type IMetaTable_SelectRole_Call struct {
|
|||
}
|
||||
|
||||
// SelectRole is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - entity *milvuspb.RoleEntity
|
||||
// - includeUserInfo bool
|
||||
// - tenant string
|
||||
// - entity *milvuspb.RoleEntity
|
||||
// - includeUserInfo bool
|
||||
func (_e *IMetaTable_Expecter) SelectRole(tenant interface{}, entity interface{}, includeUserInfo interface{}) *IMetaTable_SelectRole_Call {
|
||||
return &IMetaTable_SelectRole_Call{Call: _e.mock.On("SelectRole", tenant, entity, includeUserInfo)}
|
||||
}
|
||||
|
@ -2091,9 +1983,9 @@ type IMetaTable_SelectUser_Call struct {
|
|||
}
|
||||
|
||||
// SelectUser is a helper method to define mock.On call
|
||||
// - tenant string
|
||||
// - entity *milvuspb.UserEntity
|
||||
// - includeRoleInfo bool
|
||||
// - tenant string
|
||||
// - entity *milvuspb.UserEntity
|
||||
// - includeRoleInfo bool
|
||||
func (_e *IMetaTable_Expecter) SelectUser(tenant interface{}, entity interface{}, includeRoleInfo interface{}) *IMetaTable_SelectUser_Call {
|
||||
return &IMetaTable_SelectUser_Call{Call: _e.mock.On("SelectUser", tenant, entity, includeRoleInfo)}
|
||||
}
|
||||
|
|
|
@ -49,7 +49,6 @@ import (
|
|||
tso2 "github.com/milvus-io/milvus/internal/tso"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/importutil"
|
||||
"github.com/milvus-io/milvus/internal/util/proxyutil"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
tsoutil2 "github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
|
@ -125,8 +124,6 @@ type Core struct {
|
|||
|
||||
factory dependency.Factory
|
||||
|
||||
importManager *importManager
|
||||
|
||||
enableActiveStandBy bool
|
||||
activateFunc func() error
|
||||
}
|
||||
|
@ -422,27 +419,6 @@ func (c *Core) initTSOAllocator() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (c *Core) initImportManager() error {
|
||||
impTaskKv, err := c.metaKVCreator()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
f := NewImportFactory(c)
|
||||
c.importManager = newImportManager(
|
||||
c.ctx,
|
||||
impTaskKv,
|
||||
f.NewIDAllocator(),
|
||||
f.NewImportFunc(),
|
||||
f.NewGetSegmentStatesFunc(),
|
||||
f.NewGetCollectionNameFunc(),
|
||||
f.NewUnsetIsImportingStateFunc(),
|
||||
)
|
||||
c.importManager.init(c.ctx)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Core) initInternal() error {
|
||||
c.UpdateStateCode(commonpb.StateCode_Initializing)
|
||||
c.initKVCreator()
|
||||
|
@ -487,11 +463,6 @@ func (c *Core) initInternal() error {
|
|||
c.quotaCenter = NewQuotaCenter(c.proxyClientManager, c.queryCoord, c.dataCoord, c.tsoAllocator, c.meta)
|
||||
log.Debug("RootCoord init QuotaCenter done")
|
||||
|
||||
if err := c.initImportManager(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("init import manager done")
|
||||
|
||||
if err := c.initCredentials(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -726,13 +697,10 @@ func (c *Core) startInternal() error {
|
|||
}
|
||||
|
||||
func (c *Core) startServerLoop() {
|
||||
c.wg.Add(6)
|
||||
c.wg.Add(3)
|
||||
go c.startTimeTickLoop()
|
||||
go c.tsLoop()
|
||||
go c.chanTimeTick.startWatch(&c.wg)
|
||||
go c.importManager.cleanupLoop(&c.wg)
|
||||
go c.importManager.sendOutTasksLoop(&c.wg)
|
||||
go c.importManager.flipTaskStateLoop(&c.wg)
|
||||
}
|
||||
|
||||
// Start starts RootCoord.
|
||||
|
@ -1928,203 +1896,6 @@ func (c *Core) ListAliases(ctx context.Context, in *milvuspb.ListAliasesRequest)
|
|||
}, nil
|
||||
}
|
||||
|
||||
// Import imports large files (json, numpy, etc.) on MinIO/S3 storage into Milvus storage.
|
||||
func (c *Core) Import(ctx context.Context, req *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error) {
|
||||
if err := merr.CheckHealthy(c.GetStateCode()); err != nil {
|
||||
return &milvuspb.ImportResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Get collection/partition ID from collection/partition name.
|
||||
var colInfo *model.Collection
|
||||
var err error
|
||||
if colInfo, err = c.meta.GetCollectionByName(ctx, req.GetDbName(), req.GetCollectionName(), typeutil.MaxTimestamp); err != nil {
|
||||
log.Error("failed to find collection ID from its name",
|
||||
zap.String("collectionName", req.GetCollectionName()),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
isBackUp := importutil.IsBackup(req.GetOptions())
|
||||
cID := colInfo.CollectionID
|
||||
req.ChannelNames = c.meta.GetCollectionVirtualChannels(cID)
|
||||
|
||||
hasPartitionKey := false
|
||||
for _, field := range colInfo.Fields {
|
||||
if field.IsPartitionKey {
|
||||
hasPartitionKey = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Get partition ID by partition name
|
||||
var pID UniqueID
|
||||
if isBackUp {
|
||||
// Currently, Backup tool call import must with a partition name, each time restore a partition
|
||||
if req.GetPartitionName() != "" {
|
||||
if pID, err = c.meta.GetPartitionByName(cID, req.GetPartitionName(), typeutil.MaxTimestamp); err != nil {
|
||||
log.Warn("failed to get partition ID from its name", zap.String("partitionName", req.GetPartitionName()), zap.Error(err))
|
||||
return &milvuspb.ImportResponse{
|
||||
Status: merr.Status(merr.WrapErrPartitionNotFound(req.GetPartitionName())),
|
||||
}, nil
|
||||
}
|
||||
} else {
|
||||
log.Info("partition name not specified when backup recovery",
|
||||
zap.String("collectionName", req.GetCollectionName()))
|
||||
return &milvuspb.ImportResponse{
|
||||
Status: merr.Status(merr.WrapErrParameterInvalidMsg("partition not specified")),
|
||||
}, nil
|
||||
}
|
||||
} else {
|
||||
if hasPartitionKey {
|
||||
if req.GetPartitionName() != "" {
|
||||
msg := "not allow to set partition name for collection with partition key"
|
||||
log.Warn(msg, zap.String("collectionName", req.GetCollectionName()))
|
||||
return &milvuspb.ImportResponse{
|
||||
Status: merr.Status(merr.WrapErrParameterInvalidMsg(msg)),
|
||||
}, nil
|
||||
}
|
||||
} else {
|
||||
if req.GetPartitionName() == "" {
|
||||
req.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue()
|
||||
}
|
||||
if pID, err = c.meta.GetPartitionByName(cID, req.GetPartitionName(), typeutil.MaxTimestamp); err != nil {
|
||||
log.Warn("failed to get partition ID from its name",
|
||||
zap.String("partition name", req.GetPartitionName()),
|
||||
zap.Error(err))
|
||||
return &milvuspb.ImportResponse{
|
||||
Status: merr.Status(merr.WrapErrPartitionNotFound(req.GetPartitionName())),
|
||||
}, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
log.Info("RootCoord receive import request",
|
||||
zap.String("collectionName", req.GetCollectionName()),
|
||||
zap.Int64("collectionID", cID),
|
||||
zap.String("partitionName", req.GetPartitionName()),
|
||||
zap.Strings("virtualChannelNames", req.GetChannelNames()),
|
||||
zap.Int64("partitionID", pID),
|
||||
zap.Int("# of files = ", len(req.GetFiles())),
|
||||
)
|
||||
importJobResp := c.importManager.importJob(ctx, req, cID, pID)
|
||||
return importJobResp, nil
|
||||
}
|
||||
|
||||
// GetImportState returns the current state of an import task.
|
||||
func (c *Core) GetImportState(ctx context.Context, req *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error) {
|
||||
if err := merr.CheckHealthy(c.GetStateCode()); err != nil {
|
||||
return &milvuspb.GetImportStateResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
return c.importManager.getTaskState(req.GetTask()), nil
|
||||
}
|
||||
|
||||
// ListImportTasks returns id array of all import tasks.
|
||||
func (c *Core) ListImportTasks(ctx context.Context, req *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error) {
|
||||
if err := merr.CheckHealthy(c.GetStateCode()); err != nil {
|
||||
return &milvuspb.ListImportTasksResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
|
||||
colID := int64(-1)
|
||||
collectionName := req.GetCollectionName()
|
||||
if len(collectionName) != 0 {
|
||||
// if the collection name is specified but not found, user may input a wrong name, the collection doesn't exist or has been dropped.
|
||||
// we will return error to notify user the name is incorrect.
|
||||
colInfo, err := c.meta.GetCollectionByName(ctx, req.GetDbName(), req.GetCollectionName(), typeutil.MaxTimestamp)
|
||||
if err != nil {
|
||||
err = fmt.Errorf("failed to find collection ID from its name: '%s', error: %w", req.GetCollectionName(), err)
|
||||
log.Error("ListImportTasks failed", zap.Error(err))
|
||||
status := merr.Status(err)
|
||||
return &milvuspb.ListImportTasksResponse{
|
||||
Status: status,
|
||||
}, nil
|
||||
}
|
||||
colID = colInfo.CollectionID
|
||||
}
|
||||
|
||||
// if the collection name is not specified, the colID is -1, listAllTasks will return all tasks
|
||||
tasks, err := c.importManager.listAllTasks(colID, req.GetLimit())
|
||||
if err != nil {
|
||||
err = fmt.Errorf("failed to list import tasks, collection name: '%s', error: %w", req.GetCollectionName(), err)
|
||||
log.Error("ListImportTasks failed", zap.Error(err))
|
||||
return &milvuspb.ListImportTasksResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
}
|
||||
|
||||
resp := &milvuspb.ListImportTasksResponse{
|
||||
Status: merr.Success(),
|
||||
Tasks: tasks,
|
||||
}
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
// ReportImport reports import task state to RootCoord.
|
||||
func (c *Core) ReportImport(ctx context.Context, ir *rootcoordpb.ImportResult) (*commonpb.Status, error) {
|
||||
log.Info("RootCoord receive import state report",
|
||||
zap.Int64("task ID", ir.GetTaskId()),
|
||||
zap.Any("import state", ir.GetState()))
|
||||
if err := merr.CheckHealthy(c.GetStateCode()); err != nil {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// This method update a busy node to idle node, and send import task to idle node
|
||||
resendTaskFunc := func() {
|
||||
func() {
|
||||
c.importManager.busyNodesLock.Lock()
|
||||
defer c.importManager.busyNodesLock.Unlock()
|
||||
delete(c.importManager.busyNodes, ir.GetDatanodeId())
|
||||
log.Info("a DataNode is no longer busy after processing task",
|
||||
zap.Int64("dataNode ID", ir.GetDatanodeId()),
|
||||
zap.Int64("task ID", ir.GetTaskId()))
|
||||
}()
|
||||
err := c.importManager.sendOutTasks(c.importManager.ctx)
|
||||
if err != nil {
|
||||
log.Error("fail to send out import task to datanodes")
|
||||
}
|
||||
}
|
||||
|
||||
// If setting ImportState_ImportCompleted, simply update the state and return directly.
|
||||
if ir.GetState() == commonpb.ImportState_ImportCompleted {
|
||||
log.Warn("this should not be called!")
|
||||
}
|
||||
// Upon receiving ReportImport request, update the related task's state in task store.
|
||||
ti, err := c.importManager.updateTaskInfo(ir)
|
||||
if err != nil {
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
||||
// If task failed, send task to idle datanode
|
||||
if ir.GetState() == commonpb.ImportState_ImportFailed {
|
||||
// When a DataNode failed importing, remove this DataNode from the busy node list and send out import tasks again.
|
||||
log.Info("an import task has failed, marking DataNode available and resending import task",
|
||||
zap.Int64("task ID", ir.GetTaskId()))
|
||||
resendTaskFunc()
|
||||
} else if ir.GetState() == commonpb.ImportState_ImportCompleted {
|
||||
// When a DataNode completes importing, remove this DataNode from the busy node list and send out import tasks again.
|
||||
log.Info("an import task has completed, marking DataNode available and resending import task",
|
||||
zap.Int64("task ID", ir.GetTaskId()))
|
||||
resendTaskFunc()
|
||||
} else if ir.GetState() == commonpb.ImportState_ImportPersisted {
|
||||
// Here ir.GetState() == commonpb.ImportState_ImportPersisted
|
||||
// Seal these import segments, so they can be auto-flushed later.
|
||||
log.Info("an import task turns to persisted state, flush segments to be sealed",
|
||||
zap.Int64("task ID", ir.GetTaskId()), zap.Any("segments", ir.GetSegments()))
|
||||
if err := c.broker.Flush(ctx, ti.GetCollectionId(), ir.GetSegments()); err != nil {
|
||||
log.Error("failed to call Flush on bulk insert segments",
|
||||
zap.Int64("task ID", ir.GetTaskId()))
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
}
|
||||
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
// ExpireCredCache will call invalidate credential cache
|
||||
func (c *Core) ExpireCredCache(ctx context.Context, username string) error {
|
||||
req := proxypb.InvalidateCredCacheRequest{
|
||||
|
|
|
@ -21,24 +21,17 @@ import (
|
|||
"fmt"
|
||||
"math/rand"
|
||||
"os"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"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/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
||||
"github.com/milvus-io/milvus/internal/kv/mocks"
|
||||
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||
mocksutil "github.com/milvus-io/milvus/internal/mocks"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||
|
@ -46,9 +39,7 @@ import (
|
|||
mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv"
|
||||
"github.com/milvus-io/milvus/internal/util/importutil"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util"
|
||||
"github.com/milvus-io/milvus/pkg/util/etcd"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
|
@ -1112,529 +1103,6 @@ func TestRootCoord_GetMetrics(t *testing.T) {
|
|||
})
|
||||
}
|
||||
|
||||
func TestCore_Import(t *testing.T) {
|
||||
meta := newMockMetaTable()
|
||||
meta.AddCollectionFunc = func(ctx context.Context, coll *model.Collection) error {
|
||||
return nil
|
||||
}
|
||||
meta.ChangeCollectionStateFunc = func(ctx context.Context, collectionID UniqueID, state etcdpb.CollectionState, ts Timestamp) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
t.Run("not healthy", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withAbnormalCode())
|
||||
resp, err := c.Import(ctx, &milvuspb.ImportRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("bad collection name", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(),
|
||||
withMeta(meta))
|
||||
meta.GetCollectionIDByNameFunc = func(name string) (UniqueID, error) {
|
||||
return 0, errors.New("error mock GetCollectionIDByName")
|
||||
}
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return nil, errors.New("collection name not found")
|
||||
}
|
||||
_, err := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-bad-name",
|
||||
})
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("bad partition name", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(),
|
||||
withMeta(meta))
|
||||
coll := &model.Collection{Name: "a-good-name"}
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return coll, nil
|
||||
}
|
||||
meta.GetCollectionVirtualChannelsFunc = func(colID int64) []string {
|
||||
return []string{"ch-1", "ch-2"}
|
||||
}
|
||||
meta.GetPartitionByNameFunc = func(collID UniqueID, partitionName string, ts Timestamp) (UniqueID, error) {
|
||||
return 0, errors.New("mock GetPartitionByNameFunc error")
|
||||
}
|
||||
resp, err := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-good-name",
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.ErrorIs(t, merr.Error(resp.GetStatus()), merr.ErrPartitionNotFound)
|
||||
})
|
||||
|
||||
t.Run("normal case", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(),
|
||||
withMeta(meta))
|
||||
meta.GetCollectionIDByNameFunc = func(name string) (UniqueID, error) {
|
||||
return 100, nil
|
||||
}
|
||||
meta.GetCollectionVirtualChannelsFunc = func(colID int64) []string {
|
||||
return []string{"ch-1", "ch-2"}
|
||||
}
|
||||
meta.GetPartitionByNameFunc = func(collID UniqueID, partitionName string, ts Timestamp) (UniqueID, error) {
|
||||
return 101, nil
|
||||
}
|
||||
coll := &model.Collection{Name: "a-good-name"}
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return coll.Clone(), nil
|
||||
}
|
||||
_, err := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-good-name",
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("backup without partition name", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(),
|
||||
withMeta(meta))
|
||||
|
||||
coll := &model.Collection{
|
||||
Name: "a-good-name",
|
||||
}
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return coll.Clone(), nil
|
||||
}
|
||||
resp, _ := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-good-name",
|
||||
Options: []*commonpb.KeyValuePair{
|
||||
{Key: importutil.BackupFlag, Value: "true"},
|
||||
},
|
||||
})
|
||||
assert.NotNil(t, resp)
|
||||
assert.ErrorIs(t, merr.Error(resp.GetStatus()), merr.ErrParameterInvalid)
|
||||
})
|
||||
|
||||
// Remove the following case after bulkinsert can support partition key
|
||||
t.Run("unsupport partition key", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(),
|
||||
withMeta(meta))
|
||||
|
||||
coll := &model.Collection{
|
||||
Name: "a-good-name",
|
||||
Fields: []*model.Field{
|
||||
{IsPartitionKey: true},
|
||||
},
|
||||
}
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return coll.Clone(), nil
|
||||
}
|
||||
resp, _ := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-good-name",
|
||||
})
|
||||
assert.NotNil(t, resp)
|
||||
})
|
||||
|
||||
t.Run("not allow partiton name with partition key", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(),
|
||||
withMeta(meta))
|
||||
meta.GetCollectionIDByNameFunc = func(name string) (UniqueID, error) {
|
||||
return 100, nil
|
||||
}
|
||||
meta.GetCollectionVirtualChannelsFunc = func(colID int64) []string {
|
||||
return []string{"ch-1", "ch-2"}
|
||||
}
|
||||
meta.GetPartitionByNameFunc = func(collID UniqueID, partitionName string, ts Timestamp) (UniqueID, error) {
|
||||
return 101, nil
|
||||
}
|
||||
coll := &model.Collection{
|
||||
CollectionID: 100,
|
||||
Name: "a-good-name",
|
||||
Fields: []*model.Field{
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "test_field_name_1",
|
||||
IsPrimaryKey: false,
|
||||
IsPartitionKey: true,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
},
|
||||
}
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return coll.Clone(), nil
|
||||
}
|
||||
resp, err := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-good-name",
|
||||
PartitionName: "p1",
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.ErrorIs(t, merr.Error(resp.GetStatus()), merr.ErrParameterInvalid)
|
||||
})
|
||||
|
||||
t.Run("backup should set partition name", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(),
|
||||
withMeta(meta))
|
||||
meta.GetCollectionIDByNameFunc = func(name string) (UniqueID, error) {
|
||||
return 100, nil
|
||||
}
|
||||
meta.GetCollectionVirtualChannelsFunc = func(colID int64) []string {
|
||||
return []string{"ch-1", "ch-2"}
|
||||
}
|
||||
meta.GetPartitionByNameFunc = func(collID UniqueID, partitionName string, ts Timestamp) (UniqueID, error) {
|
||||
return 101, nil
|
||||
}
|
||||
coll := &model.Collection{
|
||||
CollectionID: 100,
|
||||
Name: "a-good-name",
|
||||
Fields: []*model.Field{
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "test_field_name_1",
|
||||
IsPrimaryKey: false,
|
||||
IsPartitionKey: true,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
},
|
||||
}
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return coll.Clone(), nil
|
||||
}
|
||||
resp1, err := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-good-name",
|
||||
Options: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: importutil.BackupFlag,
|
||||
Value: "true",
|
||||
},
|
||||
},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.ErrorIs(t, merr.Error(resp1.GetStatus()), merr.ErrParameterInvalid)
|
||||
|
||||
meta.GetPartitionByNameFunc = func(collID UniqueID, partitionName string, ts Timestamp) (UniqueID, error) {
|
||||
return common.InvalidPartitionID, fmt.Errorf("partition ID not found for partition name '%s'", partitionName)
|
||||
}
|
||||
resp2, _ := c.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: "a-good-name",
|
||||
PartitionName: "a-bad-name",
|
||||
Options: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: importutil.BackupFlag,
|
||||
Value: "true",
|
||||
},
|
||||
},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.ErrorIs(t, merr.Error(resp2.GetStatus()), merr.ErrPartitionNotFound)
|
||||
})
|
||||
}
|
||||
|
||||
func TestCore_GetImportState(t *testing.T) {
|
||||
mockKv := memkv.NewMemoryKV()
|
||||
ti1 := &datapb.ImportTaskInfo{
|
||||
Id: 100,
|
||||
State: &datapb.ImportTaskState{
|
||||
StateCode: commonpb.ImportState_ImportPending,
|
||||
},
|
||||
CreateTs: time.Now().Unix() - 100,
|
||||
}
|
||||
ti2 := &datapb.ImportTaskInfo{
|
||||
Id: 200,
|
||||
State: &datapb.ImportTaskState{
|
||||
StateCode: commonpb.ImportState_ImportPersisted,
|
||||
},
|
||||
CreateTs: time.Now().Unix() - 100,
|
||||
}
|
||||
taskInfo1, err := proto.Marshal(ti1)
|
||||
assert.NoError(t, err)
|
||||
taskInfo2, err := proto.Marshal(ti2)
|
||||
assert.NoError(t, err)
|
||||
mockKv.Save(BuildImportTaskKey(1), "value")
|
||||
mockKv.Save(BuildImportTaskKey(100), string(taskInfo1))
|
||||
mockKv.Save(BuildImportTaskKey(200), string(taskInfo2))
|
||||
|
||||
t.Run("not healthy", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withAbnormalCode())
|
||||
resp, err := c.GetImportState(ctx, &milvuspb.GetImportStateRequest{
|
||||
Task: 100,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("normal case", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode())
|
||||
c.importManager = newImportManager(ctx, mockKv, nil, nil, nil, nil, nil)
|
||||
resp, err := c.GetImportState(ctx, &milvuspb.GetImportStateRequest{
|
||||
Task: 100,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(100), resp.GetId())
|
||||
assert.NotEqual(t, 0, resp.GetCreateTs())
|
||||
assert.Equal(t, commonpb.ImportState_ImportPending, resp.GetState())
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
})
|
||||
}
|
||||
|
||||
func TestCore_ListImportTasks(t *testing.T) {
|
||||
mockKv := memkv.NewMemoryKV()
|
||||
ti1 := &datapb.ImportTaskInfo{
|
||||
Id: 100,
|
||||
CollectionName: "collection-A",
|
||||
CollectionId: 1,
|
||||
State: &datapb.ImportTaskState{
|
||||
StateCode: commonpb.ImportState_ImportPending,
|
||||
},
|
||||
CreateTs: time.Now().Unix() - 300,
|
||||
}
|
||||
ti2 := &datapb.ImportTaskInfo{
|
||||
Id: 200,
|
||||
CollectionName: "collection-A",
|
||||
CollectionId: 1,
|
||||
State: &datapb.ImportTaskState{
|
||||
StateCode: commonpb.ImportState_ImportPersisted,
|
||||
},
|
||||
CreateTs: time.Now().Unix() - 200,
|
||||
}
|
||||
ti3 := &datapb.ImportTaskInfo{
|
||||
Id: 300,
|
||||
CollectionName: "collection-B",
|
||||
CollectionId: 2,
|
||||
State: &datapb.ImportTaskState{
|
||||
StateCode: commonpb.ImportState_ImportPersisted,
|
||||
},
|
||||
CreateTs: time.Now().Unix() - 100,
|
||||
}
|
||||
taskInfo1, err := proto.Marshal(ti1)
|
||||
assert.NoError(t, err)
|
||||
taskInfo2, err := proto.Marshal(ti2)
|
||||
assert.NoError(t, err)
|
||||
taskInfo3, err := proto.Marshal(ti3)
|
||||
assert.NoError(t, err)
|
||||
mockKv.Save(BuildImportTaskKey(1), "value") // this item will trigger an error log in importManager.loadFromTaskStore()
|
||||
mockKv.Save(BuildImportTaskKey(100), string(taskInfo1))
|
||||
mockKv.Save(BuildImportTaskKey(200), string(taskInfo2))
|
||||
mockKv.Save(BuildImportTaskKey(300), string(taskInfo3))
|
||||
|
||||
t.Run("not healthy", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withAbnormalCode())
|
||||
resp, err := c.ListImportTasks(ctx, &milvuspb.ListImportTasksRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
})
|
||||
|
||||
verifyTaskFunc := func(task *milvuspb.GetImportStateResponse, taskID int64, colID int64, state commonpb.ImportState) {
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, task.GetStatus().ErrorCode)
|
||||
assert.Equal(t, taskID, task.GetId())
|
||||
assert.Equal(t, state, task.GetState())
|
||||
assert.Equal(t, colID, task.GetCollectionId())
|
||||
}
|
||||
|
||||
t.Run("normal case", func(t *testing.T) {
|
||||
meta := newMockMetaTable()
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
if collectionName == ti1.CollectionName {
|
||||
return &model.Collection{
|
||||
CollectionID: ti1.CollectionId,
|
||||
}, nil
|
||||
} else if collectionName == ti3.CollectionName {
|
||||
return &model.Collection{
|
||||
CollectionID: ti3.CollectionId,
|
||||
}, nil
|
||||
}
|
||||
return nil, merr.WrapErrCollectionNotFound(collectionName)
|
||||
}
|
||||
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode(), withMeta(meta))
|
||||
c.importManager = newImportManager(ctx, mockKv, nil, nil, nil, nil, nil)
|
||||
|
||||
// list all tasks
|
||||
resp, err := c.ListImportTasks(ctx, &milvuspb.ListImportTasksRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 3, len(resp.GetTasks()))
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
verifyTaskFunc(resp.GetTasks()[0], 100, 1, commonpb.ImportState_ImportPending)
|
||||
verifyTaskFunc(resp.GetTasks()[1], 200, 1, commonpb.ImportState_ImportPersisted)
|
||||
verifyTaskFunc(resp.GetTasks()[2], 300, 2, commonpb.ImportState_ImportPersisted)
|
||||
|
||||
// list tasks of collection-A
|
||||
resp, err = c.ListImportTasks(ctx, &milvuspb.ListImportTasksRequest{
|
||||
CollectionName: "collection-A",
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 2, len(resp.GetTasks()))
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
|
||||
// list tasks of collection-B
|
||||
resp, err = c.ListImportTasks(ctx, &milvuspb.ListImportTasksRequest{
|
||||
CollectionName: "collection-B",
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(resp.GetTasks()))
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
|
||||
// invalid collection name
|
||||
resp, err = c.ListImportTasks(ctx, &milvuspb.ListImportTasksRequest{
|
||||
CollectionName: "dummy",
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 0, len(resp.GetTasks()))
|
||||
assert.ErrorIs(t, merr.Error(resp.GetStatus()), merr.ErrCollectionNotFound)
|
||||
|
||||
// list the latest 2 tasks
|
||||
resp, err = c.ListImportTasks(ctx, &milvuspb.ListImportTasksRequest{
|
||||
Limit: 2,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 2, len(resp.GetTasks()))
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
verifyTaskFunc(resp.GetTasks()[0], 200, 1, commonpb.ImportState_ImportPersisted)
|
||||
verifyTaskFunc(resp.GetTasks()[1], 300, 2, commonpb.ImportState_ImportPersisted)
|
||||
|
||||
// failed to load tasks from store
|
||||
mockTxnKV := &mocks.TxnKV{}
|
||||
mockTxnKV.EXPECT().LoadWithPrefix(mock.Anything).Return(nil, nil, errors.New("mock error"))
|
||||
c.importManager.taskStore = mockTxnKV
|
||||
resp, err = c.ListImportTasks(ctx, &milvuspb.ListImportTasksRequest{})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 0, len(resp.GetTasks()))
|
||||
assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetStatus().GetErrorCode())
|
||||
})
|
||||
}
|
||||
|
||||
func TestCore_ReportImport(t *testing.T) {
|
||||
paramtable.Get().Save(Params.RootCoordCfg.ImportTaskSubPath.Key, "importtask")
|
||||
var countLock sync.RWMutex
|
||||
globalCount := typeutil.UniqueID(0)
|
||||
idAlloc := func(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) {
|
||||
countLock.Lock()
|
||||
defer countLock.Unlock()
|
||||
globalCount++
|
||||
return globalCount, 0, nil
|
||||
}
|
||||
mockKv := memkv.NewMemoryKV()
|
||||
ti1 := &datapb.ImportTaskInfo{
|
||||
Id: 100,
|
||||
State: &datapb.ImportTaskState{
|
||||
StateCode: commonpb.ImportState_ImportPending,
|
||||
},
|
||||
CreateTs: time.Now().Unix() - 100,
|
||||
}
|
||||
ti2 := &datapb.ImportTaskInfo{
|
||||
Id: 200,
|
||||
State: &datapb.ImportTaskState{
|
||||
StateCode: commonpb.ImportState_ImportPersisted,
|
||||
},
|
||||
CreateTs: time.Now().Unix() - 100,
|
||||
}
|
||||
taskInfo1, err := proto.Marshal(ti1)
|
||||
assert.NoError(t, err)
|
||||
taskInfo2, err := proto.Marshal(ti2)
|
||||
assert.NoError(t, err)
|
||||
mockKv.Save(BuildImportTaskKey(1), "value")
|
||||
mockKv.Save(BuildImportTaskKey(100), string(taskInfo1))
|
||||
mockKv.Save(BuildImportTaskKey(200), string(taskInfo2))
|
||||
|
||||
ticker := newRocksMqTtSynchronizer()
|
||||
meta := newMockMetaTable()
|
||||
meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) {
|
||||
return nil, errors.New("error mock GetCollectionByName")
|
||||
}
|
||||
meta.AddCollectionFunc = func(ctx context.Context, coll *model.Collection) error {
|
||||
return nil
|
||||
}
|
||||
meta.ChangeCollectionStateFunc = func(ctx context.Context, collectionID UniqueID, state etcdpb.CollectionState, ts Timestamp) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
dc := mocksutil.NewMockDataCoordClient(t)
|
||||
dc.EXPECT().Flush(mock.Anything, mock.Anything).Return(&datapb.FlushResponse{Status: merr.Success()}, nil)
|
||||
|
||||
mockCallImportServiceErr := false
|
||||
callImportServiceFn := func(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||
if mockCallImportServiceErr {
|
||||
return &datapb.ImportTaskResponse{
|
||||
Status: merr.Success(),
|
||||
}, errors.New("mock err")
|
||||
}
|
||||
return &datapb.ImportTaskResponse{
|
||||
Status: merr.Success(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
callGetSegmentStates := func(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
|
||||
return &datapb.GetSegmentStatesResponse{
|
||||
Status: merr.Success(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
callUnsetIsImportingState := func(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
t.Run("not healthy", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withAbnormalCode())
|
||||
resp, err := c.ReportImport(ctx, &rootcoordpb.ImportResult{})
|
||||
assert.NoError(t, err)
|
||||
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
|
||||
})
|
||||
|
||||
t.Run("report complete import with task not found", func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(withHealthyCode())
|
||||
c.importManager = newImportManager(ctx, mockKv, idAlloc, callImportServiceFn, callGetSegmentStates, nil, nil)
|
||||
resp, err := c.ReportImport(ctx, &rootcoordpb.ImportResult{
|
||||
TaskId: 101,
|
||||
State: commonpb.ImportState_ImportCompleted,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
|
||||
})
|
||||
|
||||
testFunc := func(state commonpb.ImportState) {
|
||||
ctx := context.Background()
|
||||
c := newTestCore(
|
||||
withHealthyCode(),
|
||||
withValidIDAllocator(),
|
||||
withMeta(meta),
|
||||
withTtSynchronizer(ticker),
|
||||
withDataCoord(dc))
|
||||
c.broker = newServerBroker(c)
|
||||
c.importManager = newImportManager(ctx, mockKv, idAlloc, callImportServiceFn, callGetSegmentStates, nil, callUnsetIsImportingState)
|
||||
c.importManager.loadFromTaskStore(true)
|
||||
c.importManager.sendOutTasks(ctx)
|
||||
|
||||
resp, err := c.ReportImport(ctx, &rootcoordpb.ImportResult{
|
||||
TaskId: 100,
|
||||
State: state,
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
|
||||
// Change the state back.
|
||||
err = c.importManager.setImportTaskState(100, commonpb.ImportState_ImportPending)
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
|
||||
t.Run("report import started state", func(t *testing.T) {
|
||||
testFunc(commonpb.ImportState_ImportStarted)
|
||||
})
|
||||
|
||||
t.Run("report import persisted state", func(t *testing.T) {
|
||||
testFunc(commonpb.ImportState_ImportPersisted)
|
||||
})
|
||||
|
||||
t.Run("report import completed state", func(t *testing.T) {
|
||||
testFunc(commonpb.ImportState_ImportCompleted)
|
||||
})
|
||||
|
||||
t.Run("report import failed state", func(t *testing.T) {
|
||||
testFunc(commonpb.ImportState_ImportFailed)
|
||||
})
|
||||
}
|
||||
|
||||
func TestCore_Rbac(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
c := &Core{
|
||||
|
|
|
@ -51,7 +51,7 @@ type Allocator_GenerateTSO_Call struct {
|
|||
}
|
||||
|
||||
// GenerateTSO is a helper method to define mock.On call
|
||||
// - count uint32
|
||||
// - count uint32
|
||||
func (_e *Allocator_Expecter) GenerateTSO(count interface{}) *Allocator_GenerateTSO_Call {
|
||||
return &Allocator_GenerateTSO_Call{Call: _e.mock.On("GenerateTSO", count)}
|
||||
}
|
||||
|
@ -207,7 +207,7 @@ type Allocator_SetTSO_Call struct {
|
|||
}
|
||||
|
||||
// SetTSO is a helper method to define mock.On call
|
||||
// - _a0 uint64
|
||||
// - _a0 uint64
|
||||
func (_e *Allocator_Expecter) SetTSO(_a0 interface{}) *Allocator_SetTSO_Call {
|
||||
return &Allocator_SetTSO_Call{Call: _e.mock.On("SetTSO", _a0)}
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -1,700 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
// BinlogFile class is a wrapper of storage.BinlogReader, to read binlog file, block by block.
|
||||
// Note: for bulkoad function, we only handle normal insert log and delta log.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
// Typically, an insert log file size is 16MB.
|
||||
type BinlogFile struct {
|
||||
chunkManager storage.ChunkManager // storage interfaces to read binlog files
|
||||
reader *storage.BinlogReader // binlog reader
|
||||
}
|
||||
|
||||
func NewBinlogFile(chunkManager storage.ChunkManager) (*BinlogFile, error) {
|
||||
if chunkManager == nil {
|
||||
log.Warn("Binlog file: chunk manager pointer is nil")
|
||||
return nil, merr.WrapErrImportFailed("chunk manager pointer is nil")
|
||||
}
|
||||
|
||||
binlogFile := &BinlogFile{
|
||||
chunkManager: chunkManager,
|
||||
}
|
||||
|
||||
return binlogFile, nil
|
||||
}
|
||||
|
||||
func (p *BinlogFile) Open(filePath string) error {
|
||||
p.Close()
|
||||
if len(filePath) == 0 {
|
||||
log.Warn("Binlog file: binlog path is empty")
|
||||
return merr.WrapErrImportFailed("binlog path is empty")
|
||||
}
|
||||
|
||||
// TODO add context
|
||||
bytes, err := p.chunkManager.Read(context.TODO(), filePath)
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to open binlog", zap.String("filePath", filePath), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to open binlog %s", filePath))
|
||||
}
|
||||
|
||||
p.reader, err = storage.NewBinlogReader(bytes)
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to initialize binlog reader", zap.String("filePath", filePath), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to initialize binlog reader for binlog %s, error: %v", filePath, err))
|
||||
}
|
||||
|
||||
log.Info("Binlog file: open binlog successfully", zap.String("filePath", filePath))
|
||||
return nil
|
||||
}
|
||||
|
||||
// Close close the reader object, outer caller must call this method in defer
|
||||
func (p *BinlogFile) Close() {
|
||||
if p.reader != nil {
|
||||
p.reader.Close()
|
||||
p.reader = nil
|
||||
}
|
||||
}
|
||||
|
||||
func (p *BinlogFile) DataType() schemapb.DataType {
|
||||
if p.reader == nil {
|
||||
return schemapb.DataType_None
|
||||
}
|
||||
|
||||
return p.reader.PayloadDataType
|
||||
}
|
||||
|
||||
// ReadBool method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadBool() ([]bool, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]bool, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Bool {
|
||||
log.Warn("Binlog file: binlog data type is not bool")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not bool")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetBoolFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read bool data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read bool data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadInt8 method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadInt8() ([]int8, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]int8, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Int8 {
|
||||
log.Warn("Binlog file: binlog data type is not int8")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not int8")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetInt8FromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read int8 data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int8 data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadInt16 method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadInt16() ([]int16, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]int16, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Int16 {
|
||||
log.Warn("Binlog file: binlog data type is not int16")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not int16")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetInt16FromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read int16 data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int16 data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadInt32 method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadInt32() ([]int32, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]int32, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Int32 {
|
||||
log.Warn("Binlog file: binlog data type is not int32")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not int32")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetInt32FromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read int32 data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int32 data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadInt64 method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadInt64() ([]int64, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]int64, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Int64 {
|
||||
log.Warn("Binlog file: binlog data type is not int64")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not int64")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetInt64FromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read int64 data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int64 data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadFloat method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadFloat() ([]float32, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]float32, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Float {
|
||||
log.Warn("Binlog file: binlog data type is not float")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not float")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetFloatFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read float data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadDouble method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadDouble() ([]float64, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]float64, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Double {
|
||||
log.Warn("Binlog file: binlog data type is not double")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not double")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetDoubleFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read double data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read double data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadVarchar method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadVarchar() ([]string, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]string, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
// special case: delete event data type is varchar
|
||||
if event.TypeCode != storage.InsertEventType && event.TypeCode != storage.DeleteEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if (p.DataType() != schemapb.DataType_VarChar) && (p.DataType() != schemapb.DataType_String) {
|
||||
log.Warn("Binlog file: binlog data type is not varchar")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not varchar")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetStringFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read varchar data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read varchar data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadJSON method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadJSON() ([][]byte, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([][]byte, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_JSON {
|
||||
log.Warn("Binlog file: binlog data type is not JSON")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not JSON")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetJSONFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read JSON data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read JSON data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadArray method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
func (p *BinlogFile) ReadArray() ([]*schemapb.ScalarField, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
result := make([]*schemapb.ScalarField, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Array {
|
||||
log.Warn("Binlog file: binlog data type is not Array")
|
||||
return nil, merr.WrapErrImportFailed("binlog data type is not Array")
|
||||
}
|
||||
|
||||
data, err := event.PayloadReaderInterface.GetArrayFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read Array data", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read Array data, error: %v", err))
|
||||
}
|
||||
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// ReadBinaryVector method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
// return vectors data and the dimension
|
||||
func (p *BinlogFile) ReadBinaryVector() ([]byte, int, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
dim := 0
|
||||
result := make([]byte, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_BinaryVector {
|
||||
log.Warn("Binlog file: binlog data type is not binary vector")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog data type is not binary vector")
|
||||
}
|
||||
|
||||
data, dimenson, err := event.PayloadReaderInterface.GetBinaryVectorFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read binary vector data", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to read binary vector data, error: %v", err))
|
||||
}
|
||||
|
||||
dim = dimenson
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, dim, nil
|
||||
}
|
||||
|
||||
func (p *BinlogFile) ReadFloat16Vector() ([]byte, int, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
dim := 0
|
||||
result := make([]byte, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_Float16Vector {
|
||||
log.Warn("Binlog file: binlog data type is not float16 vector")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog data type is not float16 vector")
|
||||
}
|
||||
|
||||
data, dimenson, err := event.PayloadReaderInterface.GetFloat16VectorFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read float16 vector data", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float16 vector data, error: %v", err))
|
||||
}
|
||||
|
||||
dim = dimenson
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, dim, nil
|
||||
}
|
||||
|
||||
func (p *BinlogFile) ReadBFloat16Vector() ([]byte, int, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
dim := 0
|
||||
result := make([]byte, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_BFloat16Vector {
|
||||
log.Warn("Binlog file: binlog data type is not bfloat16 vector")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog data type is not bfloat16 vector")
|
||||
}
|
||||
|
||||
data, dimenson, err := event.PayloadReaderInterface.GetBFloat16VectorFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read float16 vector data", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to read bfloat16 vector data, error: %v", err))
|
||||
}
|
||||
|
||||
dim = dimenson
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, dim, nil
|
||||
}
|
||||
|
||||
// ReadFloatVector method reads all the blocks of a binlog by a data type.
|
||||
// A binlog is designed to support multiple blocks, but so far each binlog always contains only one block.
|
||||
// return vectors data and the dimension
|
||||
func (p *BinlogFile) ReadFloatVector() ([]float32, int, error) {
|
||||
if p.reader == nil {
|
||||
log.Warn("Binlog file: binlog reader not yet initialized")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog reader not yet initialized")
|
||||
}
|
||||
|
||||
dim := 0
|
||||
result := make([]float32, 0)
|
||||
for {
|
||||
event, err := p.reader.NextEventReader()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to iterate events reader", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to iterate events reader, error: %v", err))
|
||||
}
|
||||
|
||||
// end of the file
|
||||
if event == nil {
|
||||
break
|
||||
}
|
||||
|
||||
if event.TypeCode != storage.InsertEventType {
|
||||
log.Warn("Binlog file: binlog file is not insert log")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog file is not insert log")
|
||||
}
|
||||
|
||||
if p.DataType() != schemapb.DataType_FloatVector {
|
||||
log.Warn("Binlog file: binlog data type is not float vector")
|
||||
return nil, 0, merr.WrapErrImportFailed("binlog data type is not float vector")
|
||||
}
|
||||
|
||||
data, dimension, err := event.PayloadReaderInterface.GetFloatVectorFromPayload()
|
||||
if err != nil {
|
||||
log.Warn("Binlog file: failed to read float vector data", zap.Error(err))
|
||||
return nil, 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float vector data, error: %v", err))
|
||||
}
|
||||
|
||||
dim = dimension
|
||||
result = append(result, data...)
|
||||
}
|
||||
|
||||
return result, dim, nil
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -1,281 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"path"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
type BinlogParser struct {
|
||||
ctx context.Context // for canceling parse process
|
||||
collectionInfo *CollectionInfo // collection details including schema
|
||||
shardNum int32 // sharding number of the collection
|
||||
blockSize int64 // maximum size of a read block(unit:byte)
|
||||
chunkManager storage.ChunkManager // storage interfaces to browse/read the files
|
||||
callFlushFunc ImportFlushFunc // call back function to flush segment
|
||||
updateProgressFunc func(percent int64) // update working progress percent value
|
||||
|
||||
// a timestamp to define the start time point of restore, data before this time point will be ignored
|
||||
// set this value to 0, all the data will be imported
|
||||
// set this value to math.MaxUint64, all the data will be ignored
|
||||
// the tsStartPoint value must be less/equal than tsEndPoint
|
||||
tsStartPoint uint64
|
||||
|
||||
// a timestamp to define the end time point of restore, data after this time point will be ignored
|
||||
// set this value to 0, all the data will be ignored
|
||||
// set this value to math.MaxUint64, all the data will be imported
|
||||
// the tsEndPoint value must be larger/equal than tsStartPoint
|
||||
tsEndPoint uint64
|
||||
}
|
||||
|
||||
func NewBinlogParser(ctx context.Context,
|
||||
collectionInfo *CollectionInfo,
|
||||
blockSize int64,
|
||||
chunkManager storage.ChunkManager,
|
||||
flushFunc ImportFlushFunc,
|
||||
updateProgressFunc func(percent int64),
|
||||
tsStartPoint uint64,
|
||||
tsEndPoint uint64,
|
||||
) (*BinlogParser, error) {
|
||||
if collectionInfo == nil {
|
||||
log.Warn("Binlog parser: collection schema is nil")
|
||||
return nil, merr.WrapErrImportFailed("collection schema is nil")
|
||||
}
|
||||
|
||||
if chunkManager == nil {
|
||||
log.Warn("Binlog parser: chunk manager pointer is nil")
|
||||
return nil, merr.WrapErrImportFailed("chunk manager pointer is nil")
|
||||
}
|
||||
|
||||
if flushFunc == nil {
|
||||
log.Warn("Binlog parser: flush function is nil")
|
||||
return nil, merr.WrapErrImportFailed("flush function is nil")
|
||||
}
|
||||
|
||||
if tsStartPoint > tsEndPoint {
|
||||
log.Warn("Binlog parser: the tsStartPoint should be less than tsEndPoint",
|
||||
zap.Uint64("tsStartPoint", tsStartPoint), zap.Uint64("tsEndPoint", tsEndPoint))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("Binlog parser: the tsStartPoint %d should be less than tsEndPoint %d", tsStartPoint, tsEndPoint))
|
||||
}
|
||||
|
||||
v := &BinlogParser{
|
||||
ctx: ctx,
|
||||
collectionInfo: collectionInfo,
|
||||
blockSize: blockSize,
|
||||
chunkManager: chunkManager,
|
||||
callFlushFunc: flushFunc,
|
||||
updateProgressFunc: updateProgressFunc,
|
||||
tsStartPoint: tsStartPoint,
|
||||
tsEndPoint: tsEndPoint,
|
||||
}
|
||||
|
||||
return v, nil
|
||||
}
|
||||
|
||||
// constructSegmentHolders builds a list of SegmentFilesHolder, each SegmentFilesHolder represents a segment folder
|
||||
// For instance, the insertlogRoot is "backup/bak1/data/insert_log/435978159196147009/435978159196147010".
|
||||
// 435978159196147009 is a collection id, 435978159196147010 is a partition id,
|
||||
// there is a segment(id is 435978159261483009) under this partition.
|
||||
// ListWithPrefix() will return all the insert logs under this partition:
|
||||
//
|
||||
// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/0/435978159903735811"
|
||||
// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/1/435978159903735812"
|
||||
// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/100/435978159903735809"
|
||||
// "backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/101/435978159903735810"
|
||||
//
|
||||
// The deltalogRoot is "backup/bak1/data/delta_log/435978159196147009/435978159196147010".
|
||||
// Then we get all the delta logs under this partition:
|
||||
//
|
||||
// "backup/bak1/data/delta_log/435978159196147009/435978159196147010/435978159261483009/434574382554415105"
|
||||
//
|
||||
// In this function, we will constuct a list of SegmentFilesHolder objects, each SegmentFilesHolder holds the
|
||||
// insert logs and delta logs of a segment.
|
||||
func (p *BinlogParser) constructSegmentHolders(insertlogRoot string, deltalogRoot string) ([]*SegmentFilesHolder, error) {
|
||||
holders := make(map[int64]*SegmentFilesHolder)
|
||||
// TODO add context
|
||||
insertlogs, _, err := p.chunkManager.ListWithPrefix(context.TODO(), insertlogRoot, true)
|
||||
if err != nil {
|
||||
log.Warn("Binlog parser: list insert logs error", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to list insert logs with root path %s, error: %v", insertlogRoot, err))
|
||||
}
|
||||
|
||||
// collect insert log paths
|
||||
log.Info("Binlog parser: list insert logs", zap.Int("logsCount", len(insertlogs)))
|
||||
for _, insertlog := range insertlogs {
|
||||
log.Info("Binlog parser: mapping insert log to segment", zap.String("insertlog", insertlog))
|
||||
filePath := path.Base(insertlog)
|
||||
// skip file with prefix '.', such as .success .DS_Store
|
||||
if strings.HasPrefix(filePath, ".") {
|
||||
log.Debug("file path might not be a real bin log", zap.String("filePath", filePath))
|
||||
continue
|
||||
}
|
||||
fieldPath := path.Dir(insertlog)
|
||||
fieldStrID := path.Base(fieldPath)
|
||||
fieldID, err := strconv.ParseInt(fieldStrID, 10, 64)
|
||||
if err != nil {
|
||||
log.Warn("Binlog parser: failed to parse field id", zap.String("fieldPath", fieldPath), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse field id from insert log path %s, error: %v", insertlog, err))
|
||||
}
|
||||
|
||||
segmentPath := path.Dir(fieldPath)
|
||||
segmentStrID := path.Base(segmentPath)
|
||||
segmentID, err := strconv.ParseInt(segmentStrID, 10, 64)
|
||||
if err != nil {
|
||||
log.Warn("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse segment id from insert log path %s, error: %v", insertlog, err))
|
||||
}
|
||||
|
||||
holder, ok := holders[segmentID]
|
||||
if ok {
|
||||
holder.fieldFiles[fieldID] = append(holder.fieldFiles[fieldID], insertlog)
|
||||
} else {
|
||||
holder = &SegmentFilesHolder{
|
||||
segmentID: segmentID,
|
||||
fieldFiles: make(map[int64][]string),
|
||||
deltaFiles: make([]string, 0),
|
||||
}
|
||||
holder.fieldFiles[fieldID] = make([]string, 0)
|
||||
holder.fieldFiles[fieldID] = append(holder.fieldFiles[fieldID], insertlog)
|
||||
holders[segmentID] = holder
|
||||
}
|
||||
}
|
||||
|
||||
// sort the insert log paths of each field by ascendent sequence
|
||||
// there might be several insert logs under a field, for example:
|
||||
// 2 insert logs under field a: a_1, a_2
|
||||
// 2 insert logs under field b: b_1, b_2
|
||||
// the row count of a_1 is equal to b_1, the row count of a_2 is equal to b_2
|
||||
// when we read these logs, we firstly read a_1 and b_1, then read a_2 and b_2
|
||||
// so, here we must ensure the paths are arranged correctly
|
||||
segmentIDs := make([]int64, 0)
|
||||
for id, holder := range holders {
|
||||
segmentIDs = append(segmentIDs, id)
|
||||
for _, v := range holder.fieldFiles {
|
||||
sort.Strings(v)
|
||||
}
|
||||
}
|
||||
|
||||
// collect delta log paths
|
||||
if len(deltalogRoot) > 0 {
|
||||
// TODO add context
|
||||
deltalogs, _, err := p.chunkManager.ListWithPrefix(context.TODO(), deltalogRoot, true)
|
||||
if err != nil {
|
||||
log.Warn("Binlog parser: failed to list delta logs", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to list delta logs, error: %v", err))
|
||||
}
|
||||
|
||||
log.Info("Binlog parser: list delta logs", zap.Int("logsCount", len(deltalogs)))
|
||||
for _, deltalog := range deltalogs {
|
||||
log.Info("Binlog parser: mapping delta log to segment", zap.String("deltalog", deltalog))
|
||||
segmentPath := path.Dir(deltalog)
|
||||
segmentStrID := path.Base(segmentPath)
|
||||
segmentID, err := strconv.ParseInt(segmentStrID, 10, 64)
|
||||
if err != nil {
|
||||
log.Warn("Binlog parser: failed to parse segment id", zap.String("segmentPath", segmentPath), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse segment id from delta log path %s, error: %v", deltalog, err))
|
||||
}
|
||||
|
||||
// if the segment id doesn't exist, no need to process this deltalog
|
||||
holder, ok := holders[segmentID]
|
||||
if ok {
|
||||
holder.deltaFiles = append(holder.deltaFiles, deltalog)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// since the map in golang is not sorted, we sort the segment id array to return holder list with ascending sequence
|
||||
sort.Slice(segmentIDs, func(i, j int) bool { return segmentIDs[i] < segmentIDs[j] })
|
||||
holdersList := make([]*SegmentFilesHolder, 0)
|
||||
for _, id := range segmentIDs {
|
||||
holdersList = append(holdersList, holders[id])
|
||||
}
|
||||
|
||||
return holdersList, nil
|
||||
}
|
||||
|
||||
func (p *BinlogParser) parseSegmentFiles(segmentHolder *SegmentFilesHolder) error {
|
||||
if segmentHolder == nil {
|
||||
log.Warn("Binlog parser: segment files holder is nil")
|
||||
return merr.WrapErrImportFailed("segment files holder is nil")
|
||||
}
|
||||
|
||||
adapter, err := NewBinlogAdapter(p.ctx, p.collectionInfo, p.blockSize,
|
||||
Params.DataNodeCfg.BulkInsertMaxMemorySize.GetAsInt64(), p.chunkManager, p.callFlushFunc, p.tsStartPoint, p.tsEndPoint)
|
||||
if err != nil {
|
||||
log.Warn("Binlog parser: failed to create binlog adapter", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to create binlog adapter, error: %v", err))
|
||||
}
|
||||
|
||||
return adapter.Read(segmentHolder)
|
||||
}
|
||||
|
||||
// Parse requires two paths:
|
||||
// 1. the insert log path of a partition
|
||||
// 2. the delta log path of a partiion (optional)
|
||||
func (p *BinlogParser) Parse(filePaths []string) error {
|
||||
if len(filePaths) != 1 && len(filePaths) != 2 {
|
||||
log.Warn("Binlog parser: illegal paths for binlog import, partition binlog path and delta path are required")
|
||||
return merr.WrapErrImportFailed("illegal paths for binlog import, partition binlog path and delta path are required")
|
||||
}
|
||||
|
||||
insertlogPath := filePaths[0]
|
||||
deltalogPath := ""
|
||||
if len(filePaths) == 2 {
|
||||
deltalogPath = filePaths[1]
|
||||
}
|
||||
log.Info("Binlog parser: target paths",
|
||||
zap.String("insertlogPath", insertlogPath),
|
||||
zap.String("deltalogPath", deltalogPath))
|
||||
|
||||
segmentHolders, err := p.constructSegmentHolders(insertlogPath, deltalogPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
updateProgress := func(readBatch int) {
|
||||
if p.updateProgressFunc != nil && len(segmentHolders) != 0 {
|
||||
percent := (readBatch * ProgressValueForPersist) / len(segmentHolders)
|
||||
log.Debug("Binlog parser: working progress", zap.Int("readBatch", readBatch),
|
||||
zap.Int("totalBatchCount", len(segmentHolders)), zap.Int("percent", percent))
|
||||
p.updateProgressFunc(int64(percent))
|
||||
}
|
||||
}
|
||||
|
||||
for i, segmentHolder := range segmentHolders {
|
||||
err = p.parseSegmentFiles(segmentHolder)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
updateProgress(i + 1)
|
||||
|
||||
// trigger gb after each segment finished
|
||||
triggerGC()
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -1,411 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math"
|
||||
"path"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
)
|
||||
|
||||
func Test_BinlogParserNew(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
// nil schema
|
||||
parser, err := NewBinlogParser(ctx, nil, 1024, nil, nil, nil, 0, math.MaxUint64)
|
||||
assert.Nil(t, parser)
|
||||
assert.Error(t, err)
|
||||
|
||||
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nil chunkmanager
|
||||
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, nil, nil, nil, 0, math.MaxUint64)
|
||||
assert.Nil(t, parser)
|
||||
assert.Error(t, err)
|
||||
|
||||
// nil flushfunc
|
||||
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, nil, nil, 0, math.MaxUint64)
|
||||
assert.Nil(t, parser)
|
||||
assert.Error(t, err)
|
||||
|
||||
// succeed
|
||||
flushFunc := func(fields BlockData, shardID int, partID int64) error {
|
||||
return nil
|
||||
}
|
||||
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, flushFunc, nil, 0, math.MaxUint64)
|
||||
assert.NotNil(t, parser)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// tsStartPoint larger than tsEndPoint
|
||||
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, flushFunc, nil, 2, 1)
|
||||
assert.Nil(t, parser)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func Test_BinlogParserConstructHolders(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
flushFunc := func(fields BlockData, shardID int, partID int64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
chunkManager := &MockChunkManager{
|
||||
listResult: make(map[string][]string),
|
||||
}
|
||||
|
||||
insertPath := "insertPath"
|
||||
deltaPath := "deltaPath"
|
||||
|
||||
// the first segment has 12 fields, each field has 2 binlog files
|
||||
seg1Files := []string{
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/0/435978159903735800",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/1/435978159903735801",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/102/435978159903735802",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/103/435978159903735803",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/104/435978159903735804",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/105/435978159903735805",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/106/435978159903735806",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/107/435978159903735807",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/108/435978159903735808",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/109/435978159903735809",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/110/435978159903735810",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/111/435978159903735811",
|
||||
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/0/425978159903735800",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/1/425978159903735801",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/102/425978159903735802",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/103/425978159903735803",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/104/425978159903735804",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/105/425978159903735805",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/106/425978159903735806",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/107/425978159903735807",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/108/425978159903735808",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/109/425978159903735809",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/110/425978159903735810",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/111/425978159903735811",
|
||||
}
|
||||
|
||||
// the second segment has 12 fields, each field has 1 binlog file
|
||||
seg2Files := []string{
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/0/435978159903735811",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/1/435978159903735812",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/102/435978159903735802",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/103/435978159903735803",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/104/435978159903735804",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/105/435978159903735805",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/106/435978159903735806",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/107/435978159903735807",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/108/435978159903735808",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/109/435978159903735809",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/110/435978159903735810",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483009/111/435978159903735811",
|
||||
}
|
||||
|
||||
chunkManager.listResult[insertPath] = append(chunkManager.listResult[insertPath], seg1Files...)
|
||||
chunkManager.listResult[insertPath] = append(chunkManager.listResult[insertPath], seg2Files...)
|
||||
|
||||
// the segment has a delta log file
|
||||
chunkManager.listResult[deltaPath] = []string{
|
||||
"backup/bak1/data/delta_log/435978159196147009/435978159196147010/435978159261483009/434574382554415105",
|
||||
}
|
||||
|
||||
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
|
||||
assert.NotNil(t, parser)
|
||||
assert.NoError(t, err)
|
||||
|
||||
holders, err := parser.constructSegmentHolders(insertPath, deltaPath)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 2, len(holders))
|
||||
|
||||
// verify the first segment
|
||||
holder := holders[0]
|
||||
assert.Equal(t, int64(435978159261483008), holder.segmentID)
|
||||
assert.Equal(t, 12, len(holder.fieldFiles))
|
||||
for i := 0; i < 12; i++ {
|
||||
fieldPath := path.Dir(seg1Files[i])
|
||||
fieldStrID := path.Base(fieldPath)
|
||||
fieldID, _ := strconv.ParseInt(fieldStrID, 10, 64)
|
||||
logFiles, ok := holder.fieldFiles[fieldID]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, 2, len(logFiles))
|
||||
|
||||
// verify logs under each field is sorted
|
||||
log1 := logFiles[0]
|
||||
logID1 := path.Base(log1)
|
||||
ID1, _ := strconv.ParseInt(logID1, 10, 64)
|
||||
log2 := logFiles[1]
|
||||
logID2 := path.Base(log2)
|
||||
ID2, _ := strconv.ParseInt(logID2, 10, 64)
|
||||
assert.LessOrEqual(t, ID1, ID2)
|
||||
}
|
||||
assert.Equal(t, 0, len(holder.deltaFiles))
|
||||
|
||||
// verify the second segment
|
||||
holder = holders[1]
|
||||
assert.Equal(t, int64(435978159261483009), holder.segmentID)
|
||||
assert.Equal(t, len(seg2Files), len(holder.fieldFiles))
|
||||
for i := 0; i < len(seg2Files); i++ {
|
||||
fieldPath := path.Dir(seg2Files[i])
|
||||
fieldStrID := path.Base(fieldPath)
|
||||
fieldID, _ := strconv.ParseInt(fieldStrID, 10, 64)
|
||||
logFiles, ok := holder.fieldFiles[fieldID]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, 1, len(logFiles))
|
||||
assert.Equal(t, seg2Files[i], logFiles[0])
|
||||
}
|
||||
assert.Equal(t, 1, len(holder.deltaFiles))
|
||||
assert.Equal(t, chunkManager.listResult[deltaPath][0], holder.deltaFiles[0])
|
||||
}
|
||||
|
||||
func Test_BinlogParserConstructHoldersFailed(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
flushFunc := func(fields BlockData, shardID int, partID int64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
chunkManager := &MockChunkManager{
|
||||
listErr: errors.New("error"),
|
||||
listResult: make(map[string][]string),
|
||||
}
|
||||
|
||||
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
|
||||
assert.NotNil(t, parser)
|
||||
assert.NoError(t, err)
|
||||
|
||||
insertPath := "insertPath"
|
||||
deltaPath := "deltaPath"
|
||||
|
||||
// chunkManager return error
|
||||
holders, err := parser.constructSegmentHolders(insertPath, deltaPath)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, holders)
|
||||
|
||||
// parse field id error(insert log)
|
||||
chunkManager.listErr = nil
|
||||
chunkManager.listResult[insertPath] = []string{
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/illegal/435978159903735811",
|
||||
}
|
||||
holders, err = parser.constructSegmentHolders(insertPath, deltaPath)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, holders)
|
||||
|
||||
// parse segment id error(insert log)
|
||||
chunkManager.listResult[insertPath] = []string{
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/illegal/0/435978159903735811",
|
||||
}
|
||||
holders, err = parser.constructSegmentHolders(insertPath, deltaPath)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, holders)
|
||||
|
||||
// parse segment id error(delta log)
|
||||
chunkManager.listResult[insertPath] = []string{}
|
||||
chunkManager.listResult[deltaPath] = []string{
|
||||
"backup/bak1/data/delta_log/435978159196147009/435978159196147010/illegal/434574382554415105",
|
||||
}
|
||||
holders, err = parser.constructSegmentHolders(insertPath, deltaPath)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, holders)
|
||||
}
|
||||
|
||||
func Test_BinlogParserParseFilesFailed(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
flushFunc := func(fields BlockData, shardID int, partID int64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, &MockChunkManager{}, flushFunc, nil, 0, math.MaxUint64)
|
||||
assert.NotNil(t, parser)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = parser.parseSegmentFiles(nil)
|
||||
assert.Error(t, err)
|
||||
|
||||
parser.collectionInfo = nil
|
||||
err = parser.parseSegmentFiles(&SegmentFilesHolder{})
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func Test_BinlogParserParse(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
flushFunc := func(fields BlockData, shardID int, partID int64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
chunkManager := &MockChunkManager{}
|
||||
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Description: "schema",
|
||||
AutoID: true,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "id",
|
||||
IsPrimaryKey: true,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
},
|
||||
}
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
|
||||
assert.NotNil(t, parser)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// zero paths
|
||||
err = parser.Parse(nil)
|
||||
assert.Error(t, err)
|
||||
|
||||
// one empty path
|
||||
paths := []string{
|
||||
"insertPath",
|
||||
}
|
||||
err = parser.Parse(paths)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// two empty paths
|
||||
paths = append(paths, "deltaPath")
|
||||
err = parser.Parse(paths)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// wrong path
|
||||
chunkManager.listResult = make(map[string][]string)
|
||||
chunkManager.listResult["insertPath"] = []string{
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/illegal/101/435978159903735811",
|
||||
}
|
||||
err = parser.Parse(paths)
|
||||
assert.Error(t, err)
|
||||
|
||||
// file not found
|
||||
chunkManager.listResult["insertPath"] = []string{
|
||||
"123/0/a",
|
||||
"123/1/a",
|
||||
"123/101/a",
|
||||
}
|
||||
err = parser.Parse(paths)
|
||||
assert.Error(t, err)
|
||||
|
||||
// progress
|
||||
rowCount := 100
|
||||
fieldsData := createFieldsData(sampleSchema(), rowCount, baseTimestamp)
|
||||
chunkManager.listResult["deltaPath"] = []string{}
|
||||
chunkManager.listResult["insertPath"] = []string{
|
||||
"123/0/a",
|
||||
"123/1/a",
|
||||
"123/102/a",
|
||||
"123/103/a",
|
||||
"123/104/a",
|
||||
"123/105/a",
|
||||
"123/106/a",
|
||||
"123/107/a",
|
||||
"123/108/a",
|
||||
"123/109/a",
|
||||
"123/110/a",
|
||||
"123/111/a",
|
||||
"123/112/a",
|
||||
"123/113/a",
|
||||
}
|
||||
chunkManager.readBuf = map[string][]byte{
|
||||
"123/0/a": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)),
|
||||
"123/1/a": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)),
|
||||
"123/102/a": createBinlogBuf(t, schemapb.DataType_Bool, fieldsData[102].([]bool)),
|
||||
"123/103/a": createBinlogBuf(t, schemapb.DataType_Int8, fieldsData[103].([]int8)),
|
||||
"123/104/a": createBinlogBuf(t, schemapb.DataType_Int16, fieldsData[104].([]int16)),
|
||||
"123/105/a": createBinlogBuf(t, schemapb.DataType_Int32, fieldsData[105].([]int32)),
|
||||
"123/106/a": createBinlogBuf(t, schemapb.DataType_Int64, fieldsData[106].([]int64)), // this is primary key
|
||||
"123/107/a": createBinlogBuf(t, schemapb.DataType_Float, fieldsData[107].([]float32)),
|
||||
"123/108/a": createBinlogBuf(t, schemapb.DataType_Double, fieldsData[108].([]float64)),
|
||||
"123/109/a": createBinlogBuf(t, schemapb.DataType_VarChar, fieldsData[109].([]string)),
|
||||
"123/110/a": createBinlogBuf(t, schemapb.DataType_BinaryVector, fieldsData[110].([][]byte)),
|
||||
"123/111/a": createBinlogBuf(t, schemapb.DataType_FloatVector, fieldsData[111].([][]float32)),
|
||||
"123/112/a": createBinlogBuf(t, schemapb.DataType_JSON, fieldsData[112].([][]byte)),
|
||||
"123/113/a": createBinlogBuf(t, schemapb.DataType_Array, fieldsData[113].([]*schemapb.ScalarField)),
|
||||
}
|
||||
|
||||
callTime := 0
|
||||
updateProgress := func(percent int64) {
|
||||
assert.GreaterOrEqual(t, percent, int64(0))
|
||||
assert.LessOrEqual(t, percent, int64(100))
|
||||
callTime++
|
||||
}
|
||||
collectionInfo, err = NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
parser, err = NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, updateProgress, 0, math.MaxUint64)
|
||||
assert.NotNil(t, parser)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = parser.Parse(paths)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, callTime)
|
||||
}
|
||||
|
||||
func Test_BinlogParserSkipFlagFile(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
flushFunc := func(fields BlockData, shardID int, partID int64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
chunkManager := &MockChunkManager{
|
||||
listErr: errors.New("error"),
|
||||
listResult: make(map[string][]string),
|
||||
}
|
||||
|
||||
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
parser, err := NewBinlogParser(ctx, collectionInfo, 1024, chunkManager, flushFunc, nil, 0, math.MaxUint64)
|
||||
assert.NotNil(t, parser)
|
||||
assert.NoError(t, err)
|
||||
|
||||
insertPath := "insertPath"
|
||||
deltaPath := "deltaPath"
|
||||
|
||||
// chunkManager return error
|
||||
holders, err := parser.constructSegmentHolders(insertPath, deltaPath)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, holders)
|
||||
|
||||
// parse field id error(insert log)
|
||||
chunkManager.listErr = nil
|
||||
chunkManager.listResult[insertPath] = []string{
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/0/435978159903735811",
|
||||
"backup/bak1/data/insert_log/435978159196147009/435978159196147010/435978159261483008/.DS_Store",
|
||||
}
|
||||
_, err = parser.constructSegmentHolders(insertPath, deltaPath)
|
||||
assert.NoError(t, err)
|
||||
}
|
|
@ -1,115 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
type CollectionInfo struct {
|
||||
Schema *schemapb.CollectionSchema
|
||||
ShardNum int32
|
||||
|
||||
PartitionIDs []int64 // target partitions of bulkinsert
|
||||
|
||||
PrimaryKey *schemapb.FieldSchema
|
||||
PartitionKey *schemapb.FieldSchema
|
||||
DynamicField *schemapb.FieldSchema
|
||||
|
||||
Name2FieldID map[string]int64 // this member is for Numpy file name validation and JSON row validation
|
||||
}
|
||||
|
||||
func NewCollectionInfo(collectionSchema *schemapb.CollectionSchema,
|
||||
shardNum int32,
|
||||
partitionIDs []int64,
|
||||
) (*CollectionInfo, error) {
|
||||
if shardNum <= 0 {
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("illegal shard number %d", shardNum))
|
||||
}
|
||||
|
||||
if len(partitionIDs) == 0 {
|
||||
return nil, merr.WrapErrImportFailed("partition list is empty")
|
||||
}
|
||||
|
||||
info := &CollectionInfo{
|
||||
ShardNum: shardNum,
|
||||
PartitionIDs: partitionIDs,
|
||||
}
|
||||
|
||||
err := info.resetSchema(collectionSchema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return info, nil
|
||||
}
|
||||
|
||||
func (c *CollectionInfo) resetSchema(collectionSchema *schemapb.CollectionSchema) error {
|
||||
if collectionSchema == nil {
|
||||
return merr.WrapErrImportFailed("collection schema is null")
|
||||
}
|
||||
|
||||
fields := make([]*schemapb.FieldSchema, 0)
|
||||
name2FieldID := make(map[string]int64)
|
||||
var primaryKey *schemapb.FieldSchema
|
||||
var dynamicField *schemapb.FieldSchema
|
||||
var partitionKey *schemapb.FieldSchema
|
||||
for i := 0; i < len(collectionSchema.Fields); i++ {
|
||||
schema := collectionSchema.Fields[i]
|
||||
// RowIDField and TimeStampField is internal field, no need to parse
|
||||
if schema.GetName() == common.RowIDFieldName || schema.GetName() == common.TimeStampFieldName {
|
||||
continue
|
||||
}
|
||||
fields = append(fields, schema)
|
||||
name2FieldID[schema.GetName()] = schema.GetFieldID()
|
||||
|
||||
if schema.GetIsPrimaryKey() {
|
||||
primaryKey = schema
|
||||
} else if schema.GetIsDynamic() {
|
||||
dynamicField = schema
|
||||
} else if schema.GetIsPartitionKey() {
|
||||
partitionKey = schema
|
||||
}
|
||||
}
|
||||
|
||||
if primaryKey == nil {
|
||||
return merr.WrapErrImportFailed("collection schema has no primary key")
|
||||
}
|
||||
|
||||
if partitionKey == nil && len(c.PartitionIDs) != 1 {
|
||||
return merr.WrapErrImportFailed("only allow one partition when there is no partition key")
|
||||
}
|
||||
|
||||
c.Schema = &schemapb.CollectionSchema{
|
||||
Name: collectionSchema.GetName(),
|
||||
Description: collectionSchema.GetDescription(),
|
||||
AutoID: collectionSchema.GetAutoID(),
|
||||
Fields: fields,
|
||||
EnableDynamicField: collectionSchema.GetEnableDynamicField(),
|
||||
}
|
||||
|
||||
c.PrimaryKey = primaryKey
|
||||
c.DynamicField = dynamicField
|
||||
c.PartitionKey = partitionKey
|
||||
c.Name2FieldID = name2FieldID
|
||||
|
||||
return nil
|
||||
}
|
|
@ -1,119 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
)
|
||||
|
||||
func Test_CollectionInfoNew(t *testing.T) {
|
||||
t.Run("succeed", func(t *testing.T) {
|
||||
info, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, info)
|
||||
assert.Greater(t, len(info.Name2FieldID), 0)
|
||||
assert.Nil(t, info.PartitionKey)
|
||||
assert.Nil(t, info.DynamicField)
|
||||
assert.NotNil(t, info.PrimaryKey)
|
||||
assert.Equal(t, int32(2), info.ShardNum)
|
||||
assert.Equal(t, 1, len(info.PartitionIDs))
|
||||
|
||||
// has partition key, has dynamic field
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Description: "schema",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 0,
|
||||
Name: "RowID",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100,
|
||||
Name: "ID",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: false,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "PartitionKey",
|
||||
IsPartitionKey: true,
|
||||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
{
|
||||
FieldID: 102,
|
||||
Name: "$meta",
|
||||
IsDynamic: true,
|
||||
DataType: schemapb.DataType_JSON,
|
||||
},
|
||||
},
|
||||
}
|
||||
info, err = NewCollectionInfo(schema, 2, []int64{1, 2})
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, info)
|
||||
assert.NotNil(t, info.PrimaryKey)
|
||||
assert.NotNil(t, int64(100), info.PrimaryKey.GetFieldID())
|
||||
assert.False(t, info.PrimaryKey.GetAutoID())
|
||||
assert.NotNil(t, info.DynamicField)
|
||||
assert.Equal(t, int64(102), info.DynamicField.GetFieldID())
|
||||
assert.NotNil(t, info.PartitionKey)
|
||||
assert.Equal(t, int64(101), info.PartitionKey.GetFieldID())
|
||||
})
|
||||
|
||||
t.Run("error cases", func(t *testing.T) {
|
||||
schema := sampleSchema()
|
||||
// shard number is 0
|
||||
info, err := NewCollectionInfo(schema, 0, []int64{1})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, info)
|
||||
|
||||
// partiton ID list is empty
|
||||
info, err = NewCollectionInfo(schema, 2, []int64{})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, info)
|
||||
|
||||
// only allow one partition when there is no partition key
|
||||
info, err = NewCollectionInfo(schema, 2, []int64{1, 2})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, info)
|
||||
|
||||
// collection schema is nil
|
||||
info, err = NewCollectionInfo(nil, 2, []int64{1})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, info)
|
||||
|
||||
// no primary key
|
||||
schema = &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Description: "schema",
|
||||
AutoID: true,
|
||||
Fields: make([]*schemapb.FieldSchema, 0),
|
||||
}
|
||||
info, err = NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, info)
|
||||
|
||||
// partition key is nil
|
||||
info, err = NewCollectionInfo(schema, 2, []int64{1, 2})
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, info)
|
||||
})
|
||||
}
|
|
@ -1,122 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"math"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
)
|
||||
|
||||
// Extra option keys to pass through import API
|
||||
const (
|
||||
Bucket = "bucket" // the source files' minio bucket
|
||||
StartTs = "start_ts" // start timestamp to filter data, only data between StartTs and EndTs will be imported
|
||||
EndTs = "end_ts" // end timestamp to filter data, only data between StartTs and EndTs will be imported
|
||||
OptionFormat = "start_ts: 10-digit physical timestamp, e.g. 1665995420, default 0 \n" +
|
||||
"end_ts: 10-digit physical timestamp, e.g. 1665995420, default math.MaxInt \n"
|
||||
BackupFlag = "backup"
|
||||
)
|
||||
|
||||
type ImportOptions struct {
|
||||
OnlyValidate bool
|
||||
TsStartPoint uint64
|
||||
TsEndPoint uint64
|
||||
IsBackup bool // whether is triggered by backup tool
|
||||
}
|
||||
|
||||
func DefaultImportOptions() ImportOptions {
|
||||
options := ImportOptions{
|
||||
OnlyValidate: false,
|
||||
TsStartPoint: 0,
|
||||
TsEndPoint: math.MaxUint64,
|
||||
}
|
||||
return options
|
||||
}
|
||||
|
||||
// ValidateOptions the options is illegal, return nil if illegal, return error if not.
|
||||
// Illegal options:
|
||||
//
|
||||
// start_ts: 10-digit physical timestamp, e.g. 1665995420
|
||||
// end_ts: 10-digit physical timestamp, e.g. 1665995420
|
||||
func ValidateOptions(options []*commonpb.KeyValuePair) error {
|
||||
optionMap := funcutil.KeyValuePair2Map(options)
|
||||
// StartTs should be int
|
||||
_, ok := optionMap[StartTs]
|
||||
var startTs uint64
|
||||
var endTs uint64 = math.MaxInt64
|
||||
var err error
|
||||
if ok {
|
||||
startTs, err = strconv.ParseUint(optionMap[StartTs], 10, 64)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// EndTs should be int
|
||||
_, ok = optionMap[EndTs]
|
||||
if ok {
|
||||
endTs, err = strconv.ParseUint(optionMap[EndTs], 10, 64)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if startTs > endTs {
|
||||
return merr.WrapErrImportFailed("start_ts shouldn't be larger than end_ts")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ParseTSFromOptions get (start_ts, end_ts, error) from input options.
|
||||
// return value will be composed to milvus system timestamp from physical timestamp
|
||||
func ParseTSFromOptions(options []*commonpb.KeyValuePair) (uint64, uint64, error) {
|
||||
err := ValidateOptions(options)
|
||||
if err != nil {
|
||||
return 0, 0, err
|
||||
}
|
||||
var tsStart uint64
|
||||
var tsEnd uint64
|
||||
importOptions := funcutil.KeyValuePair2Map(options)
|
||||
value, ok := importOptions[StartTs]
|
||||
if ok {
|
||||
pTs, _ := strconv.ParseInt(value, 10, 64)
|
||||
tsStart = tsoutil.ComposeTS(pTs, 0)
|
||||
} else {
|
||||
tsStart = 0
|
||||
}
|
||||
value, ok = importOptions[EndTs]
|
||||
if ok {
|
||||
pTs, _ := strconv.ParseInt(value, 10, 64)
|
||||
tsEnd = tsoutil.ComposeTS(pTs, 0)
|
||||
} else {
|
||||
tsEnd = math.MaxUint64
|
||||
}
|
||||
return tsStart, tsEnd, nil
|
||||
}
|
||||
|
||||
// IsBackup returns if the request is triggered by backup tool
|
||||
func IsBackup(options []*commonpb.KeyValuePair) bool {
|
||||
isBackup, err := funcutil.GetAttrByKeyFromRepeatedKV(BackupFlag, options)
|
||||
if err != nil || strings.ToLower(isBackup) != "true" {
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
|
@ -1,112 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"math"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
)
|
||||
|
||||
func Test_ValidateOptions(t *testing.T) {
|
||||
assert.NoError(t, ValidateOptions([]*commonpb.KeyValuePair{}))
|
||||
assert.NoError(t, ValidateOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "1666007457"},
|
||||
{Key: "end_ts", Value: "1666007459"},
|
||||
}))
|
||||
assert.NoError(t, ValidateOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "0"},
|
||||
{Key: "end_ts", Value: "0"},
|
||||
}))
|
||||
assert.NoError(t, ValidateOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "0"},
|
||||
{Key: "end_ts", Value: "1666007457"},
|
||||
}))
|
||||
assert.Error(t, ValidateOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "-1"},
|
||||
{Key: "end_ts", Value: "-1"},
|
||||
}))
|
||||
assert.Error(t, ValidateOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "2"},
|
||||
{Key: "end_ts", Value: "1"},
|
||||
}))
|
||||
assert.Error(t, ValidateOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "3.14"},
|
||||
{Key: "end_ts", Value: "1666007457"},
|
||||
}))
|
||||
assert.Error(t, ValidateOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "1666007457"},
|
||||
{Key: "end_ts", Value: "3.14"},
|
||||
}))
|
||||
}
|
||||
|
||||
func Test_ParseTSFromOptions(t *testing.T) {
|
||||
var tsStart uint64
|
||||
var tsEnd uint64
|
||||
var err error
|
||||
|
||||
tsStart, tsEnd, err = ParseTSFromOptions([]*commonpb.KeyValuePair{})
|
||||
assert.Equal(t, uint64(0), tsStart)
|
||||
assert.Equal(t, uint64(0), math.MaxUint64-tsEnd)
|
||||
assert.NoError(t, err)
|
||||
|
||||
tsStart, tsEnd, err = ParseTSFromOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "0"},
|
||||
{Key: "end_ts", Value: "0"},
|
||||
})
|
||||
assert.Equal(t, uint64(0), tsStart)
|
||||
assert.Equal(t, uint64(0), tsEnd)
|
||||
assert.NoError(t, err)
|
||||
|
||||
tsStart, tsEnd, err = ParseTSFromOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "0"},
|
||||
{Key: "end_ts", Value: "1666007457"},
|
||||
})
|
||||
assert.Equal(t, uint64(0), tsStart)
|
||||
assert.Equal(t, uint64(436733858807808), tsEnd)
|
||||
assert.NoError(t, err)
|
||||
|
||||
tsStart, tsEnd, err = ParseTSFromOptions([]*commonpb.KeyValuePair{
|
||||
{Key: "start_ts", Value: "2"},
|
||||
{Key: "end_ts", Value: "1"},
|
||||
})
|
||||
assert.Equal(t, uint64(0), tsStart)
|
||||
assert.Equal(t, uint64(0), tsEnd)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func Test_IsBackup(t *testing.T) {
|
||||
isBackup := IsBackup([]*commonpb.KeyValuePair{
|
||||
{Key: "backup", Value: "true"},
|
||||
})
|
||||
assert.Equal(t, true, isBackup)
|
||||
isBackup2 := IsBackup([]*commonpb.KeyValuePair{
|
||||
{Key: "backup", Value: "True"},
|
||||
})
|
||||
assert.Equal(t, true, isBackup2)
|
||||
falseBackup := IsBackup([]*commonpb.KeyValuePair{
|
||||
{Key: "backup", Value: "false"},
|
||||
})
|
||||
assert.Equal(t, false, falseBackup)
|
||||
noBackup := IsBackup([]*commonpb.KeyValuePair{
|
||||
{Key: "backup", Value: "false"},
|
||||
})
|
||||
assert.Equal(t, false, noBackup)
|
||||
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -1,613 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"context"
|
||||
"fmt"
|
||||
"strconv"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||
)
|
||||
|
||||
const (
|
||||
JSONFileExt = ".json"
|
||||
NumpyFileExt = ".npy"
|
||||
ParquetFileExt = ".parquet"
|
||||
|
||||
// progress percent value of persist state
|
||||
ProgressValueForPersist = 90
|
||||
|
||||
// keywords of import task informations
|
||||
FailedReason = "failed_reason"
|
||||
Files = "files"
|
||||
CollectionName = "collection"
|
||||
PartitionName = "partition"
|
||||
PersistTimeCost = "persist_cost"
|
||||
ProgressPercent = "progress_percent"
|
||||
)
|
||||
|
||||
var Params *paramtable.ComponentParam = paramtable.Get()
|
||||
|
||||
// ReportImportAttempts is the maximum # of attempts to retry when import fails.
|
||||
var ReportImportAttempts uint = 10
|
||||
|
||||
type (
|
||||
ImportFlushFunc func(fields BlockData, shardID int, partID int64) error
|
||||
AssignSegmentFunc func(shardID int, partID int64) (int64, string, error)
|
||||
CreateBinlogsFunc func(fields BlockData, segmentID int64, partID int64) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error)
|
||||
SaveSegmentFunc func(fieldsInsert []*datapb.FieldBinlog, fieldsStats []*datapb.FieldBinlog, segmentID int64, targetChName string, rowCount int64, partID int64) error
|
||||
ReportFunc func(res *rootcoordpb.ImportResult) error
|
||||
)
|
||||
|
||||
type WorkingSegment struct {
|
||||
segmentID int64 // segment ID
|
||||
shardID int // shard ID
|
||||
partitionID int64 // partition ID
|
||||
targetChName string // target dml channel
|
||||
rowCount int64 // accumulate row count
|
||||
memSize int // total memory size of all binlogs
|
||||
fieldsInsert []*datapb.FieldBinlog // persisted binlogs
|
||||
fieldsStats []*datapb.FieldBinlog // stats of persisted binlogs
|
||||
}
|
||||
|
||||
type ImportWrapper struct {
|
||||
ctx context.Context // for canceling parse process
|
||||
cancel context.CancelFunc // for canceling parse process
|
||||
collectionInfo *CollectionInfo // collection details including schema
|
||||
segmentSize int64 // maximum size of a segment(unit:byte) defined by dataCoord.segment.maxSize (milvus.yml)
|
||||
binlogSize int64 // average binlog size(unit:byte), the max biglog file size is no more than 2*binlogSize
|
||||
rowIDAllocator *allocator.IDAllocator // autoid allocator
|
||||
chunkManager storage.ChunkManager
|
||||
|
||||
assignSegmentFunc AssignSegmentFunc // function to prepare a new segment
|
||||
createBinlogsFunc CreateBinlogsFunc // function to create binlog for a segment
|
||||
saveSegmentFunc SaveSegmentFunc // function to persist a segment
|
||||
|
||||
importResult *rootcoordpb.ImportResult // import result
|
||||
reportFunc ReportFunc // report import state to rootcoord
|
||||
reportImportAttempts uint // attempts count if report function get error
|
||||
|
||||
workingSegments map[int]map[int64]*WorkingSegment // two-level map shard id and partition id to working segments
|
||||
progressPercent int64 // working progress percent
|
||||
}
|
||||
|
||||
func NewImportWrapper(ctx context.Context, collectionInfo *CollectionInfo, segmentSize int64, maxBinlogSize int64,
|
||||
idAlloc *allocator.IDAllocator, cm storage.ChunkManager, importResult *rootcoordpb.ImportResult,
|
||||
reportFunc func(res *rootcoordpb.ImportResult) error,
|
||||
) *ImportWrapper {
|
||||
if collectionInfo == nil || collectionInfo.Schema == nil {
|
||||
log.Warn("import wrapper: collection schema is nil")
|
||||
return nil
|
||||
}
|
||||
log.Info("import wrapper: collection info", zap.Int32("ShardNum", collectionInfo.ShardNum),
|
||||
zap.Int("PartitionsNum", len(collectionInfo.PartitionIDs)), zap.Any("Fields", collectionInfo.Name2FieldID))
|
||||
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
|
||||
// average binlogSize is expected to be half of the maxBinlogSize
|
||||
// and avoid binlogSize to be a tiny value
|
||||
binlogSize := int64(float32(maxBinlogSize) * 0.5)
|
||||
if binlogSize < Params.DataNodeCfg.BulkInsertReadBufferSize.GetAsInt64() {
|
||||
binlogSize = Params.DataNodeCfg.BulkInsertReadBufferSize.GetAsInt64()
|
||||
}
|
||||
|
||||
wrapper := &ImportWrapper{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
collectionInfo: collectionInfo,
|
||||
segmentSize: segmentSize,
|
||||
binlogSize: binlogSize,
|
||||
rowIDAllocator: idAlloc,
|
||||
chunkManager: cm,
|
||||
importResult: importResult,
|
||||
reportFunc: reportFunc,
|
||||
reportImportAttempts: ReportImportAttempts,
|
||||
workingSegments: make(map[int]map[int64]*WorkingSegment),
|
||||
}
|
||||
|
||||
return wrapper
|
||||
}
|
||||
|
||||
func (p *ImportWrapper) SetCallbackFunctions(assignSegmentFunc AssignSegmentFunc, createBinlogsFunc CreateBinlogsFunc, saveSegmentFunc SaveSegmentFunc) error {
|
||||
if assignSegmentFunc == nil {
|
||||
log.Warn("import wrapper: callback function AssignSegmentFunc is nil")
|
||||
return merr.WrapErrImportFailed("callback function AssignSegmentFunc is nil")
|
||||
}
|
||||
|
||||
if createBinlogsFunc == nil {
|
||||
log.Warn("import wrapper: callback function CreateBinlogsFunc is nil")
|
||||
return merr.WrapErrImportFailed("callback function CreateBinlogsFunc is nil")
|
||||
}
|
||||
|
||||
if saveSegmentFunc == nil {
|
||||
log.Warn("import wrapper: callback function SaveSegmentFunc is nil")
|
||||
return merr.WrapErrImportFailed("callback function SaveSegmentFunc is nil")
|
||||
}
|
||||
|
||||
p.assignSegmentFunc = assignSegmentFunc
|
||||
p.createBinlogsFunc = createBinlogsFunc
|
||||
p.saveSegmentFunc = saveSegmentFunc
|
||||
return nil
|
||||
}
|
||||
|
||||
// Cancel method can be used to cancel parse process
|
||||
func (p *ImportWrapper) Cancel() error {
|
||||
p.cancel()
|
||||
return nil
|
||||
}
|
||||
|
||||
// fileValidation verify the input paths
|
||||
// if all the files are json type, return true
|
||||
// if all the files are numpy type, return false, and not allow duplicate file name
|
||||
func (p *ImportWrapper) fileValidation(filePaths []string) (bool, error) {
|
||||
// use this map to check duplicate file name(only for numpy file)
|
||||
fileNames := make(map[string]struct{})
|
||||
|
||||
totalSize := int64(0)
|
||||
rowBased := false
|
||||
for i := 0; i < len(filePaths); i++ {
|
||||
filePath := filePaths[i]
|
||||
name, fileType := GetFileNameAndExt(filePath)
|
||||
|
||||
// only allow json file, numpy file and csv file
|
||||
if fileType != JSONFileExt && fileType != NumpyFileExt && fileType != ParquetFileExt {
|
||||
log.Warn("import wrapper: unsupported file type", zap.String("filePath", filePath))
|
||||
return false, merr.WrapErrImportFailed(fmt.Sprintf("unsupported file type: '%s'", filePath))
|
||||
}
|
||||
|
||||
// we use the first file to determine row-based or column-based
|
||||
if i == 0 && fileType == JSONFileExt {
|
||||
rowBased = true
|
||||
}
|
||||
|
||||
// check file type
|
||||
// row-based only support json and csv type, column-based only support numpy type
|
||||
if rowBased {
|
||||
if fileType != JSONFileExt {
|
||||
log.Warn("import wrapper: unsupported file type for row-based mode", zap.String("filePath", filePath))
|
||||
return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("unsupported file type for row-based mode: '%s'", filePath))
|
||||
}
|
||||
} else {
|
||||
if fileType != NumpyFileExt && fileType != ParquetFileExt {
|
||||
log.Warn("import wrapper: unsupported file type for column-based mode", zap.String("filePath", filePath))
|
||||
return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("unsupported file type for column-based mode: '%s'", filePath))
|
||||
}
|
||||
}
|
||||
|
||||
// check dupliate file
|
||||
_, ok := fileNames[name]
|
||||
if ok {
|
||||
log.Warn("import wrapper: duplicate file name", zap.String("filePath", filePath))
|
||||
return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("duplicate file: '%s'", filePath))
|
||||
}
|
||||
fileNames[name] = struct{}{}
|
||||
|
||||
// check file size, single file size cannot exceed MaxFileSize
|
||||
size, err := p.chunkManager.Size(p.ctx, filePath)
|
||||
if err != nil {
|
||||
log.Warn("import wrapper: failed to get file size", zap.String("filePath", filePath), zap.Error(err))
|
||||
return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("failed to get file size of '%s', error:%v", filePath, err))
|
||||
}
|
||||
|
||||
// empty file
|
||||
if size == 0 {
|
||||
log.Warn("import wrapper: file size is zero", zap.String("filePath", filePath))
|
||||
return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("the file '%s' size is zero", filePath))
|
||||
}
|
||||
|
||||
if size > Params.CommonCfg.ImportMaxFileSize.GetAsInt64() {
|
||||
log.Warn("import wrapper: file size exceeds the maximum size", zap.String("filePath", filePath),
|
||||
zap.Int64("fileSize", size), zap.String("MaxFileSize", Params.CommonCfg.ImportMaxFileSize.GetValue()))
|
||||
return rowBased, merr.WrapErrImportFailed(fmt.Sprintf("the file '%s' size exceeds the maximum size: %s bytes",
|
||||
filePath, Params.CommonCfg.ImportMaxFileSize.GetValue()))
|
||||
}
|
||||
totalSize += size
|
||||
}
|
||||
|
||||
return rowBased, nil
|
||||
}
|
||||
|
||||
// Import is the entry of import operation
|
||||
// filePath and rowBased are from ImportTask
|
||||
// if onlyValidate is true, this process only do validation, no data generated, flushFunc will not be called
|
||||
func (p *ImportWrapper) Import(filePaths []string, options ImportOptions) error {
|
||||
log.Info("import wrapper: begin import", zap.Any("filePaths", filePaths), zap.Any("options", options))
|
||||
|
||||
// data restore function to import milvus native binlog files(for backup/restore tools)
|
||||
// the backup/restore tool provide two paths for a partition, the first path is binlog path, the second is deltalog path
|
||||
if options.IsBackup && p.isBinlogImport(filePaths) {
|
||||
return p.doBinlogImport(filePaths, options.TsStartPoint, options.TsEndPoint)
|
||||
}
|
||||
|
||||
// normal logic for import general data files
|
||||
rowBased, err := p.fileValidation(filePaths)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
tr := timerecord.NewTimeRecorder("Import task")
|
||||
if rowBased {
|
||||
// parse and consume row-based files
|
||||
// for row-based files, the JSONRowConsumer will generate autoid for primary key, and split rows into segments
|
||||
// according to shard number, so the flushFunc will be called in the JSONRowConsumer
|
||||
for i := 0; i < len(filePaths); i++ {
|
||||
filePath := filePaths[i]
|
||||
_, fileType := GetFileNameAndExt(filePath)
|
||||
log.Info("import wrapper: row-based file ", zap.Any("filePath", filePath), zap.Any("fileType", fileType))
|
||||
|
||||
if fileType == JSONFileExt {
|
||||
err = p.parseRowBasedJSON(filePath, options.OnlyValidate)
|
||||
if err != nil {
|
||||
log.Warn("import wrapper: failed to parse row-based json file", zap.Error(err), zap.String("filePath", filePath))
|
||||
return err
|
||||
}
|
||||
} // no need to check else, since the fileValidation() already do this
|
||||
|
||||
// trigger gc after each file finished
|
||||
triggerGC()
|
||||
}
|
||||
} else {
|
||||
// parse and consume column-based files(currently support numpy)
|
||||
// for column-based files, the NumpyParser will generate autoid for primary key, and split rows into segments
|
||||
// according to shard number, so the flushFunc will be called in the NumpyParser
|
||||
flushFunc := func(fields BlockData, shardID int, partitionID int64) error {
|
||||
printFieldsDataInfo(fields, "import wrapper: prepare to flush binlog data", filePaths)
|
||||
return p.flushFunc(fields, shardID, partitionID)
|
||||
}
|
||||
_, fileType := GetFileNameAndExt(filePaths[0])
|
||||
if fileType == NumpyFileExt {
|
||||
parser, err := NewNumpyParser(p.ctx, p.collectionInfo, p.rowIDAllocator, p.binlogSize,
|
||||
p.chunkManager, flushFunc, p.updateProgressPercent)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = parser.Parse(filePaths)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
p.importResult.AutoIds = append(p.importResult.AutoIds, parser.IDRange()...)
|
||||
} else if fileType == ParquetFileExt {
|
||||
parser, err := NewParquetParser(p.ctx, p.collectionInfo, p.rowIDAllocator, p.binlogSize,
|
||||
p.chunkManager, filePaths[0], flushFunc, p.updateProgressPercent)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer parser.Close()
|
||||
|
||||
err = parser.Parse()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
p.importResult.AutoIds = append(p.importResult.AutoIds, parser.IDRange()...)
|
||||
}
|
||||
|
||||
// trigger after parse finished
|
||||
triggerGC()
|
||||
}
|
||||
|
||||
return p.reportPersisted(p.reportImportAttempts, tr)
|
||||
}
|
||||
|
||||
// reportPersisted notify the rootcoord to mark the task state to be ImportPersisted
|
||||
func (p *ImportWrapper) reportPersisted(reportAttempts uint, tr *timerecord.TimeRecorder) error {
|
||||
// force close all segments
|
||||
err := p.closeAllWorkingSegments()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if tr != nil {
|
||||
ts := tr.Elapse("persist finished").Seconds()
|
||||
p.importResult.Infos = append(p.importResult.Infos,
|
||||
&commonpb.KeyValuePair{Key: PersistTimeCost, Value: strconv.FormatFloat(ts, 'f', 2, 64)})
|
||||
}
|
||||
|
||||
// report file process state
|
||||
p.importResult.State = commonpb.ImportState_ImportPersisted
|
||||
progressValue := strconv.Itoa(ProgressValueForPersist)
|
||||
UpdateKVInfo(&p.importResult.Infos, ProgressPercent, progressValue)
|
||||
|
||||
log.Info("import wrapper: report import result", zap.Any("importResult", p.importResult))
|
||||
// persist state task is valuable, retry more times in case fail this task only because of network error
|
||||
reportErr := retry.Do(p.ctx, func() error {
|
||||
return p.reportFunc(p.importResult)
|
||||
}, retry.Attempts(reportAttempts))
|
||||
if reportErr != nil {
|
||||
log.Warn("import wrapper: fail to report import state to RootCoord", zap.Error(reportErr))
|
||||
return reportErr
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// isBinlogImport is to judge whether it is binlog import operation
|
||||
// For internal usage by the restore tool: https://github.com/zilliztech/milvus-backup
|
||||
// This tool exports data from a milvus service, and call bulkload interface to import native data into another milvus service.
|
||||
// This tool provides two paths: one is insert log path of a partition,the other is delta log path of this partition.
|
||||
// This method checks the filePaths, if the file paths is exist and not a file, we say it is native import.
|
||||
func (p *ImportWrapper) isBinlogImport(filePaths []string) bool {
|
||||
// must contains the insert log path, and the delta log path is optional to be empty string
|
||||
if len(filePaths) != 2 {
|
||||
log.Info("import wrapper: paths count is not 2, not binlog import", zap.Int("len", len(filePaths)))
|
||||
return false
|
||||
}
|
||||
|
||||
checkFunc := func(filePath string) bool {
|
||||
// contains file extension, is not a path
|
||||
_, fileType := GetFileNameAndExt(filePath)
|
||||
if len(fileType) != 0 {
|
||||
log.Info("import wrapper: not a path, not binlog import", zap.String("filePath", filePath), zap.String("fileType", fileType))
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// the first path is insert log path
|
||||
filePath := filePaths[0]
|
||||
if len(filePath) == 0 {
|
||||
log.Info("import wrapper: the first path is empty string, not binlog import")
|
||||
return false
|
||||
}
|
||||
|
||||
if !checkFunc(filePath) {
|
||||
return false
|
||||
}
|
||||
|
||||
// the second path is delta log path
|
||||
filePath = filePaths[1]
|
||||
if len(filePath) > 0 && !checkFunc(filePath) {
|
||||
return false
|
||||
}
|
||||
|
||||
log.Info("import wrapper: do binlog import")
|
||||
return true
|
||||
}
|
||||
|
||||
// doBinlogImport is the entry of binlog import operation
|
||||
func (p *ImportWrapper) doBinlogImport(filePaths []string, tsStartPoint uint64, tsEndPoint uint64) error {
|
||||
tr := timerecord.NewTimeRecorder("Import task")
|
||||
|
||||
flushFunc := func(fields BlockData, shardID int, partitionID int64) error {
|
||||
printFieldsDataInfo(fields, "import wrapper: prepare to flush binlog data", filePaths)
|
||||
return p.flushFunc(fields, shardID, partitionID)
|
||||
}
|
||||
parser, err := NewBinlogParser(p.ctx, p.collectionInfo, p.binlogSize,
|
||||
p.chunkManager, flushFunc, p.updateProgressPercent, tsStartPoint, tsEndPoint)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = parser.Parse(filePaths)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return p.reportPersisted(p.reportImportAttempts, tr)
|
||||
}
|
||||
|
||||
// parseRowBasedJSON is the entry of row-based json import operation
|
||||
func (p *ImportWrapper) parseRowBasedJSON(filePath string, onlyValidate bool) error {
|
||||
tr := timerecord.NewTimeRecorder("json row-based parser: " + filePath)
|
||||
|
||||
// for minio storage, chunkManager will download file into local memory
|
||||
// for local storage, chunkManager open the file directly
|
||||
file, err := p.chunkManager.Reader(p.ctx, filePath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer file.Close()
|
||||
|
||||
size, err := p.chunkManager.Size(p.ctx, filePath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// parse file
|
||||
reader := bufio.NewReader(file)
|
||||
parser := NewJSONParser(p.ctx, p.collectionInfo, p.updateProgressPercent)
|
||||
|
||||
// if only validate, we input a empty flushFunc so that the consumer do nothing but only validation.
|
||||
var flushFunc ImportFlushFunc
|
||||
if onlyValidate {
|
||||
flushFunc = func(fields BlockData, shardID int, partitionID int64) error {
|
||||
return nil
|
||||
}
|
||||
} else {
|
||||
flushFunc = func(fields BlockData, shardID int, partitionID int64) error {
|
||||
filePaths := []string{filePath}
|
||||
printFieldsDataInfo(fields, "import wrapper: prepare to flush binlogs", filePaths)
|
||||
return p.flushFunc(fields, shardID, partitionID)
|
||||
}
|
||||
}
|
||||
|
||||
consumer, err := NewJSONRowConsumer(p.ctx, p.collectionInfo, p.rowIDAllocator, p.binlogSize, flushFunc)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: size}, consumer)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// for row-based files, auto-id is generated within JSONRowConsumer
|
||||
p.importResult.AutoIds = append(p.importResult.AutoIds, consumer.IDRange()...)
|
||||
|
||||
tr.Elapse("parsed")
|
||||
return nil
|
||||
}
|
||||
|
||||
// flushFunc is the callback function for parsers generate segment and save binlog files
|
||||
func (p *ImportWrapper) flushFunc(fields BlockData, shardID int, partitionID int64) error {
|
||||
logFields := []zap.Field{
|
||||
zap.Int("shardID", shardID),
|
||||
zap.Int64("partitionID", partitionID),
|
||||
}
|
||||
|
||||
// if fields data is empty, do nothing
|
||||
rowNum := 0
|
||||
memSize := 0
|
||||
for _, field := range fields {
|
||||
rowNum = field.RowNum()
|
||||
memSize += field.GetMemorySize()
|
||||
}
|
||||
if rowNum <= 0 {
|
||||
log.Warn("import wrapper: fields data is empty", logFields...)
|
||||
return nil
|
||||
}
|
||||
|
||||
logFields = append(logFields, zap.Int("rowNum", rowNum), zap.Int("memSize", memSize))
|
||||
log.Info("import wrapper: flush block data to binlog", logFields...)
|
||||
|
||||
// if there is no segment for this shard, create a new one
|
||||
// if the segment exists and its size almost exceed segmentSize, close it and create a new one
|
||||
var segment *WorkingSegment
|
||||
if shard, ok := p.workingSegments[shardID]; ok {
|
||||
if segmentTemp, exists := shard[partitionID]; exists {
|
||||
log.Info("import wrapper: compare working segment memSize with segmentSize",
|
||||
zap.Int("memSize", segmentTemp.memSize), zap.Int64("segmentSize", p.segmentSize))
|
||||
if int64(segmentTemp.memSize)+int64(memSize) >= p.segmentSize {
|
||||
// the segment already exists, check its size, if the size exceeds(or almost) segmentSize, close the segment
|
||||
err := p.closeWorkingSegment(segmentTemp)
|
||||
if err != nil {
|
||||
logFields = append(logFields, zap.Error(err))
|
||||
log.Warn("import wrapper: failed to close working segment", logFields...)
|
||||
return err
|
||||
}
|
||||
p.workingSegments[shardID][partitionID] = nil
|
||||
} else {
|
||||
// the exist segment size is small, no need to close
|
||||
segment = segmentTemp
|
||||
}
|
||||
}
|
||||
} else {
|
||||
p.workingSegments[shardID] = make(map[int64]*WorkingSegment)
|
||||
}
|
||||
|
||||
if segment == nil {
|
||||
// create a new segment
|
||||
segID, channelName, err := p.assignSegmentFunc(shardID, partitionID)
|
||||
if err != nil {
|
||||
logFields = append(logFields, zap.Error(err))
|
||||
log.Warn("import wrapper: failed to assign a new segment", logFields...)
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to assign a new segment for shard id %d, error: %v", shardID, err))
|
||||
}
|
||||
|
||||
segment = &WorkingSegment{
|
||||
segmentID: segID,
|
||||
shardID: shardID,
|
||||
partitionID: partitionID,
|
||||
targetChName: channelName,
|
||||
rowCount: int64(0),
|
||||
memSize: 0,
|
||||
fieldsInsert: make([]*datapb.FieldBinlog, 0),
|
||||
fieldsStats: make([]*datapb.FieldBinlog, 0),
|
||||
}
|
||||
p.workingSegments[shardID][partitionID] = segment
|
||||
}
|
||||
|
||||
// save binlogs
|
||||
fieldsInsert, fieldsStats, err := p.createBinlogsFunc(fields, segment.segmentID, partitionID)
|
||||
if err != nil {
|
||||
logFields = append(logFields, zap.Error(err), zap.Int64("segmentID", segment.segmentID),
|
||||
zap.String("targetChannel", segment.targetChName))
|
||||
log.Warn("import wrapper: failed to save binlogs", logFields...)
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to save binlogs, shard id %d, segment id %d, channel '%s', error: %v",
|
||||
shardID, segment.segmentID, segment.targetChName, err))
|
||||
}
|
||||
|
||||
segment.fieldsInsert = append(segment.fieldsInsert, fieldsInsert...)
|
||||
segment.fieldsStats = append(segment.fieldsStats, fieldsStats...)
|
||||
segment.rowCount += int64(rowNum)
|
||||
segment.memSize += memSize
|
||||
|
||||
// report working progress percent value to rootcoord
|
||||
// if failed to report, ignore the error, the percent value might be improper but the task can be succeed
|
||||
progressValue := strconv.Itoa(int(p.progressPercent))
|
||||
UpdateKVInfo(&p.importResult.Infos, ProgressPercent, progressValue)
|
||||
reportErr := retry.Do(p.ctx, func() error {
|
||||
return p.reportFunc(p.importResult)
|
||||
}, retry.Attempts(p.reportImportAttempts))
|
||||
if reportErr != nil {
|
||||
logFields = append(logFields, zap.Error(err))
|
||||
log.Warn("import wrapper: fail to report working progress percent value to RootCoord", logFields...)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// closeWorkingSegment marks a segment to be sealed
|
||||
func (p *ImportWrapper) closeWorkingSegment(segment *WorkingSegment) error {
|
||||
logFields := []zap.Field{
|
||||
zap.Int("shardID", segment.shardID),
|
||||
zap.Int64("segmentID", segment.segmentID),
|
||||
zap.String("targetChannel", segment.targetChName),
|
||||
zap.Int64("rowCount", segment.rowCount),
|
||||
zap.Int("insertLogCount", len(segment.fieldsInsert)),
|
||||
zap.Int("statsLogCount", len(segment.fieldsStats)),
|
||||
}
|
||||
log.Info("import wrapper: adding segment to the correct DataNode flow graph and saving binlog paths", logFields...)
|
||||
|
||||
err := p.saveSegmentFunc(segment.fieldsInsert, segment.fieldsStats, segment.segmentID, segment.targetChName,
|
||||
segment.rowCount, segment.partitionID)
|
||||
if err != nil {
|
||||
logFields = append(logFields, zap.Error(err))
|
||||
log.Warn("import wrapper: failed to seal segment", logFields...)
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to seal segment, shard id %d, segment id %d, channel '%s', error: %v",
|
||||
segment.shardID, segment.segmentID, segment.targetChName, err))
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// closeAllWorkingSegments mark all segments to be sealed at the end of import operation
|
||||
func (p *ImportWrapper) closeAllWorkingSegments() error {
|
||||
for _, shard := range p.workingSegments {
|
||||
for _, segment := range shard {
|
||||
err := p.closeWorkingSegment(segment)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
p.workingSegments = make(map[int]map[int64]*WorkingSegment)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *ImportWrapper) updateProgressPercent(percent int64) {
|
||||
// ignore illegal percent value
|
||||
if percent < 0 || percent > 100 {
|
||||
return
|
||||
}
|
||||
p.progressPercent = percent
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -1,317 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"strconv"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
// JSONRowHandler is the interface to process rows data
|
||||
type JSONRowHandler interface {
|
||||
Handle(rows []map[storage.FieldID]interface{}) error
|
||||
}
|
||||
|
||||
func getKeyValue(obj interface{}, fieldName string, isString bool) (string, error) {
|
||||
// varchar type primary field, the value must be a string
|
||||
if isString {
|
||||
if value, ok := obj.(string); ok {
|
||||
return value, nil
|
||||
}
|
||||
return "", merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for varchar type key field '%s'", obj, fieldName))
|
||||
}
|
||||
|
||||
// int64 type primary field, the value must be json.Number
|
||||
if num, ok := obj.(json.Number); ok {
|
||||
return string(num), nil
|
||||
}
|
||||
return "", merr.WrapErrImportFailed(fmt.Sprintf("illegal value '%v' for int64 type key field '%s'", obj, fieldName))
|
||||
}
|
||||
|
||||
// JSONRowConsumer is row-based json format consumer class
|
||||
type JSONRowConsumer struct {
|
||||
ctx context.Context // for canceling parse process
|
||||
collectionInfo *CollectionInfo // collection details including schema
|
||||
rowIDAllocator *allocator.IDAllocator // autoid allocator
|
||||
validators map[storage.FieldID]*Validator // validators for each field
|
||||
rowCounter int64 // how many rows have been consumed
|
||||
shardsData []ShardData // in-memory shards data
|
||||
blockSize int64 // maximum size of a read block(unit:byte)
|
||||
autoIDRange []int64 // auto-generated id range, for example: [1, 10, 20, 25] means id from 1 to 10 and 20 to 25
|
||||
|
||||
callFlushFunc ImportFlushFunc // call back function to flush segment
|
||||
}
|
||||
|
||||
func NewJSONRowConsumer(ctx context.Context,
|
||||
collectionInfo *CollectionInfo,
|
||||
idAlloc *allocator.IDAllocator,
|
||||
blockSize int64,
|
||||
flushFunc ImportFlushFunc,
|
||||
) (*JSONRowConsumer, error) {
|
||||
if collectionInfo == nil {
|
||||
log.Warn("JSON row consumer: collection schema is nil")
|
||||
return nil, merr.WrapErrImportFailed("collection schema is nil")
|
||||
}
|
||||
|
||||
v := &JSONRowConsumer{
|
||||
ctx: ctx,
|
||||
collectionInfo: collectionInfo,
|
||||
rowIDAllocator: idAlloc,
|
||||
validators: make(map[storage.FieldID]*Validator),
|
||||
blockSize: blockSize,
|
||||
rowCounter: 0,
|
||||
autoIDRange: make([]int64, 0),
|
||||
callFlushFunc: flushFunc,
|
||||
}
|
||||
|
||||
err := initValidators(collectionInfo.Schema, v.validators)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: fail to initialize json row-based consumer", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("fail to initialize json row-based consumer, error: %v", err))
|
||||
}
|
||||
|
||||
v.shardsData = make([]ShardData, 0, collectionInfo.ShardNum)
|
||||
for i := 0; i < int(collectionInfo.ShardNum); i++ {
|
||||
shardData := initShardData(collectionInfo.Schema, collectionInfo.PartitionIDs)
|
||||
if shardData == nil {
|
||||
log.Warn("JSON row consumer: fail to initialize in-memory segment data", zap.Int("shardID", i))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("fail to initialize in-memory segment data for shard id %d", i))
|
||||
}
|
||||
v.shardsData = append(v.shardsData, shardData)
|
||||
}
|
||||
|
||||
// primary key is autoid, id generator is required
|
||||
if v.collectionInfo.PrimaryKey.GetAutoID() && idAlloc == nil {
|
||||
log.Warn("JSON row consumer: ID allocator is nil")
|
||||
return nil, merr.WrapErrImportFailed("ID allocator is nil")
|
||||
}
|
||||
|
||||
return v, nil
|
||||
}
|
||||
|
||||
func (v *JSONRowConsumer) IDRange() []int64 {
|
||||
return v.autoIDRange
|
||||
}
|
||||
|
||||
func (v *JSONRowConsumer) RowCount() int64 {
|
||||
return v.rowCounter
|
||||
}
|
||||
|
||||
func (v *JSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
|
||||
if v == nil || v.validators == nil || len(v.validators) == 0 {
|
||||
log.Warn("JSON row consumer is not initialized")
|
||||
return merr.WrapErrImportFailed("JSON row consumer is not initialized")
|
||||
}
|
||||
|
||||
// if rows is nil, that means read to end of file, force flush all data
|
||||
if rows == nil {
|
||||
err := tryFlushBlocks(v.ctx, v.shardsData, v.collectionInfo.Schema, v.callFlushFunc, v.blockSize, Params.DataNodeCfg.BulkInsertMaxMemorySize.GetAsInt64(), true)
|
||||
log.Info("JSON row consumer finished")
|
||||
return err
|
||||
}
|
||||
|
||||
// rows is not nil, flush in necessary:
|
||||
// 1. data block size larger than v.blockSize will be flushed
|
||||
// 2. total data size exceeds MaxTotalSizeInMemory, the largest data block will be flushed
|
||||
err := tryFlushBlocks(v.ctx, v.shardsData, v.collectionInfo.Schema, v.callFlushFunc, v.blockSize, Params.DataNodeCfg.BulkInsertMaxMemorySize.GetAsInt64(), false)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: try flush data but failed", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("try flush data but failed, error: %v", err))
|
||||
}
|
||||
|
||||
// prepare autoid, no matter int64 or varchar pk, we always generate autoid since the hidden field RowIDField requires them
|
||||
primaryKeyID := v.collectionInfo.PrimaryKey.FieldID
|
||||
primaryValidator := v.validators[primaryKeyID]
|
||||
var rowIDBegin typeutil.UniqueID
|
||||
var rowIDEnd typeutil.UniqueID
|
||||
if primaryValidator.autoID {
|
||||
if v.rowIDAllocator == nil {
|
||||
log.Warn("JSON row consumer: primary keys is auto-generated but IDAllocator is nil")
|
||||
return merr.WrapErrImportFailed("primary keys is auto-generated but IDAllocator is nil")
|
||||
}
|
||||
var err error
|
||||
rowIDBegin, rowIDEnd, err = v.rowIDAllocator.Alloc(uint32(len(rows)))
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to generate primary keys", zap.Int("count", len(rows)), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to generate %d primary keys, error: %v", len(rows), err))
|
||||
}
|
||||
if rowIDEnd-rowIDBegin != int64(len(rows)) {
|
||||
log.Warn("JSON row consumer: try to generate primary keys but allocated ids are not enough",
|
||||
zap.Int("count", len(rows)), zap.Int64("generated", rowIDEnd-rowIDBegin))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("try to generate %d primary keys but only %d keys were allocated", len(rows), rowIDEnd-rowIDBegin))
|
||||
}
|
||||
log.Info("JSON row consumer: auto-generate primary keys", zap.Int64("begin", rowIDBegin), zap.Int64("end", rowIDEnd))
|
||||
if !primaryValidator.isString {
|
||||
// if pk is varchar, no need to record auto-generated row ids
|
||||
v.autoIDRange = append(v.autoIDRange, rowIDBegin, rowIDEnd)
|
||||
}
|
||||
}
|
||||
|
||||
// consume rows
|
||||
for i := 0; i < len(rows); i++ {
|
||||
row := rows[i]
|
||||
rowNumber := v.rowCounter + int64(i)
|
||||
|
||||
// hash to a shard number
|
||||
var shard uint32
|
||||
var partitionID int64
|
||||
if primaryValidator.isString {
|
||||
var pk string
|
||||
if primaryValidator.autoID {
|
||||
pk = strconv.FormatInt(rowIDBegin+int64(i), 10)
|
||||
} else {
|
||||
value := row[primaryKeyID]
|
||||
pk, err = getKeyValue(value, primaryValidator.fieldName, primaryValidator.isString)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to parse primary key at the row",
|
||||
zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key at the row %d, error: %v", rowNumber, err))
|
||||
}
|
||||
}
|
||||
|
||||
// hash to shard based on pk, hash to partition if partition key exist
|
||||
hash := typeutil.HashString2Uint32(pk)
|
||||
shard = hash % uint32(v.collectionInfo.ShardNum)
|
||||
partitionID, err = v.hashToPartition(row, rowNumber)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
pkArray := v.shardsData[shard][partitionID][primaryKeyID].(*storage.StringFieldData)
|
||||
pkArray.Data = append(pkArray.Data, pk)
|
||||
} else {
|
||||
// get/generate the row id
|
||||
var pk int64
|
||||
if primaryValidator.autoID {
|
||||
pk = rowIDBegin + int64(i)
|
||||
} else {
|
||||
value := row[primaryKeyID]
|
||||
strValue, err := getKeyValue(value, primaryValidator.fieldName, primaryValidator.isString)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to parse primary key at the row",
|
||||
zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key at the row %d, error: %v", rowNumber, err))
|
||||
}
|
||||
|
||||
// parse the pk from a string
|
||||
pk, err = strconv.ParseInt(strValue, 10, 64)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to parse primary key at the row",
|
||||
zap.String("value", strValue), zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse primary key '%s' at the row %d, error: %v",
|
||||
strValue, rowNumber, err))
|
||||
}
|
||||
}
|
||||
|
||||
hash, err := typeutil.Hash32Int64(pk)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to hash primary key at the row",
|
||||
zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to hash primary key %d at the row %d, error: %v", pk, rowNumber, err))
|
||||
}
|
||||
|
||||
// hash to shard based on pk, hash to partition if partition key exist
|
||||
shard = hash % uint32(v.collectionInfo.ShardNum)
|
||||
partitionID, err = v.hashToPartition(row, rowNumber)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
pkArray := v.shardsData[shard][partitionID][primaryKeyID].(*storage.Int64FieldData)
|
||||
pkArray.Data = append(pkArray.Data, pk)
|
||||
}
|
||||
|
||||
// set rowid field
|
||||
rowIDField := v.shardsData[shard][partitionID][common.RowIDField].(*storage.Int64FieldData)
|
||||
rowIDField.Data = append(rowIDField.Data, rowIDBegin+int64(i))
|
||||
|
||||
// convert value and consume
|
||||
for fieldID, validator := range v.validators {
|
||||
if validator.primaryKey {
|
||||
continue
|
||||
}
|
||||
value := row[fieldID]
|
||||
if err := validator.convertFunc(value, v.shardsData[shard][partitionID][fieldID]); err != nil {
|
||||
log.Warn("JSON row consumer: failed to convert value for field at the row",
|
||||
zap.String("fieldName", validator.fieldName), zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert value for field '%s' at the row %d, error: %v",
|
||||
validator.fieldName, rowNumber, err))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
v.rowCounter += int64(len(rows))
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// hashToPartition hash partition key to get an partition ID, return the first partition ID if no partition key exist
|
||||
// CollectionInfo ensures only one partition ID in the PartitionIDs if no partition key exist
|
||||
func (v *JSONRowConsumer) hashToPartition(row map[storage.FieldID]interface{}, rowNumber int64) (int64, error) {
|
||||
if v.collectionInfo.PartitionKey == nil {
|
||||
if len(v.collectionInfo.PartitionIDs) != 1 {
|
||||
return 0, merr.WrapErrImportFailed(fmt.Sprintf("collection '%s' partition list is empty", v.collectionInfo.Schema.Name))
|
||||
}
|
||||
// no partition key, directly return the target partition id
|
||||
return v.collectionInfo.PartitionIDs[0], nil
|
||||
}
|
||||
|
||||
partitionKeyID := v.collectionInfo.PartitionKey.GetFieldID()
|
||||
partitionKeyValidator := v.validators[partitionKeyID]
|
||||
value := row[partitionKeyID]
|
||||
strValue, err := getKeyValue(value, partitionKeyValidator.fieldName, partitionKeyValidator.isString)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to parse partition key at the row",
|
||||
zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse partition key at the row %d, error: %v", rowNumber, err))
|
||||
}
|
||||
|
||||
var hashValue uint32
|
||||
if partitionKeyValidator.isString {
|
||||
hashValue = typeutil.HashString2Uint32(strValue)
|
||||
} else {
|
||||
// parse the value from a string
|
||||
pk, err := strconv.ParseInt(strValue, 10, 64)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to parse partition key at the row",
|
||||
zap.String("value", strValue), zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse partition key '%s' at the row %d, error: %v",
|
||||
strValue, rowNumber, err))
|
||||
}
|
||||
|
||||
hashValue, err = typeutil.Hash32Int64(pk)
|
||||
if err != nil {
|
||||
log.Warn("JSON row consumer: failed to hash partition key at the row",
|
||||
zap.Int64("key", pk), zap.Int64("rowNumber", rowNumber), zap.Error(err))
|
||||
return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to hash partition key %d at the row %d, error: %v", pk, rowNumber, err))
|
||||
}
|
||||
}
|
||||
|
||||
index := int64(hashValue % uint32(len(v.collectionInfo.PartitionIDs)))
|
||||
return v.collectionInfo.PartitionIDs[index], nil
|
||||
}
|
|
@ -1,664 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
type mockIDAllocator struct {
|
||||
allocErr error
|
||||
}
|
||||
|
||||
func (a *mockIDAllocator) AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocIDResponse, error) {
|
||||
return &rootcoordpb.AllocIDResponse{
|
||||
Status: merr.Success(),
|
||||
ID: int64(1),
|
||||
Count: req.Count,
|
||||
}, a.allocErr
|
||||
}
|
||||
|
||||
func newIDAllocator(ctx context.Context, t *testing.T, allocErr error) *allocator.IDAllocator {
|
||||
mockIDAllocator := &mockIDAllocator{
|
||||
allocErr: allocErr,
|
||||
}
|
||||
|
||||
idAllocator, err := allocator.NewIDAllocator(ctx, mockIDAllocator, int64(1))
|
||||
assert.NoError(t, err)
|
||||
err = idAllocator.Start()
|
||||
assert.NoError(t, err)
|
||||
|
||||
return idAllocator
|
||||
}
|
||||
|
||||
func Test_GetKeyValue(t *testing.T) {
|
||||
fieldName := "dummy"
|
||||
var obj1 interface{} = "aa"
|
||||
val, err := getKeyValue(obj1, fieldName, true)
|
||||
assert.Equal(t, val, "aa")
|
||||
assert.NoError(t, err)
|
||||
|
||||
val, err = getKeyValue(obj1, fieldName, false)
|
||||
assert.Empty(t, val)
|
||||
assert.Error(t, err)
|
||||
|
||||
var obj2 interface{} = json.Number("10")
|
||||
val, err = getKeyValue(obj2, fieldName, false)
|
||||
assert.Equal(t, val, "10")
|
||||
assert.NoError(t, err)
|
||||
|
||||
val, err = getKeyValue(obj2, fieldName, true)
|
||||
assert.Empty(t, val)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func Test_JSONRowConsumerNew(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
t.Run("nil schema", func(t *testing.T) {
|
||||
consumer, err := NewJSONRowConsumer(ctx, nil, nil, 16, nil)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, consumer)
|
||||
})
|
||||
|
||||
t.Run("wrong schema", func(t *testing.T) {
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
AutoID: true,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "uid",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: false,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
},
|
||||
}
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
schema.Fields[0].DataType = schemapb.DataType_None
|
||||
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, consumer)
|
||||
})
|
||||
|
||||
t.Run("primary key is autoid but no IDAllocator", func(t *testing.T) {
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
AutoID: true,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "uid",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: true,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
},
|
||||
}
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, consumer)
|
||||
})
|
||||
|
||||
t.Run("succeed", func(t *testing.T) {
|
||||
collectionInfo, err := NewCollectionInfo(sampleSchema(), 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
|
||||
assert.NotNil(t, consumer)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_JSONRowConsumerHandleIntPK(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
t.Run("nil input", func(t *testing.T) {
|
||||
var consumer *JSONRowConsumer
|
||||
err := consumer.Handle(nil)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "FieldInt64",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: true,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 102,
|
||||
Name: "FieldVarchar",
|
||||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
{
|
||||
FieldID: 103,
|
||||
Name: "FieldFloat",
|
||||
DataType: schemapb.DataType_Float,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
createConsumeFunc := func(shardNum int32, partitionIDs []int64, flushFunc ImportFlushFunc) *JSONRowConsumer {
|
||||
collectionInfo, err := NewCollectionInfo(schema, shardNum, partitionIDs)
|
||||
assert.NoError(t, err)
|
||||
|
||||
idAllocator := newIDAllocator(ctx, t, nil)
|
||||
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, idAllocator, 1, flushFunc)
|
||||
assert.NotNil(t, consumer)
|
||||
assert.NoError(t, err)
|
||||
|
||||
return consumer
|
||||
}
|
||||
|
||||
t.Run("auto pk no partition key", func(t *testing.T) {
|
||||
flushErrFunc := func(fields BlockData, shard int, partID int64) error {
|
||||
return errors.New("dummy error")
|
||||
}
|
||||
|
||||
// rows to input
|
||||
intputRowCount := 100
|
||||
input := make([]map[storage.FieldID]interface{}, intputRowCount)
|
||||
for j := 0; j < intputRowCount; j++ {
|
||||
input[j] = map[int64]interface{}{
|
||||
102: "string",
|
||||
103: json.Number("6.18"),
|
||||
}
|
||||
}
|
||||
|
||||
shardNum := int32(2)
|
||||
partitionID := int64(1)
|
||||
consumer := createConsumeFunc(shardNum, []int64{partitionID}, flushErrFunc)
|
||||
consumer.rowIDAllocator = newIDAllocator(ctx, t, errors.New("error"))
|
||||
|
||||
waitFlushRowCount := 10
|
||||
fieldsData := createFieldsData(schema, waitFlushRowCount, baseTimestamp)
|
||||
consumer.shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
|
||||
|
||||
// nil input will trigger force flush, flushErrFunc returns error
|
||||
err := consumer.Handle(nil)
|
||||
assert.Error(t, err)
|
||||
|
||||
// optional flush, flushErrFunc returns error
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reset flushFunc
|
||||
var callTime int32
|
||||
var flushedRowCount int
|
||||
consumer.callFlushFunc = func(fields BlockData, shard int, partID int64) error {
|
||||
callTime++
|
||||
assert.Less(t, int32(shard), shardNum)
|
||||
assert.Equal(t, partitionID, partID)
|
||||
assert.Greater(t, len(fields), 0)
|
||||
for _, v := range fields {
|
||||
assert.Greater(t, v.RowNum(), 0)
|
||||
}
|
||||
flushedRowCount += fields[102].RowNum()
|
||||
return nil
|
||||
}
|
||||
|
||||
// optional flush succeed, each shard has 10 rows, idErrAllocator returns error
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, waitFlushRowCount*int(shardNum), flushedRowCount)
|
||||
assert.Equal(t, shardNum, callTime)
|
||||
|
||||
// optional flush again, large blockSize, nothing flushed, idAllocator returns error
|
||||
callTime = int32(0)
|
||||
flushedRowCount = 0
|
||||
consumer.shardsData = createShardsData(schema, fieldsData, shardNum, []int64{partitionID})
|
||||
consumer.rowIDAllocator = nil
|
||||
consumer.blockSize = 8 * 1024 * 1024
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, 0, flushedRowCount)
|
||||
assert.Equal(t, int32(0), callTime)
|
||||
|
||||
// idAllocator is ok, consume 100 rows, the previous shardsData(10 rows per shard) is flushed
|
||||
callTime = int32(0)
|
||||
flushedRowCount = 0
|
||||
consumer.blockSize = 1
|
||||
consumer.rowIDAllocator = newIDAllocator(ctx, t, nil)
|
||||
err = consumer.Handle(input)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, waitFlushRowCount*int(shardNum), flushedRowCount)
|
||||
assert.Equal(t, shardNum, callTime)
|
||||
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
|
||||
assert.Equal(t, 2, len(consumer.IDRange()))
|
||||
assert.Equal(t, int64(1), consumer.IDRange()[0])
|
||||
assert.Equal(t, int64(1+intputRowCount), consumer.IDRange()[1])
|
||||
|
||||
// call handle again, the 100 rows are flushed
|
||||
callTime = int32(0)
|
||||
flushedRowCount = 0
|
||||
err = consumer.Handle(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, intputRowCount, flushedRowCount)
|
||||
assert.Equal(t, shardNum, callTime)
|
||||
})
|
||||
|
||||
schema.Fields[0].AutoID = false
|
||||
t.Run("manual pk no partition key", func(t *testing.T) {
|
||||
shardNum := int32(1)
|
||||
partitionID := int64(100)
|
||||
|
||||
var callTime int32
|
||||
var flushedRowCount int
|
||||
flushFunc := func(fields BlockData, shard int, partID int64) error {
|
||||
callTime++
|
||||
assert.Less(t, int32(shard), shardNum)
|
||||
assert.Equal(t, partitionID, partID)
|
||||
assert.Greater(t, len(fields), 0)
|
||||
flushedRowCount += fields[102].RowNum()
|
||||
return nil
|
||||
}
|
||||
|
||||
consumer := createConsumeFunc(shardNum, []int64{partitionID}, flushFunc)
|
||||
|
||||
// failed to parse primary key
|
||||
input := make([]map[storage.FieldID]interface{}, 1)
|
||||
input[0] = map[int64]interface{}{
|
||||
101: int64(99),
|
||||
102: "string",
|
||||
103: 11.11,
|
||||
}
|
||||
|
||||
err := consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
|
||||
// failed to convert pk to int value
|
||||
input[0] = map[int64]interface{}{
|
||||
101: json.Number("a"),
|
||||
102: "string",
|
||||
103: 11.11,
|
||||
}
|
||||
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
|
||||
// failed to hash to partition
|
||||
input[0] = map[int64]interface{}{
|
||||
101: json.Number("99"),
|
||||
102: "string",
|
||||
103: json.Number("4.56"),
|
||||
}
|
||||
consumer.collectionInfo.PartitionIDs = nil
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
consumer.collectionInfo.PartitionIDs = []int64{partitionID}
|
||||
|
||||
// failed to convert value
|
||||
input[0] = map[int64]interface{}{
|
||||
101: json.Number("99"),
|
||||
102: "string",
|
||||
103: json.Number("abc.56"),
|
||||
}
|
||||
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
consumer.shardsData = createShardsData(schema, nil, shardNum, []int64{partitionID}) // in-memory data is dirty, reset
|
||||
|
||||
// succeed, consume 1 row
|
||||
input[0] = map[int64]interface{}{
|
||||
101: json.Number("99"),
|
||||
102: "string",
|
||||
103: json.Number("4.56"),
|
||||
}
|
||||
|
||||
err = consumer.Handle(input)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(1), consumer.RowCount())
|
||||
assert.Equal(t, 0, len(consumer.IDRange()))
|
||||
|
||||
// call handle again, the 1 row is flushed
|
||||
callTime = int32(0)
|
||||
flushedRowCount = 0
|
||||
err = consumer.Handle(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, flushedRowCount)
|
||||
assert.Equal(t, shardNum, callTime)
|
||||
})
|
||||
|
||||
schema.Fields[1].IsPartitionKey = true
|
||||
t.Run("manual pk with partition key", func(t *testing.T) {
|
||||
// 10 partitions
|
||||
partitionIDs := make([]int64, 0)
|
||||
for j := 0; j < 10; j++ {
|
||||
partitionIDs = append(partitionIDs, int64(j))
|
||||
}
|
||||
|
||||
shardNum := int32(2)
|
||||
var flushedRowCount int
|
||||
flushFunc := func(fields BlockData, shard int, partID int64) error {
|
||||
assert.Less(t, int32(shard), shardNum)
|
||||
assert.Contains(t, partitionIDs, partID)
|
||||
assert.Greater(t, len(fields), 0)
|
||||
flushedRowCount += fields[102].RowNum()
|
||||
return nil
|
||||
}
|
||||
|
||||
consumer := createConsumeFunc(shardNum, partitionIDs, flushFunc)
|
||||
|
||||
// rows to input
|
||||
intputRowCount := 100
|
||||
input := make([]map[storage.FieldID]interface{}, intputRowCount)
|
||||
for j := 0; j < intputRowCount; j++ {
|
||||
input[j] = map[int64]interface{}{
|
||||
101: json.Number(strconv.Itoa(j)),
|
||||
102: "partitionKey_" + strconv.Itoa(j),
|
||||
103: json.Number("6.18"),
|
||||
}
|
||||
}
|
||||
|
||||
// 100 rows are consumed to different partitions
|
||||
err := consumer.Handle(input)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
|
||||
|
||||
// call handle again, 100 rows are flushed
|
||||
flushedRowCount = 0
|
||||
err = consumer.Handle(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, intputRowCount, flushedRowCount)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_JSONRowConsumerHandleVarcharPK(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "FieldVarchar",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: false,
|
||||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
{
|
||||
FieldID: 102,
|
||||
Name: "FieldInt64",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 103,
|
||||
Name: "FieldFloat",
|
||||
DataType: schemapb.DataType_Float,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
createConsumeFunc := func(shardNum int32, partitionIDs []int64, flushFunc ImportFlushFunc) *JSONRowConsumer {
|
||||
collectionInfo, err := NewCollectionInfo(schema, shardNum, partitionIDs)
|
||||
assert.NoError(t, err)
|
||||
|
||||
idAllocator := newIDAllocator(ctx, t, nil)
|
||||
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, idAllocator, 1, flushFunc)
|
||||
assert.NotNil(t, consumer)
|
||||
assert.NoError(t, err)
|
||||
|
||||
return consumer
|
||||
}
|
||||
|
||||
t.Run("no partition key", func(t *testing.T) {
|
||||
shardNum := int32(2)
|
||||
partitionID := int64(1)
|
||||
var callTime int32
|
||||
var flushedRowCount int
|
||||
flushFunc := func(fields BlockData, shard int, partID int64) error {
|
||||
callTime++
|
||||
assert.Less(t, int32(shard), shardNum)
|
||||
assert.Equal(t, partitionID, partID)
|
||||
assert.Greater(t, len(fields), 0)
|
||||
for _, v := range fields {
|
||||
assert.Greater(t, v.RowNum(), 0)
|
||||
}
|
||||
flushedRowCount += fields[102].RowNum()
|
||||
return nil
|
||||
}
|
||||
|
||||
consumer := createConsumeFunc(shardNum, []int64{partitionID}, flushFunc)
|
||||
consumer.shardsData = createShardsData(schema, nil, shardNum, []int64{partitionID})
|
||||
|
||||
input := make([]map[storage.FieldID]interface{}, 1)
|
||||
input[0] = map[int64]interface{}{
|
||||
101: true,
|
||||
102: json.Number("1"),
|
||||
103: json.Number("1.56"),
|
||||
}
|
||||
consumer.validators[101].autoID = true
|
||||
err := consumer.Handle(input)
|
||||
assert.NoError(t, err)
|
||||
callTime--
|
||||
flushedRowCount--
|
||||
consumer.rowCounter = 0
|
||||
consumer.validators[101].autoID = false
|
||||
|
||||
// failed to parse primary key
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
|
||||
// failed to hash to partition
|
||||
input[0] = map[int64]interface{}{
|
||||
101: "primaryKey_0",
|
||||
102: json.Number("1"),
|
||||
103: json.Number("1.56"),
|
||||
}
|
||||
consumer.collectionInfo.PartitionIDs = nil
|
||||
err = consumer.Handle(input)
|
||||
assert.Error(t, err)
|
||||
consumer.collectionInfo.PartitionIDs = []int64{partitionID}
|
||||
|
||||
// rows to input
|
||||
intputRowCount := 100
|
||||
input = make([]map[storage.FieldID]interface{}, intputRowCount)
|
||||
for j := 0; j < intputRowCount; j++ {
|
||||
input[j] = map[int64]interface{}{
|
||||
101: "primaryKey_" + strconv.Itoa(j),
|
||||
102: json.Number(strconv.Itoa(j)),
|
||||
103: json.Number("0.618"),
|
||||
}
|
||||
}
|
||||
|
||||
// rows are consumed
|
||||
err = consumer.Handle(input)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
|
||||
assert.Equal(t, 0, len(consumer.IDRange()))
|
||||
|
||||
// call handle again, 100 rows are flushed
|
||||
err = consumer.Handle(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, intputRowCount, flushedRowCount)
|
||||
assert.Equal(t, shardNum, callTime)
|
||||
})
|
||||
|
||||
schema.Fields[1].IsPartitionKey = true
|
||||
t.Run("has partition key", func(t *testing.T) {
|
||||
// 10 partitions
|
||||
partitionIDs := make([]int64, 0)
|
||||
for j := 0; j < 10; j++ {
|
||||
partitionIDs = append(partitionIDs, int64(j))
|
||||
}
|
||||
|
||||
shardNum := int32(2)
|
||||
var flushedRowCount int
|
||||
flushFunc := func(fields BlockData, shard int, partID int64) error {
|
||||
assert.Less(t, int32(shard), shardNum)
|
||||
assert.Contains(t, partitionIDs, partID)
|
||||
assert.Greater(t, len(fields), 0)
|
||||
flushedRowCount += fields[102].RowNum()
|
||||
return nil
|
||||
}
|
||||
|
||||
consumer := createConsumeFunc(shardNum, partitionIDs, flushFunc)
|
||||
|
||||
// rows to input
|
||||
intputRowCount := 100
|
||||
input := make([]map[storage.FieldID]interface{}, intputRowCount)
|
||||
for j := 0; j < intputRowCount; j++ {
|
||||
input[j] = map[int64]interface{}{
|
||||
101: "primaryKey_" + strconv.Itoa(j),
|
||||
102: json.Number(strconv.Itoa(j)),
|
||||
103: json.Number("0.618"),
|
||||
}
|
||||
}
|
||||
|
||||
// 100 rows are consumed to different partitions
|
||||
err := consumer.Handle(input)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(intputRowCount), consumer.RowCount())
|
||||
|
||||
// call handle again, 100 rows are flushed
|
||||
flushedRowCount = 0
|
||||
err = consumer.Handle(nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, intputRowCount, flushedRowCount)
|
||||
|
||||
consumer.validators[101].autoID = true
|
||||
err = consumer.Handle(input)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_JSONRowHashToPartition(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 100,
|
||||
Name: "ID",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: false,
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "FieldVarchar",
|
||||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
{
|
||||
FieldID: 102,
|
||||
Name: "FieldInt64",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
partitionID := int64(1)
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{partitionID})
|
||||
assert.NoError(t, err)
|
||||
consumer, err := NewJSONRowConsumer(ctx, collectionInfo, nil, 16, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, consumer)
|
||||
|
||||
input := make(map[int64]interface{})
|
||||
input[100] = int64(1)
|
||||
input[101] = "abc"
|
||||
input[102] = int64(100)
|
||||
|
||||
t.Run("no partition key", func(t *testing.T) {
|
||||
partID, err := consumer.hashToPartition(input, 0)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, partitionID, partID)
|
||||
})
|
||||
|
||||
t.Run("partition list is empty", func(t *testing.T) {
|
||||
collectionInfo.PartitionIDs = []int64{}
|
||||
partID, err := consumer.hashToPartition(input, 0)
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, int64(0), partID)
|
||||
collectionInfo.PartitionIDs = []int64{partitionID}
|
||||
})
|
||||
|
||||
schema.Fields[1].IsPartitionKey = true
|
||||
err = collectionInfo.resetSchema(schema)
|
||||
assert.NoError(t, err)
|
||||
collectionInfo.PartitionIDs = []int64{1, 2, 3}
|
||||
|
||||
t.Run("varchar partition key", func(t *testing.T) {
|
||||
input := make(map[int64]interface{})
|
||||
input[100] = int64(1)
|
||||
input[101] = true
|
||||
input[102] = int64(100)
|
||||
|
||||
// getKeyValue failed
|
||||
partID, err := consumer.hashToPartition(input, 0)
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, int64(0), partID)
|
||||
|
||||
// succeed
|
||||
input[101] = "abc"
|
||||
partID, err = consumer.hashToPartition(input, 0)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, collectionInfo.PartitionIDs, partID)
|
||||
})
|
||||
|
||||
schema.Fields[1].IsPartitionKey = false
|
||||
schema.Fields[2].IsPartitionKey = true
|
||||
err = collectionInfo.resetSchema(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
t.Run("int64 partition key", func(t *testing.T) {
|
||||
input := make(map[int64]interface{})
|
||||
input[100] = int64(1)
|
||||
input[101] = "abc"
|
||||
input[102] = 100
|
||||
|
||||
// getKeyValue failed
|
||||
partID, err := consumer.hashToPartition(input, 0)
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, int64(0), partID)
|
||||
|
||||
// parse int failed
|
||||
input[102] = json.Number("d")
|
||||
partID, err = consumer.hashToPartition(input, 0)
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, int64(0), partID)
|
||||
|
||||
// succeed
|
||||
input[102] = json.Number("100")
|
||||
partID, err = consumer.hashToPartition(input, 0)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, collectionInfo.PartitionIDs, partID)
|
||||
})
|
||||
}
|
|
@ -1,346 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
"strings"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/exp/maps"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
const (
|
||||
// root field of row-based json format
|
||||
RowRootNode = "rows"
|
||||
)
|
||||
|
||||
type IOReader struct {
|
||||
r io.Reader
|
||||
fileSize int64
|
||||
}
|
||||
|
||||
type JSONParser struct {
|
||||
ctx context.Context // for canceling parse process
|
||||
collectionInfo *CollectionInfo // collection details including schema
|
||||
bufRowCount int // max rows in a buffer
|
||||
updateProgressFunc func(percent int64) // update working progress percent value
|
||||
}
|
||||
|
||||
// NewJSONParser helper function to create a JSONParser
|
||||
func NewJSONParser(ctx context.Context, collectionInfo *CollectionInfo, updateProgressFunc func(percent int64)) *JSONParser {
|
||||
parser := &JSONParser{
|
||||
ctx: ctx,
|
||||
collectionInfo: collectionInfo,
|
||||
bufRowCount: 1024,
|
||||
updateProgressFunc: updateProgressFunc,
|
||||
}
|
||||
adjustBufSize(parser, collectionInfo.Schema)
|
||||
|
||||
return parser
|
||||
}
|
||||
|
||||
func adjustBufSize(parser *JSONParser, collectionSchema *schemapb.CollectionSchema) {
|
||||
sizePerRecord, _ := typeutil.EstimateSizePerRecord(collectionSchema)
|
||||
if sizePerRecord <= 0 {
|
||||
return
|
||||
}
|
||||
|
||||
// for high dimensional vector, the bufSize is a small value, read few rows each time
|
||||
// for low dimensional vector, the bufSize is a large value, read more rows each time
|
||||
bufRowCount := parser.bufRowCount
|
||||
for {
|
||||
if bufRowCount*sizePerRecord > Params.DataNodeCfg.BulkInsertReadBufferSize.GetAsInt() {
|
||||
bufRowCount--
|
||||
} else {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// at least one row per buffer
|
||||
if bufRowCount <= 0 {
|
||||
bufRowCount = 1
|
||||
}
|
||||
|
||||
log.Info("JSON parser: reset bufRowCount", zap.Int("sizePerRecord", sizePerRecord), zap.Int("bufRowCount", bufRowCount))
|
||||
parser.bufRowCount = bufRowCount
|
||||
}
|
||||
|
||||
func (p *JSONParser) combineDynamicRow(dynamicValues map[string]interface{}, row map[storage.FieldID]interface{}) error {
|
||||
if p.collectionInfo.DynamicField == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
dynamicFieldID := p.collectionInfo.DynamicField.GetFieldID()
|
||||
// combine the dynamic field value
|
||||
// valid input:
|
||||
// case 1: {"id": 1, "vector": [], "x": 8, "$meta": "{\"y\": 8}"} ==>> {"id": 1, "vector": [], "$meta": "{\"y\": 8, \"x\": 8}"}
|
||||
// case 2: {"id": 1, "vector": [], "x": 8, "$meta": {}} ==>> {"id": 1, "vector": [], "$meta": {\"x\": 8}}
|
||||
// case 3: {"id": 1, "vector": [], "$meta": "{\"x\": 8}"}
|
||||
// case 4: {"id": 1, "vector": [], "$meta": {"x": 8}}
|
||||
// case 5: {"id": 1, "vector": [], "$meta": {}}
|
||||
// case 6: {"id": 1, "vector": [], "x": 8} ==>> {"id": 1, "vector": [], "$meta": "{\"x\": 8}"}
|
||||
// case 7: {"id": 1, "vector": []}
|
||||
obj, ok := row[dynamicFieldID]
|
||||
if ok {
|
||||
if len(dynamicValues) > 0 {
|
||||
if value, is := obj.(string); is {
|
||||
// case 1
|
||||
mp := make(map[string]interface{})
|
||||
desc := json.NewDecoder(strings.NewReader(value))
|
||||
desc.UseNumber()
|
||||
err := desc.Decode(&mp)
|
||||
if err != nil {
|
||||
// invalid input
|
||||
return merr.WrapErrImportFailed("illegal value for dynamic field, not a JSON format string")
|
||||
}
|
||||
|
||||
maps.Copy(dynamicValues, mp)
|
||||
} else if mp, is := obj.(map[string]interface{}); is {
|
||||
// case 2
|
||||
maps.Copy(dynamicValues, mp)
|
||||
} else {
|
||||
// invalid input
|
||||
return merr.WrapErrImportFailed("illegal value for dynamic field, not a JSON object")
|
||||
}
|
||||
row[dynamicFieldID] = dynamicValues
|
||||
}
|
||||
// else case 3/4/5
|
||||
} else {
|
||||
if len(dynamicValues) > 0 {
|
||||
// case 6
|
||||
row[dynamicFieldID] = dynamicValues
|
||||
} else {
|
||||
// case 7
|
||||
row[dynamicFieldID] = "{}"
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *JSONParser) verifyRow(raw interface{}) (map[storage.FieldID]interface{}, error) {
|
||||
stringMap, ok := raw.(map[string]interface{})
|
||||
if !ok {
|
||||
log.Warn("JSON parser: invalid JSON format, each row should be a key-value map")
|
||||
return nil, merr.WrapErrImportFailed("invalid JSON format, each row should be a key-value map")
|
||||
}
|
||||
|
||||
dynamicValues := make(map[string]interface{})
|
||||
row := make(map[storage.FieldID]interface{})
|
||||
// some fields redundant?
|
||||
for k, v := range stringMap {
|
||||
fieldID, ok := p.collectionInfo.Name2FieldID[k]
|
||||
if (fieldID == p.collectionInfo.PrimaryKey.GetFieldID()) && p.collectionInfo.PrimaryKey.GetAutoID() {
|
||||
// primary key is auto-id, no need to provide
|
||||
log.Warn("JSON parser: the primary key is auto-generated, no need to provide", zap.String("fieldName", k))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the primary key '%s' is auto-generated, no need to provide", k))
|
||||
}
|
||||
|
||||
if ok {
|
||||
row[fieldID] = v
|
||||
} else if p.collectionInfo.DynamicField != nil {
|
||||
// has dynamic field. put redundant pair to dynamicValues
|
||||
dynamicValues[k] = v
|
||||
} else {
|
||||
// no dynamic field. if user provided redundant field, return error
|
||||
log.Warn("JSON parser: the field is not defined in collection schema", zap.String("fieldName", k))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the field '%s' is not defined in collection schema", k))
|
||||
}
|
||||
}
|
||||
|
||||
// some fields not provided?
|
||||
if len(row) != len(p.collectionInfo.Name2FieldID) {
|
||||
for k, v := range p.collectionInfo.Name2FieldID {
|
||||
if (p.collectionInfo.DynamicField != nil) && (v == p.collectionInfo.DynamicField.GetFieldID()) {
|
||||
// ignore dyanmic field, user don't have to provide values for dynamic field
|
||||
continue
|
||||
}
|
||||
|
||||
if v == p.collectionInfo.PrimaryKey.GetFieldID() && p.collectionInfo.PrimaryKey.GetAutoID() {
|
||||
// ignore auto-generaed primary key
|
||||
continue
|
||||
}
|
||||
|
||||
_, ok := row[v]
|
||||
if !ok {
|
||||
// not auto-id primary key, no dynamic field, must provide value
|
||||
log.Warn("JSON parser: a field value is missed", zap.String("fieldName", k))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("value of field '%s' is missed", k))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// combine the redundant pairs into dynamic field(if has)
|
||||
err := p.combineDynamicRow(dynamicValues, row)
|
||||
if err != nil {
|
||||
log.Warn("JSON parser: failed to combine dynamic values", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return row, err
|
||||
}
|
||||
|
||||
func (p *JSONParser) ParseRows(reader *IOReader, handler JSONRowHandler) error {
|
||||
if handler == nil || reader == nil {
|
||||
log.Warn("JSON parse handler is nil")
|
||||
return merr.WrapErrImportFailed("JSON parse handler is nil")
|
||||
}
|
||||
|
||||
dec := json.NewDecoder(reader.r)
|
||||
|
||||
oldPercent := int64(0)
|
||||
updateProgress := func() {
|
||||
if p.updateProgressFunc != nil && reader.fileSize > 0 {
|
||||
percent := (dec.InputOffset() * ProgressValueForPersist) / reader.fileSize
|
||||
if percent > oldPercent { // avoid too many log
|
||||
log.Debug("JSON parser: working progress", zap.Int64("offset", dec.InputOffset()),
|
||||
zap.Int64("fileSize", reader.fileSize), zap.Int64("percent", percent))
|
||||
}
|
||||
oldPercent = percent
|
||||
p.updateProgressFunc(percent)
|
||||
}
|
||||
}
|
||||
|
||||
// treat number value as a string instead of a float64.
|
||||
// by default, json lib treat all number values as float64, but if an int64 value
|
||||
// has more than 15 digits, the value would be incorrect after converting from float64
|
||||
dec.UseNumber()
|
||||
t, err := dec.Token()
|
||||
if err != nil {
|
||||
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err))
|
||||
}
|
||||
if t != json.Delim('{') && t != json.Delim('[') {
|
||||
log.Warn("JSON parser: invalid JSON format, the content should be started with '{' or '['")
|
||||
return merr.WrapErrImportFailed("invalid JSON format, the content should be started with '{' or '['")
|
||||
}
|
||||
|
||||
// read the first level
|
||||
isEmpty := true
|
||||
isOldFormat := t == json.Delim('{')
|
||||
for dec.More() {
|
||||
if isOldFormat {
|
||||
// read the key
|
||||
t, err := dec.Token()
|
||||
if err != nil {
|
||||
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err))
|
||||
}
|
||||
key := t.(string)
|
||||
keyLower := strings.ToLower(key)
|
||||
// the root key should be RowRootNode
|
||||
if keyLower != RowRootNode {
|
||||
log.Warn("JSON parser: invalid JSON format, the root key is not found", zap.String("RowRootNode", RowRootNode), zap.String("key", key))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("invalid JSON format, the root key should be '%s', but get '%s'", RowRootNode, key))
|
||||
}
|
||||
|
||||
// started by '['
|
||||
t, err = dec.Token()
|
||||
if err != nil {
|
||||
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err))
|
||||
}
|
||||
|
||||
if t != json.Delim('[') {
|
||||
log.Warn("JSON parser: invalid JSON format, rows list should begin with '['")
|
||||
return merr.WrapErrImportFailed("invalid JSON format, rows list should begin with '['")
|
||||
}
|
||||
}
|
||||
|
||||
// read buffer
|
||||
buf := make([]map[storage.FieldID]interface{}, 0, p.bufRowCount)
|
||||
for dec.More() {
|
||||
var value interface{}
|
||||
if err := dec.Decode(&value); err != nil {
|
||||
log.Warn("JSON parser: failed to parse row value", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to parse row value, error: %v", err))
|
||||
}
|
||||
|
||||
row, err := p.verifyRow(value)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
updateProgress()
|
||||
|
||||
buf = append(buf, row)
|
||||
if len(buf) >= p.bufRowCount {
|
||||
isEmpty = false
|
||||
if err = handler.Handle(buf); err != nil {
|
||||
log.Warn("JSON parser: failed to convert row value to entity", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert row value to entity, error: %v", err))
|
||||
}
|
||||
|
||||
// clear the buffer
|
||||
buf = make([]map[storage.FieldID]interface{}, 0, p.bufRowCount)
|
||||
}
|
||||
}
|
||||
|
||||
// some rows in buffer not parsed, parse them
|
||||
if len(buf) > 0 {
|
||||
isEmpty = false
|
||||
if err = handler.Handle(buf); err != nil {
|
||||
log.Warn("JSON parser: failed to convert row value to entity", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to convert row value to entity, error: %v", err))
|
||||
}
|
||||
}
|
||||
|
||||
// end by ']'
|
||||
t, err = dec.Token()
|
||||
if err != nil {
|
||||
log.Warn("JSON parser: failed to decode the JSON file", zap.Error(err))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("failed to decode the JSON file, error: %v", err))
|
||||
}
|
||||
|
||||
if t != json.Delim(']') {
|
||||
log.Warn("JSON parser: invalid JSON format, rows list should end with a ']'")
|
||||
return merr.WrapErrImportFailed("invalid JSON format, rows list should end with a ']'")
|
||||
}
|
||||
|
||||
// outside context might be canceled(service stop, or future enhancement for canceling import task)
|
||||
if isCanceled(p.ctx) {
|
||||
log.Warn("JSON parser: import task was canceled")
|
||||
return merr.WrapErrImportFailed("import task was canceled")
|
||||
}
|
||||
|
||||
// nolint
|
||||
// this break means we require the first node must be RowRootNode
|
||||
// once the RowRootNode is parsed, just finish
|
||||
break
|
||||
}
|
||||
|
||||
// empty file is allowed, don't return error
|
||||
if isEmpty {
|
||||
log.Info("JSON parser: row count is 0")
|
||||
return nil
|
||||
}
|
||||
|
||||
updateProgress()
|
||||
|
||||
// send nil to notify the handler all have done
|
||||
return handler.Handle(nil)
|
||||
}
|
|
@ -1,687 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"math"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"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/storage"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
)
|
||||
|
||||
// mock class of JSONRowCounsumer
|
||||
type mockJSONRowConsumer struct {
|
||||
handleErr error
|
||||
rows []map[storage.FieldID]interface{}
|
||||
handleCount int
|
||||
}
|
||||
|
||||
func (v *mockJSONRowConsumer) Handle(rows []map[storage.FieldID]interface{}) error {
|
||||
if v.handleErr != nil {
|
||||
return v.handleErr
|
||||
}
|
||||
if rows != nil {
|
||||
v.rows = append(v.rows, rows...)
|
||||
}
|
||||
v.handleCount++
|
||||
return nil
|
||||
}
|
||||
|
||||
func Test_AdjustBufSize(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
// small row
|
||||
schema := sampleSchema()
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
parser := NewJSONParser(ctx, collectionInfo, nil)
|
||||
assert.NotNil(t, parser)
|
||||
assert.Greater(t, parser.bufRowCount, 0)
|
||||
|
||||
// huge row
|
||||
schema.Fields[9].TypeParams = []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "32768"},
|
||||
}
|
||||
parser = NewJSONParser(ctx, collectionInfo, nil)
|
||||
assert.NotNil(t, parser)
|
||||
assert.Greater(t, parser.bufRowCount, 0)
|
||||
|
||||
// no change
|
||||
schema = &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Description: "schema",
|
||||
AutoID: true,
|
||||
Fields: []*schemapb.FieldSchema{},
|
||||
}
|
||||
parser = NewJSONParser(ctx, collectionInfo, nil)
|
||||
assert.NotNil(t, parser)
|
||||
assert.Greater(t, parser.bufRowCount, 0)
|
||||
adjustBufSize(parser, schema)
|
||||
}
|
||||
|
||||
func Test_JSONParserParseRows_IntPK(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
schema := sampleSchema()
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
parser := NewJSONParser(ctx, collectionInfo, nil)
|
||||
assert.NotNil(t, parser)
|
||||
|
||||
// prepare test data
|
||||
content := &sampleContent{
|
||||
Rows: make([]sampleRow, 0),
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
row := sampleRow{
|
||||
FieldBool: i%2 == 0,
|
||||
FieldInt8: int8(i % math.MaxInt8),
|
||||
FieldInt16: int16(100 + i),
|
||||
FieldInt32: int32(1000 + i),
|
||||
FieldInt64: int64(99999999999999999 + i),
|
||||
FieldFloat: 3 + float32(i)/11,
|
||||
FieldDouble: 1 + float64(i)/7,
|
||||
FieldString: "No." + strconv.FormatInt(int64(i), 10),
|
||||
FieldJSON: fmt.Sprintf("{\"x\": %d}", i),
|
||||
FieldBinaryVector: []int{(200 + i) % math.MaxUint8, 0},
|
||||
FieldFloatVector: []float32{float32(i) + 0.1, float32(i) + 0.2, float32(i) + 0.3, float32(i) + 0.4},
|
||||
FieldArray: []int32{1, 2, 3},
|
||||
}
|
||||
content.Rows = append(content.Rows, row)
|
||||
}
|
||||
|
||||
verifyRows := func(ioReader *IOReader) {
|
||||
consumer := &mockJSONRowConsumer{
|
||||
handleErr: nil,
|
||||
rows: make([]map[int64]interface{}, 0),
|
||||
handleCount: 0,
|
||||
}
|
||||
|
||||
// set bufRowCount = 4, means call handle() after reading 4 rows
|
||||
parser.bufRowCount = 4
|
||||
err = parser.ParseRows(ioReader, consumer)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(content.Rows), len(consumer.rows))
|
||||
for i := 0; i < len(consumer.rows); i++ {
|
||||
contenctRow := content.Rows[i]
|
||||
parsedRow := consumer.rows[i]
|
||||
|
||||
v1, ok := parsedRow[102].(bool)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, contenctRow.FieldBool, v1)
|
||||
|
||||
v2, ok := parsedRow[103].(json.Number)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, strconv.FormatInt(int64(contenctRow.FieldInt8), 10), string(v2))
|
||||
|
||||
v3, ok := parsedRow[104].(json.Number)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, strconv.FormatInt(int64(contenctRow.FieldInt16), 10), string(v3))
|
||||
|
||||
v4, ok := parsedRow[105].(json.Number)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, strconv.FormatInt(int64(contenctRow.FieldInt32), 10), string(v4))
|
||||
|
||||
v5, ok := parsedRow[106].(json.Number)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, strconv.FormatInt(contenctRow.FieldInt64, 10), string(v5))
|
||||
|
||||
v6, ok := parsedRow[107].(json.Number)
|
||||
assert.True(t, ok)
|
||||
f32, err := parseFloat(string(v6), 32, "")
|
||||
assert.NoError(t, err)
|
||||
assert.InDelta(t, contenctRow.FieldFloat, float32(f32), 10e-6)
|
||||
|
||||
v7, ok := parsedRow[108].(json.Number)
|
||||
assert.True(t, ok)
|
||||
f64, err := parseFloat(string(v7), 64, "")
|
||||
assert.NoError(t, err)
|
||||
assert.InDelta(t, contenctRow.FieldDouble, f64, 10e-14)
|
||||
|
||||
v8, ok := parsedRow[109].(string)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, contenctRow.FieldString, v8)
|
||||
|
||||
v9, ok := parsedRow[110].([]interface{})
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(contenctRow.FieldBinaryVector), len(v9))
|
||||
for k := 0; k < len(v9); k++ {
|
||||
val, ok := v9[k].(json.Number)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, strconv.FormatInt(int64(contenctRow.FieldBinaryVector[k]), 10), string(val))
|
||||
}
|
||||
|
||||
v10, ok := parsedRow[111].([]interface{})
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(contenctRow.FieldFloatVector), len(v10))
|
||||
for k := 0; k < len(v10); k++ {
|
||||
val, ok := v10[k].(json.Number)
|
||||
assert.True(t, ok)
|
||||
fval, err := parseFloat(string(val), 64, "")
|
||||
assert.NoError(t, err)
|
||||
assert.InDelta(t, contenctRow.FieldFloatVector[k], float32(fval), 10e-6)
|
||||
}
|
||||
|
||||
v11, ok := parsedRow[113].([]interface{})
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(contenctRow.FieldArray), len(v11))
|
||||
for k := 0; k < len(v11); k++ {
|
||||
val, ok := v11[k].(json.Number)
|
||||
assert.True(t, ok)
|
||||
ival, err := strconv.ParseInt(string(val), 0, 32)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, contenctRow.FieldArray[k], int32(ival))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
consumer := &mockJSONRowConsumer{
|
||||
handleErr: nil,
|
||||
rows: make([]map[int64]interface{}, 0),
|
||||
handleCount: 0,
|
||||
}
|
||||
|
||||
t.Run("parse old format success", func(t *testing.T) {
|
||||
binContent, err := json.Marshal(content)
|
||||
assert.NoError(t, err)
|
||||
strContent := string(binContent)
|
||||
reader := strings.NewReader(strContent)
|
||||
|
||||
ioReader := &IOReader{r: reader, fileSize: int64(len(strContent))}
|
||||
verifyRows(ioReader)
|
||||
|
||||
// empty content
|
||||
reader = strings.NewReader(`{}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(2)}, consumer)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// row count is 0
|
||||
reader = strings.NewReader(`{
|
||||
"rows":[]
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("parse new format success", func(t *testing.T) {
|
||||
binContent, err := json.Marshal(content.Rows)
|
||||
assert.NoError(t, err)
|
||||
strContent := string(binContent)
|
||||
reader := strings.NewReader(strContent)
|
||||
fmt.Println(strContent)
|
||||
|
||||
ioReader := &IOReader{r: reader, fileSize: int64(len(strContent))}
|
||||
verifyRows(ioReader)
|
||||
|
||||
// empty list
|
||||
reader = strings.NewReader(`[]`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(2)}, consumer)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("error cases", func(t *testing.T) {
|
||||
// handler is nil
|
||||
reader := strings.NewReader("")
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(0)}, nil)
|
||||
assert.Error(t, err)
|
||||
|
||||
// not a valid JSON format
|
||||
reader = strings.NewReader(`{[]`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// not a row-based format
|
||||
reader = strings.NewReader(`{
|
||||
"dummy":[]
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// rows is not a list
|
||||
reader = strings.NewReader(`{
|
||||
"rows":
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(5)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// typo
|
||||
reader = strings.NewReader(`{
|
||||
"rows": [}
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(6)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// rows is not a list
|
||||
reader = strings.NewReader(`{
|
||||
"rows": {}
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(8)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// rows is not a list of list
|
||||
reader = strings.NewReader(`{
|
||||
"rows": [[]]
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// typo
|
||||
reader = strings.NewReader(`{
|
||||
"rows": ["]
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(10)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// empty file
|
||||
reader = strings.NewReader(``)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(0)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// redundant field
|
||||
reader = strings.NewReader(`{
|
||||
"rows":[
|
||||
{"dummy": 1, "FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 10, "b": true}}
|
||||
]
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// field missed
|
||||
reader = strings.NewReader(`{
|
||||
"rows":[
|
||||
{"FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 10, "b": true}}
|
||||
]
|
||||
}`)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// handle() error
|
||||
content := `{
|
||||
"rows":[
|
||||
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 7, "b": true}},
|
||||
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 8, "b": false}},
|
||||
{"FieldBool": true, "FieldInt8": 10, "FieldInt16": 101, "FieldInt32": 1001, "FieldInt64": 10001, "FieldFloat": 3.14, "FieldDouble": 1.56, "FieldString": "hello world", "FieldBinaryVector": [254, 0], "FieldFloatVector": [1.1, 1.2, 1.3, 1.4], "FieldJSON": {"a": 9, "b": true}}
|
||||
]
|
||||
}`
|
||||
consumer.handleErr = errors.New("error")
|
||||
reader = strings.NewReader(content)
|
||||
parser.bufRowCount = 2
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
reader = strings.NewReader(content)
|
||||
parser.bufRowCount = 5
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
|
||||
assert.Error(t, err)
|
||||
|
||||
// canceled
|
||||
consumer.handleErr = nil
|
||||
cancel()
|
||||
reader = strings.NewReader(content)
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(100)}, consumer)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_JSONParserParseRows_StrPK(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
schema := strKeySchema()
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
updateProgress := func(percent int64) {
|
||||
assert.Greater(t, percent, int64(0))
|
||||
}
|
||||
parser := NewJSONParser(ctx, collectionInfo, updateProgress)
|
||||
assert.NotNil(t, parser)
|
||||
|
||||
// prepare test data
|
||||
content := &strKeyContent{
|
||||
Rows: make([]strKeyRow, 0),
|
||||
}
|
||||
for i := 0; i < 10; i++ {
|
||||
row := strKeyRow{
|
||||
UID: "strID_" + strconv.FormatInt(int64(i), 10),
|
||||
FieldInt32: int32(10000 + i),
|
||||
FieldFloat: 1 + float32(i)/13,
|
||||
FieldString: strconv.FormatInt(int64(i+1), 10) + " this string contains unicode character: 🎵",
|
||||
FieldBool: i%3 == 0,
|
||||
FieldFloatVector: []float32{float32(i) / 2, float32(i) / 3, float32(i) / 6, float32(i) / 9},
|
||||
}
|
||||
content.Rows = append(content.Rows, row)
|
||||
}
|
||||
|
||||
binContent, err := json.Marshal(content)
|
||||
assert.NoError(t, err)
|
||||
strContent := string(binContent)
|
||||
reader := strings.NewReader(strContent)
|
||||
|
||||
consumer := &mockJSONRowConsumer{
|
||||
handleErr: nil,
|
||||
rows: make([]map[int64]interface{}, 0),
|
||||
handleCount: 0,
|
||||
}
|
||||
|
||||
err = parser.ParseRows(&IOReader{r: reader, fileSize: int64(len(binContent))}, consumer)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(content.Rows), len(consumer.rows))
|
||||
for i := 0; i < len(consumer.rows); i++ {
|
||||
contenctRow := content.Rows[i]
|
||||
parsedRow := consumer.rows[i]
|
||||
|
||||
v1, ok := parsedRow[101].(string)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, contenctRow.UID, v1)
|
||||
|
||||
v2, ok := parsedRow[102].(json.Number)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, strconv.FormatInt(int64(contenctRow.FieldInt32), 10), string(v2))
|
||||
|
||||
v3, ok := parsedRow[103].(json.Number)
|
||||
assert.True(t, ok)
|
||||
f32, err := parseFloat(string(v3), 32, "")
|
||||
assert.NoError(t, err)
|
||||
assert.InDelta(t, contenctRow.FieldFloat, float32(f32), 10e-6)
|
||||
|
||||
v4, ok := parsedRow[104].(string)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, contenctRow.FieldString, v4)
|
||||
|
||||
v5, ok := parsedRow[105].(bool)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, contenctRow.FieldBool, v5)
|
||||
|
||||
v6, ok := parsedRow[106].([]interface{})
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(contenctRow.FieldFloatVector), len(v6))
|
||||
for k := 0; k < len(v6); k++ {
|
||||
val, ok := v6[k].(json.Number)
|
||||
assert.True(t, ok)
|
||||
fval, err := parseFloat(string(val), 64, "")
|
||||
assert.NoError(t, err)
|
||||
assert.InDelta(t, contenctRow.FieldFloatVector[k], float32(fval), 10e-6)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func Test_JSONParserCombineDynamicRow(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Description: "schema",
|
||||
EnableDynamicField: true,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 106,
|
||||
Name: "FieldID",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: false,
|
||||
Description: "int64",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 113,
|
||||
Name: "FieldDynamic",
|
||||
IsPrimaryKey: false,
|
||||
IsDynamic: true,
|
||||
Description: "dynamic field",
|
||||
DataType: schemapb.DataType_JSON,
|
||||
},
|
||||
},
|
||||
}
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
parser := NewJSONParser(ctx, collectionInfo, nil)
|
||||
assert.NotNil(t, parser)
|
||||
|
||||
// valid input:
|
||||
// case 1: {"id": 1, "vector": [], "x": 8, "$meta": "{\"y\": 8}"}
|
||||
// case 2: {"id": 1, "vector": [], "x": 8, "$meta": {}}
|
||||
// case 3: {"id": 1, "vector": [], "$meta": "{\"x\": 8}"}
|
||||
// case 4: {"id": 1, "vector": [], "$meta": {"x": 8}}
|
||||
// case 5: {"id": 1, "vector": [], "$meta": {}}
|
||||
// case 6: {"id": 1, "vector": [], "x": 8}
|
||||
// case 7: {"id": 1, "vector": []}
|
||||
|
||||
t.Run("values combined for dynamic field", func(t *testing.T) {
|
||||
dynamicValues := map[string]interface{}{
|
||||
"x": 8,
|
||||
}
|
||||
row := map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
113: "{\"y\": 8}",
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(113))
|
||||
assert.Contains(t, row[113], "x")
|
||||
assert.Contains(t, row[113], "y")
|
||||
})
|
||||
|
||||
t.Run("outside value for dynamic field", func(t *testing.T) {
|
||||
dynamicValues := map[string]interface{}{
|
||||
"x": 8,
|
||||
}
|
||||
row := map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
113: map[string]interface{}{},
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(113))
|
||||
assert.Contains(t, row[113], "x")
|
||||
})
|
||||
|
||||
t.Run("JSON format string/object for dynamic field", func(t *testing.T) {
|
||||
dynamicValues := map[string]interface{}{}
|
||||
row := map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
113: "{\"x\": 8}",
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(113))
|
||||
})
|
||||
|
||||
t.Run("dynamic field is hidden", func(t *testing.T) {
|
||||
dynamicValues := map[string]interface{}{
|
||||
"x": 8,
|
||||
}
|
||||
row := map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(113))
|
||||
assert.Contains(t, row[113], "x")
|
||||
})
|
||||
|
||||
t.Run("no values for dynamic field", func(t *testing.T) {
|
||||
dynamicValues := map[string]interface{}{}
|
||||
row := map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(113))
|
||||
assert.Equal(t, "{}", row[113])
|
||||
})
|
||||
|
||||
t.Run("invalid input for dynamic field", func(t *testing.T) {
|
||||
dynamicValues := map[string]interface{}{
|
||||
"x": 8,
|
||||
}
|
||||
row := map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
113: 5,
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.Error(t, err)
|
||||
|
||||
row = map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
113: "abc",
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("not allow dynamic values if no dynamic field", func(t *testing.T) {
|
||||
parser.collectionInfo.DynamicField = nil
|
||||
dynamicValues := map[string]interface{}{
|
||||
"x": 8,
|
||||
}
|
||||
row := map[storage.FieldID]interface{}{
|
||||
106: 1,
|
||||
}
|
||||
err = parser.combineDynamicRow(dynamicValues, row)
|
||||
assert.NoError(t, err)
|
||||
assert.NotContains(t, row, int64(113))
|
||||
})
|
||||
}
|
||||
|
||||
func Test_JSONParserVerifyRow(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Description: "schema",
|
||||
EnableDynamicField: true,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 106,
|
||||
Name: "FieldID",
|
||||
IsPrimaryKey: true,
|
||||
AutoID: false,
|
||||
Description: "int64",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 113,
|
||||
Name: "FieldDynamic",
|
||||
IsPrimaryKey: false,
|
||||
IsDynamic: true,
|
||||
Description: "dynamic field",
|
||||
DataType: schemapb.DataType_JSON,
|
||||
},
|
||||
},
|
||||
}
|
||||
collectionInfo, err := NewCollectionInfo(schema, 2, []int64{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
parser := NewJSONParser(ctx, collectionInfo, nil)
|
||||
assert.NotNil(t, parser)
|
||||
|
||||
t.Run("input is not key-value map", func(t *testing.T) {
|
||||
_, err = parser.verifyRow(nil)
|
||||
assert.Error(t, err)
|
||||
|
||||
_, err = parser.verifyRow([]int{0})
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("not auto-id, dynamic field provided", func(t *testing.T) {
|
||||
raw := map[string]interface{}{
|
||||
"FieldID": 100,
|
||||
"FieldDynamic": "{\"x\": 8}",
|
||||
"y": true,
|
||||
}
|
||||
row, err := parser.verifyRow(raw)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(106))
|
||||
assert.Contains(t, row, int64(113))
|
||||
assert.Contains(t, row[113], "x")
|
||||
assert.Contains(t, row[113], "y")
|
||||
})
|
||||
|
||||
t.Run("not auto-id, dynamic field not provided", func(t *testing.T) {
|
||||
raw := map[string]interface{}{
|
||||
"FieldID": 100,
|
||||
}
|
||||
row, err := parser.verifyRow(raw)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(106))
|
||||
assert.Contains(t, row, int64(113))
|
||||
assert.Equal(t, "{}", row[113])
|
||||
})
|
||||
|
||||
t.Run("not auto-id, invalid input dynamic field", func(t *testing.T) {
|
||||
raw := map[string]interface{}{
|
||||
"FieldID": 100,
|
||||
"FieldDynamic": true,
|
||||
"y": true,
|
||||
}
|
||||
_, err = parser.verifyRow(raw)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
schema.Fields[0].AutoID = true
|
||||
err = collectionInfo.resetSchema(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
t.Run("no need to provide value for auto-id", func(t *testing.T) {
|
||||
raw := map[string]interface{}{
|
||||
"FieldID": 100,
|
||||
"FieldDynamic": "{\"x\": 8}",
|
||||
"y": true,
|
||||
}
|
||||
_, err := parser.verifyRow(raw)
|
||||
assert.Error(t, err)
|
||||
|
||||
raw = map[string]interface{}{
|
||||
"FieldDynamic": "{\"x\": 8}",
|
||||
"y": true,
|
||||
}
|
||||
row, err := parser.verifyRow(raw)
|
||||
assert.NoError(t, err)
|
||||
assert.Contains(t, row, int64(113))
|
||||
})
|
||||
|
||||
schema.Fields[1].IsDynamic = false
|
||||
err = collectionInfo.resetSchema(schema)
|
||||
assert.NoError(t, err)
|
||||
|
||||
t.Run("auto id, no dynamic field", func(t *testing.T) {
|
||||
raw := map[string]interface{}{
|
||||
"FieldDynamic": "{\"x\": 8}",
|
||||
"y": true,
|
||||
}
|
||||
_, err := parser.verifyRow(raw)
|
||||
assert.Error(t, err)
|
||||
|
||||
raw = map[string]interface{}{}
|
||||
_, err = parser.verifyRow(raw)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
}
|
|
@ -1,704 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"reflect"
|
||||
"regexp"
|
||||
"strconv"
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/sbinet/npyio"
|
||||
"github.com/sbinet/npyio/npy"
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/text/encoding/unicode"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
)
|
||||
|
||||
var (
|
||||
reStrPre = regexp.MustCompile(`^[|]*?(\d.*)[Sa]$`)
|
||||
reStrPost = regexp.MustCompile(`^[|]*?[Sa](\d.*)$`)
|
||||
reUniPre = regexp.MustCompile(`^[<|>]*?(\d.*)U$`)
|
||||
reUniPost = regexp.MustCompile(`^[<|>]*?U(\d.*)$`)
|
||||
)
|
||||
|
||||
func CreateNumpyFile(path string, data interface{}) error {
|
||||
f, err := os.Create(path)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
err = npyio.Write(f, data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func CreateNumpyData(data interface{}) ([]byte, error) {
|
||||
buf := new(bytes.Buffer)
|
||||
err := npyio.Write(buf, data)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return buf.Bytes(), nil
|
||||
}
|
||||
|
||||
// NumpyAdapter is the class to expand other numpy lib ability
|
||||
// we evaluate two go-numpy lins: github.com/kshedden/gonpy and github.com/sbinet/npyio
|
||||
// the npyio lib read data one by one, the performance is poor, we expand the read methods
|
||||
// to read data in one batch, the performance is 100X faster
|
||||
// the gonpy lib also read data in one batch, but it has no method to read bool data, and the ability
|
||||
// to handle different data type is not strong as the npylib, so we choose the npyio lib to expand.
|
||||
type NumpyAdapter struct {
|
||||
reader io.Reader // data source, typically is os.File
|
||||
npyReader *npy.Reader // reader of npyio lib
|
||||
order binary.ByteOrder // LittleEndian or BigEndian
|
||||
readPosition int // how many elements have been read
|
||||
dataType schemapb.DataType // data type parsed from numpy file header
|
||||
}
|
||||
|
||||
func NewNumpyAdapter(reader io.Reader) (*NumpyAdapter, error) {
|
||||
r, err := npyio.NewReader(reader)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read numpy header", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
dataType, err := convertNumpyType(r.Header.Descr.Type)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to detect data type", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
adapter := &NumpyAdapter{
|
||||
reader: reader,
|
||||
npyReader: r,
|
||||
readPosition: 0,
|
||||
dataType: dataType,
|
||||
}
|
||||
adapter.setByteOrder()
|
||||
|
||||
log.Info("Numpy adapter: numpy header info",
|
||||
zap.Any("shape", r.Header.Descr.Shape),
|
||||
zap.String("dType", r.Header.Descr.Type),
|
||||
zap.Uint8("majorVer", r.Header.Major),
|
||||
zap.Uint8("minorVer", r.Header.Minor),
|
||||
zap.String("ByteOrder", adapter.order.String()))
|
||||
|
||||
return adapter, nil
|
||||
}
|
||||
|
||||
// convertNumpyType gets data type converted from numpy header description, for vector field, the type is int8(binary vector) or float32(float vector)
|
||||
func convertNumpyType(typeStr string) (schemapb.DataType, error) {
|
||||
switch typeStr {
|
||||
case "b1", "<b1", "|b1", "bool":
|
||||
return schemapb.DataType_Bool, nil
|
||||
case "u1", "<u1", "|u1", "uint8": // binary vector data type is uint8
|
||||
return schemapb.DataType_BinaryVector, nil
|
||||
case "i1", "<i1", "|i1", ">i1", "int8":
|
||||
return schemapb.DataType_Int8, nil
|
||||
case "i2", "<i2", "|i2", ">i2", "int16":
|
||||
return schemapb.DataType_Int16, nil
|
||||
case "i4", "<i4", "|i4", ">i4", "int32":
|
||||
return schemapb.DataType_Int32, nil
|
||||
case "i8", "<i8", "|i8", ">i8", "int64":
|
||||
return schemapb.DataType_Int64, nil
|
||||
case "f4", "<f4", "|f4", ">f4", "float32":
|
||||
return schemapb.DataType_Float, nil
|
||||
case "f8", "<f8", "|f8", ">f8", "float64":
|
||||
return schemapb.DataType_Double, nil
|
||||
default:
|
||||
if isStringType(typeStr) {
|
||||
// Note: JSON field and VARCHAR field are using string type numpy
|
||||
return schemapb.DataType_VarChar, nil
|
||||
}
|
||||
log.Warn("Numpy adapter: the numpy file data type is not supported", zap.String("dtype", typeStr))
|
||||
return schemapb.DataType_None, merr.WrapErrImportFailed(fmt.Sprintf("the numpy file dtype '%s' is not supported", typeStr))
|
||||
}
|
||||
}
|
||||
|
||||
func stringLen(dtype string) (int, bool, error) {
|
||||
var utf bool
|
||||
switch {
|
||||
case reStrPre.MatchString(dtype), reStrPost.MatchString(dtype):
|
||||
utf = false
|
||||
case reUniPre.MatchString(dtype), reUniPost.MatchString(dtype):
|
||||
utf = true
|
||||
}
|
||||
|
||||
if m := reStrPre.FindStringSubmatch(dtype); m != nil {
|
||||
v, err := strconv.Atoi(m[1])
|
||||
if err != nil {
|
||||
return 0, false, err
|
||||
}
|
||||
return v, utf, nil
|
||||
}
|
||||
if m := reStrPost.FindStringSubmatch(dtype); m != nil {
|
||||
v, err := strconv.Atoi(m[1])
|
||||
if err != nil {
|
||||
return 0, false, err
|
||||
}
|
||||
return v, utf, nil
|
||||
}
|
||||
if m := reUniPre.FindStringSubmatch(dtype); m != nil {
|
||||
v, err := strconv.Atoi(m[1])
|
||||
if err != nil {
|
||||
return 0, false, err
|
||||
}
|
||||
return v, utf, nil
|
||||
}
|
||||
if m := reUniPost.FindStringSubmatch(dtype); m != nil {
|
||||
v, err := strconv.Atoi(m[1])
|
||||
if err != nil {
|
||||
return 0, false, err
|
||||
}
|
||||
return v, utf, nil
|
||||
}
|
||||
|
||||
log.Warn("Numpy adapter: the numpy file dtype is not varchar data type", zap.String("dtype", dtype))
|
||||
return 0, false, merr.WrapErrImportFailed(fmt.Sprintf("dtype '%s' of numpy file is not varchar data type", dtype))
|
||||
}
|
||||
|
||||
func isStringType(typeStr string) bool {
|
||||
rt := npyio.TypeFrom(typeStr)
|
||||
return rt == reflect.TypeOf((*string)(nil)).Elem()
|
||||
}
|
||||
|
||||
// setByteOrder sets BigEndian/LittleEndian, the logic of this method is copied from npyio lib
|
||||
func (n *NumpyAdapter) setByteOrder() {
|
||||
var nativeEndian binary.ByteOrder
|
||||
v := uint16(1)
|
||||
switch byte(v >> 8) {
|
||||
case 0:
|
||||
nativeEndian = binary.LittleEndian
|
||||
case 1:
|
||||
nativeEndian = binary.BigEndian
|
||||
}
|
||||
|
||||
switch n.npyReader.Header.Descr.Type[0] {
|
||||
case '<':
|
||||
n.order = binary.LittleEndian
|
||||
case '>':
|
||||
n.order = binary.BigEndian
|
||||
default:
|
||||
n.order = nativeEndian
|
||||
}
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) Reader() io.Reader {
|
||||
return n.reader
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) NpyReader() *npy.Reader {
|
||||
return n.npyReader
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) GetType() schemapb.DataType {
|
||||
return n.dataType
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) GetShape() []int {
|
||||
return n.npyReader.Header.Descr.Shape
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) checkCount(count int) int {
|
||||
shape := n.GetShape()
|
||||
|
||||
// empty file?
|
||||
if len(shape) == 0 {
|
||||
return 0
|
||||
}
|
||||
|
||||
total := 1
|
||||
for i := 0; i < len(shape); i++ {
|
||||
total *= shape[i]
|
||||
}
|
||||
|
||||
if total == 0 {
|
||||
return 0
|
||||
}
|
||||
|
||||
// overflow?
|
||||
if count > (total - n.readPosition) {
|
||||
return total - n.readPosition
|
||||
}
|
||||
|
||||
return count
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadBool(count int) ([]bool, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read bool data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read bool data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_Bool {
|
||||
log.Warn("Numpy adapter: numpy data is not bool type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not bool type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: bool")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]bool, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read bool data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf(" failed to read bool data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadUint8(count int) ([]uint8, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read uint8 data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read uint8 data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
// here we don't use n.dataType to check because currently milvus has no uint8 type
|
||||
switch n.npyReader.Header.Descr.Type {
|
||||
case "u1", "<u1", "|u1", "uint8":
|
||||
default:
|
||||
log.Warn("Numpy adapter: numpy data is not uint8 type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not uint8 type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: uint8")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]uint8, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read uint8 data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read uint8 data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadInt8(count int) ([]int8, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read int8 data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read int8 data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_Int8 {
|
||||
log.Warn("Numpy adapter: numpy data is not int8 type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not int8 type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: int8")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]int8, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read int8 data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int8 data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadInt16(count int) ([]int16, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read int16 data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read int16 data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_Int16 {
|
||||
log.Warn("Numpy adapter: numpy data is not int16 type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not int16 type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: int16")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]int16, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read int16 data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int16 data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadInt32(count int) ([]int32, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read int32 data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read int32 data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_Int32 {
|
||||
log.Warn("Numpy adapter: numpy data is not int32 type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not int32 type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: int32")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]int32, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read int32 data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int32 data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadInt64(count int) ([]int64, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read int64 data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read int64 data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_Int64 {
|
||||
log.Warn("Numpy adapter: numpy data is not int64 type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not int64 type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: int64")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]int64, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read int64 data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int64 data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadFloat32(count int) ([]float32, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read float32 data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read float32 data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_Float {
|
||||
log.Warn("Numpy adapter: numpy data is not float32 type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not float32 type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: float32")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]float32, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read float32 data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float32 data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadFloat64(count int) ([]float64, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read float64 data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read float64 data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_Double {
|
||||
log.Warn("Numpy adapter: numpy data is not float64 type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not float64 type")
|
||||
}
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: float64")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// read data
|
||||
data := make([]float64, readSize)
|
||||
err := binary.Read(n.reader, n.order, &data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read float64 data", zap.Int("count", count), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float64 data with count %d, error: %v", readSize, err))
|
||||
}
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func (n *NumpyAdapter) ReadString(count int) ([]string, error) {
|
||||
if count <= 0 {
|
||||
log.Warn("Numpy adapter: cannot read varchar data with a zero or nagative count")
|
||||
return nil, merr.WrapErrImportFailed("cannot read varchar data with a zero or nagative count")
|
||||
}
|
||||
|
||||
// incorrect type
|
||||
if n.dataType != schemapb.DataType_VarChar {
|
||||
log.Warn("Numpy adapter: numpy data is not varchar type")
|
||||
return nil, merr.WrapErrImportFailed("numpy data is not varchar type")
|
||||
}
|
||||
|
||||
// varchar length, this is the max length, some item is shorter than this length, but they also occupy bytes of max length
|
||||
maxLen, utf, err := stringLen(n.npyReader.Header.Descr.Type)
|
||||
if err != nil || maxLen <= 0 {
|
||||
log.Warn("Numpy adapter: failed to get max length of varchar from numpy file header", zap.Int("maxLen", maxLen), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to get max length %d of varchar from numpy file header, error: %v", maxLen, err))
|
||||
}
|
||||
// log.Info("Numpy adapter: get varchar max length from numpy file header", zap.Int("maxLen", maxLen), zap.Bool("utf", utf))
|
||||
|
||||
// avoid read overflow
|
||||
readSize := n.checkCount(count)
|
||||
if readSize <= 0 {
|
||||
// end of file, nothing to read
|
||||
log.Info("Numpy adapter: read to end of file, type: varchar")
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
if n.reader == nil {
|
||||
log.Warn("Numpy adapter: reader is nil")
|
||||
return nil, merr.WrapErrImportFailed("numpy reader is nil")
|
||||
}
|
||||
|
||||
// read string one by one is not efficient, here we read strings batch by batch, each bach size is no more than 16MB
|
||||
batchRead := 1 // rows of each batch, make sure this value is equal or greater than 1
|
||||
if utf {
|
||||
batchRead += Params.DataNodeCfg.BulkInsertReadBufferSize.GetAsInt() / (utf8.UTFMax * maxLen)
|
||||
} else {
|
||||
batchRead += Params.DataNodeCfg.BulkInsertReadBufferSize.GetAsInt() / maxLen
|
||||
}
|
||||
|
||||
log.Info("Numpy adapter: prepare to read varchar batch by batch",
|
||||
zap.Int("readSize", readSize), zap.Int("batchRead", batchRead))
|
||||
|
||||
// read data
|
||||
data := make([]string, 0)
|
||||
for {
|
||||
// the last batch
|
||||
readDone := len(data)
|
||||
if readDone+batchRead > readSize {
|
||||
batchRead = readSize - readDone
|
||||
}
|
||||
|
||||
if utf {
|
||||
// in the numpy file with utf32 encoding, the dType could be like "<U2",
|
||||
// "<" is byteorder(LittleEndian), "U" means it is utf32 encoding, "2" means the max length of strings is 2(characters)
|
||||
// each character occupy 4 bytes, each string occupys 4*maxLen bytes
|
||||
// for example, a numpy file has two strings: "a" and "bb", the maxLen is 2, byte order is LittleEndian
|
||||
// the character "a" occupys 2*4=8 bytes(0x97,0x00,0x00,0x00,0x00,0x00,0x00,0x00),
|
||||
// the "bb" occupys 8 bytes(0x97,0x00,0x00,0x00,0x98,0x00,0x00,0x00)
|
||||
// for non-ascii characters, the unicode could be 1 ~ 4 bytes, each character occupys 4 bytes, too
|
||||
raw, err := io.ReadAll(io.LimitReader(n.reader, utf8.UTFMax*int64(maxLen)*int64(batchRead)))
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read utf32 bytes from numpy file",
|
||||
zap.Int("readDone", readDone), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read utf32 bytes from numpy file, error: %v", err))
|
||||
}
|
||||
|
||||
// read string one by one from the buffer
|
||||
for j := 0; j < batchRead; j++ {
|
||||
str, err := decodeUtf32(raw[j*utf8.UTFMax*maxLen:(j+1)*utf8.UTFMax*maxLen], n.order)
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed todecode utf32 bytes",
|
||||
zap.Int("position", readDone+j), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to decode utf32 bytes, error: %v", err))
|
||||
}
|
||||
|
||||
data = append(data, str)
|
||||
}
|
||||
} else {
|
||||
// in the numpy file with ansi encoding, the dType could be like "S2", maxLen is 2, each string occupys 2 bytes
|
||||
// bytes.Index(buf, []byte{0}) tell us which position is the end of the string
|
||||
buf, err := io.ReadAll(io.LimitReader(n.reader, int64(maxLen)*int64(batchRead)))
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to read ascii bytes from numpy file",
|
||||
zap.Int("readDone", readDone), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read ascii bytes from numpy file, error: %v", err))
|
||||
}
|
||||
|
||||
// read string one by one from the buffer
|
||||
for j := 0; j < batchRead; j++ {
|
||||
oneBuf := buf[j*maxLen : (j+1)*maxLen]
|
||||
n := bytes.Index(oneBuf, []byte{0})
|
||||
if n > 0 {
|
||||
oneBuf = oneBuf[:n]
|
||||
}
|
||||
|
||||
data = append(data, string(oneBuf))
|
||||
}
|
||||
}
|
||||
|
||||
// quit the circle if specified size is read
|
||||
if len(data) >= readSize {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
log.Info("Numpy adapter: a block of varchar has been read", zap.Int("rowCount", len(data)))
|
||||
|
||||
// update read position after successfully read
|
||||
n.readPosition += readSize
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func decodeUtf32(src []byte, order binary.ByteOrder) (string, error) {
|
||||
if len(src)%4 != 0 {
|
||||
log.Warn("Numpy adapter: invalid utf32 bytes length, the byte array length should be multiple of 4", zap.Int("byteLen", len(src)))
|
||||
return "", merr.WrapErrImportFailed(fmt.Sprintf("invalid utf32 bytes length %d, the byte array length should be multiple of 4", len(src)))
|
||||
}
|
||||
|
||||
var str string
|
||||
for len(src) > 0 {
|
||||
// check the high bytes, if high bytes are 0, the UNICODE is less than U+FFFF, we can use unicode.UTF16 to decode
|
||||
isUtf16 := false
|
||||
var lowbytesPosition int
|
||||
uOrder := unicode.LittleEndian
|
||||
if order == binary.LittleEndian {
|
||||
if src[2] == 0 && src[3] == 0 {
|
||||
isUtf16 = true
|
||||
}
|
||||
lowbytesPosition = 0
|
||||
} else {
|
||||
if src[0] == 0 && src[1] == 0 {
|
||||
isUtf16 = true
|
||||
}
|
||||
lowbytesPosition = 2
|
||||
uOrder = unicode.BigEndian
|
||||
}
|
||||
|
||||
if isUtf16 {
|
||||
// use unicode.UTF16 to decode the low bytes to utf8
|
||||
// utf32 and utf16 is same if the unicode code is less than 65535
|
||||
if src[lowbytesPosition] != 0 || src[lowbytesPosition+1] != 0 {
|
||||
decoder := unicode.UTF16(uOrder, unicode.IgnoreBOM).NewDecoder()
|
||||
res, err := decoder.Bytes(src[lowbytesPosition : lowbytesPosition+2])
|
||||
if err != nil {
|
||||
log.Warn("Numpy adapter: failed to decode utf32 binary bytes", zap.Error(err))
|
||||
return "", merr.WrapErrImportFailed(fmt.Sprintf("failed to decode utf32 binary bytes, error: %v", err))
|
||||
}
|
||||
str += string(res)
|
||||
}
|
||||
} else {
|
||||
// convert the 4 bytes to a unicode and encode to utf8
|
||||
// Golang strongly opposes utf32 coding, this kind of encoding has been excluded from standard lib
|
||||
var x uint32
|
||||
if order == binary.LittleEndian {
|
||||
x = uint32(src[3])<<24 | uint32(src[2])<<16 | uint32(src[1])<<8 | uint32(src[0])
|
||||
} else {
|
||||
x = uint32(src[0])<<24 | uint32(src[1])<<16 | uint32(src[2])<<8 | uint32(src[3])
|
||||
}
|
||||
r := rune(x)
|
||||
utf8Code := make([]byte, 4)
|
||||
utf8.EncodeRune(utf8Code, r)
|
||||
if r == utf8.RuneError {
|
||||
log.Warn("Numpy adapter: failed to convert 4 bytes unicode to utf8 rune", zap.Uint32("code", x))
|
||||
return "", merr.WrapErrImportFailed(fmt.Sprintf("failed to convert 4 bytes unicode %d to utf8 rune", x))
|
||||
}
|
||||
str += string(utf8Code)
|
||||
}
|
||||
|
||||
src = src[4:]
|
||||
}
|
||||
return str, nil
|
||||
}
|
|
@ -1,839 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"io"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/sbinet/npyio/npy"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
)
|
||||
|
||||
type MockReader struct{}
|
||||
|
||||
func (r *MockReader) Read(p []byte) (n int, err error) {
|
||||
return 0, io.EOF
|
||||
}
|
||||
|
||||
func Test_CreateNumpyFile(t *testing.T) {
|
||||
// directory doesn't exist
|
||||
data1 := []float32{1, 2, 3, 4, 5}
|
||||
err := CreateNumpyFile("/dummy_not_exist/dummy.npy", data1)
|
||||
assert.Error(t, err)
|
||||
|
||||
// invalid data type
|
||||
data2 := make(map[string]int)
|
||||
err = CreateNumpyFile("/tmp/dummy.npy", data2)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func Test_CreateNumpyData(t *testing.T) {
|
||||
// directory doesn't exist
|
||||
data1 := []float32{1, 2, 3, 4, 5}
|
||||
buf, err := CreateNumpyData(data1)
|
||||
assert.NotNil(t, buf)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// invalid data type
|
||||
data2 := make(map[string]int)
|
||||
buf, err = CreateNumpyData(data2)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, buf)
|
||||
}
|
||||
|
||||
func Test_ConvertNumpyType(t *testing.T) {
|
||||
checkFunc := func(inputs []string, output schemapb.DataType) {
|
||||
for i := 0; i < len(inputs); i++ {
|
||||
dt, err := convertNumpyType(inputs[i])
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, output, dt)
|
||||
}
|
||||
}
|
||||
|
||||
checkFunc([]string{"b1", "<b1", "|b1", "bool"}, schemapb.DataType_Bool)
|
||||
checkFunc([]string{"i1", "<i1", "|i1", ">i1", "int8"}, schemapb.DataType_Int8)
|
||||
checkFunc([]string{"i2", "<i2", "|i2", ">i2", "int16"}, schemapb.DataType_Int16)
|
||||
checkFunc([]string{"i4", "<i4", "|i4", ">i4", "int32"}, schemapb.DataType_Int32)
|
||||
checkFunc([]string{"i8", "<i8", "|i8", ">i8", "int64"}, schemapb.DataType_Int64)
|
||||
checkFunc([]string{"f4", "<f4", "|f4", ">f4", "float32"}, schemapb.DataType_Float)
|
||||
checkFunc([]string{"f8", "<f8", "|f8", ">f8", "float64"}, schemapb.DataType_Double)
|
||||
|
||||
dt, err := convertNumpyType("dummy")
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, schemapb.DataType_None, dt)
|
||||
}
|
||||
|
||||
func Test_StringLen(t *testing.T) {
|
||||
len, utf, err := stringLen("S1")
|
||||
assert.Equal(t, 1, len)
|
||||
assert.False(t, utf)
|
||||
assert.NoError(t, err)
|
||||
|
||||
len, utf, err = stringLen("2S")
|
||||
assert.Equal(t, 2, len)
|
||||
assert.False(t, utf)
|
||||
assert.NoError(t, err)
|
||||
|
||||
len, utf, err = stringLen("<U3")
|
||||
assert.Equal(t, 3, len)
|
||||
assert.True(t, utf)
|
||||
assert.NoError(t, err)
|
||||
|
||||
len, utf, err = stringLen(">4U")
|
||||
assert.Equal(t, 4, len)
|
||||
assert.True(t, utf)
|
||||
assert.NoError(t, err)
|
||||
|
||||
len, utf, err = stringLen("dummy")
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, 0, len)
|
||||
assert.False(t, utf)
|
||||
}
|
||||
|
||||
func Test_NumpyAdapterSetByteOrder(t *testing.T) {
|
||||
adapter := &NumpyAdapter{
|
||||
reader: nil,
|
||||
npyReader: &npy.Reader{},
|
||||
}
|
||||
assert.Nil(t, adapter.Reader())
|
||||
assert.NotNil(t, adapter.NpyReader())
|
||||
|
||||
adapter.npyReader.Header.Descr.Type = "<i8"
|
||||
adapter.setByteOrder()
|
||||
assert.Equal(t, binary.LittleEndian, adapter.order)
|
||||
|
||||
adapter.npyReader.Header.Descr.Type = ">i8"
|
||||
adapter.setByteOrder()
|
||||
assert.Equal(t, binary.BigEndian, adapter.order)
|
||||
}
|
||||
|
||||
func Test_NumpyAdapterReadError(t *testing.T) {
|
||||
// reader size is zero
|
||||
// t.Run("test size is zero", func(t *testing.T) {
|
||||
// adapter := &NumpyAdapter{
|
||||
// reader: nil,
|
||||
// npyReader: nil,
|
||||
// }
|
||||
// _, err := adapter.ReadBool(0)
|
||||
// assert.Error(t, err)
|
||||
// _, err = adapter.ReadUint8(0)
|
||||
// assert.Error(t, err)
|
||||
// _, err = adapter.ReadInt8(0)
|
||||
// assert.Error(t, err)
|
||||
// _, err = adapter.ReadInt16(0)
|
||||
// assert.Error(t, err)
|
||||
// _, err = adapter.ReadInt32(0)
|
||||
// assert.Error(t, err)
|
||||
// _, err = adapter.ReadInt64(0)
|
||||
// assert.Error(t, err)
|
||||
// _, err = adapter.ReadFloat32(0)
|
||||
// assert.Error(t, err)
|
||||
// _, err = adapter.ReadFloat64(0)
|
||||
// assert.Error(t, err)
|
||||
// })
|
||||
|
||||
createAdatper := func(dt schemapb.DataType) *NumpyAdapter {
|
||||
adapter := &NumpyAdapter{
|
||||
reader: &MockReader{},
|
||||
npyReader: &npy.Reader{
|
||||
Header: npy.Header{},
|
||||
},
|
||||
dataType: dt,
|
||||
order: binary.BigEndian,
|
||||
}
|
||||
adapter.npyReader.Header.Descr.Shape = []int{1}
|
||||
return adapter
|
||||
}
|
||||
|
||||
t.Run("test read bool", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadBool(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_Bool)
|
||||
data, err = adapter.ReadBool(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1})
|
||||
data, err = adapter.ReadBool(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadBool(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read uint8", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
adapter.npyReader.Header.Descr.Type = "dummy"
|
||||
data, err := adapter.ReadUint8(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter.npyReader.Header.Descr.Type = "u1"
|
||||
data, err = adapter.ReadUint8(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1})
|
||||
data, err = adapter.ReadUint8(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadUint8(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read int8", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadInt8(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_Int8)
|
||||
data, err = adapter.ReadInt8(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1})
|
||||
data, err = adapter.ReadInt8(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadInt8(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read int16", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadInt16(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_Int16)
|
||||
data, err = adapter.ReadInt16(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1, 2})
|
||||
data, err = adapter.ReadInt16(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadInt16(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read int32", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadInt32(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_Int32)
|
||||
data, err = adapter.ReadInt32(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1, 2, 3, 4})
|
||||
data, err = adapter.ReadInt32(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadInt32(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read int64", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadInt64(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_Int64)
|
||||
data, err = adapter.ReadInt64(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1, 2, 3, 4, 5, 6, 7, 8})
|
||||
data, err = adapter.ReadInt64(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadInt64(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read float", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadFloat32(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_Float)
|
||||
data, err = adapter.ReadFloat32(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1, 2, 3, 4})
|
||||
data, err = adapter.ReadFloat32(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadFloat32(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read double", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadFloat64(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_Double)
|
||||
data, err = adapter.ReadFloat64(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = bytes.NewReader([]byte{1, 2, 3, 4, 5, 6, 7, 8})
|
||||
data, err = adapter.ReadFloat64(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadFloat64(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
t.Run("test read varchar", func(t *testing.T) {
|
||||
// type mismatch
|
||||
adapter := createAdatper(schemapb.DataType_None)
|
||||
data, err := adapter.ReadString(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// reader is nil, cannot read
|
||||
adapter = createAdatper(schemapb.DataType_VarChar)
|
||||
adapter.reader = nil
|
||||
adapter.npyReader.Header.Descr.Type = "S3"
|
||||
data, err = adapter.ReadString(1)
|
||||
assert.Nil(t, data)
|
||||
assert.Error(t, err)
|
||||
|
||||
// read one element from reader
|
||||
adapter.reader = strings.NewReader("abc")
|
||||
data, err = adapter.ReadString(1)
|
||||
assert.NotEmpty(t, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// nothing to read
|
||||
data, err = adapter.ReadString(1)
|
||||
assert.Nil(t, data)
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
}
|
||||
|
||||
func Test_NumpyAdapterRead(t *testing.T) {
|
||||
err := os.MkdirAll(TempFilesPath, os.ModePerm)
|
||||
assert.NoError(t, err)
|
||||
defer os.RemoveAll(TempFilesPath)
|
||||
|
||||
t.Run("test read bool", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "bool.npy"
|
||||
data := []bool{true, false, true, false}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadBool(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadBool(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadBool(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
|
||||
// incorrect type read
|
||||
resu1, err := adapter.ReadUint8(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resu1)
|
||||
|
||||
resi1, err := adapter.ReadInt8(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resi1)
|
||||
|
||||
resi2, err := adapter.ReadInt16(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resi2)
|
||||
|
||||
resi4, err := adapter.ReadInt32(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resi4)
|
||||
|
||||
resi8, err := adapter.ReadInt64(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resi8)
|
||||
|
||||
resf4, err := adapter.ReadFloat32(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resf4)
|
||||
|
||||
resf8, err := adapter.ReadFloat64(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resf8)
|
||||
})
|
||||
|
||||
t.Run("test read uint8", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "uint8.npy"
|
||||
data := []uint8{1, 2, 3, 4, 5, 6}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadUint8(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadUint8(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadUint8(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
|
||||
// incorrect type read
|
||||
resb, err := adapter.ReadBool(len(data))
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, resb)
|
||||
})
|
||||
|
||||
t.Run("test read int8", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "int8.npy"
|
||||
data := []int8{1, 2, 3, 4, 5, 6}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadInt8(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadInt8(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadInt8(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
})
|
||||
|
||||
t.Run("test read int16", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "int16.npy"
|
||||
data := []int16{1, 2, 3, 4, 5, 6}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadInt16(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadInt16(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadInt16(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
})
|
||||
|
||||
t.Run("test read int32", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "int32.npy"
|
||||
data := []int32{1, 2, 3, 4, 5, 6}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadInt32(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadInt32(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadInt32(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
})
|
||||
|
||||
t.Run("test read int64", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "int64.npy"
|
||||
data := []int64{1, 2, 3, 4, 5, 6}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadInt64(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadInt64(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadInt64(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
})
|
||||
|
||||
t.Run("test read float", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "float.npy"
|
||||
data := []float32{1, 2, 3, 4, 5, 6}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadFloat32(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadFloat32(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadFloat32(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
})
|
||||
|
||||
t.Run("test read double", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "double.npy"
|
||||
data := []float64{1, 2, 3, 4, 5, 6}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadFloat64(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadFloat64(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadFloat64(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
})
|
||||
|
||||
t.Run("test read ascii characters with ansi", func(t *testing.T) {
|
||||
npyReader := &npy.Reader{
|
||||
Header: npy.Header{},
|
||||
}
|
||||
|
||||
data := make([]byte, 0)
|
||||
values := []string{"ab", "ccc", "d"}
|
||||
maxLen := 0
|
||||
for _, str := range values {
|
||||
if len(str) > maxLen {
|
||||
maxLen = len(str)
|
||||
}
|
||||
}
|
||||
for _, str := range values {
|
||||
for i := 0; i < maxLen; i++ {
|
||||
if i < len(str) {
|
||||
data = append(data, str[i])
|
||||
} else {
|
||||
data = append(data, 0)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
npyReader.Header.Descr.Shape = append(npyReader.Header.Descr.Shape, len(values))
|
||||
|
||||
adapter := &NumpyAdapter{
|
||||
reader: strings.NewReader(string(data)),
|
||||
npyReader: npyReader,
|
||||
readPosition: 0,
|
||||
dataType: schemapb.DataType_VarChar,
|
||||
}
|
||||
|
||||
// count should greater than 0
|
||||
res, err := adapter.ReadString(0)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, res)
|
||||
|
||||
// maxLen is zero
|
||||
npyReader.Header.Descr.Type = "S0"
|
||||
res, err = adapter.ReadString(1)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, res)
|
||||
|
||||
npyReader.Header.Descr.Type = "S" + strconv.FormatInt(int64(maxLen), 10)
|
||||
|
||||
res, err = adapter.ReadString(len(values) + 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(values), len(res))
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, values[i], res[i])
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("test read ascii characters with utf32", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "varchar1.npy"
|
||||
data := []string{"a ", "bbb", " c", "dd", "eeee", "fff"}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// partly read
|
||||
res, err := adapter.ReadString(len(data) - 1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data)-1, len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
|
||||
// read the left data
|
||||
res, err = adapter.ReadString(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(res))
|
||||
assert.Equal(t, data[len(data)-1], res[0])
|
||||
|
||||
// nothing to read
|
||||
res, err = adapter.ReadString(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Nil(t, res)
|
||||
})
|
||||
|
||||
t.Run("test read non-ascii characters with utf32", func(t *testing.T) {
|
||||
filePath := TempFilesPath + "varchar2.npy"
|
||||
data := []string{"で と ど ", " 马克bbb", "$(한)삼각*"}
|
||||
err := CreateNumpyFile(filePath, data)
|
||||
assert.NoError(t, err)
|
||||
|
||||
file, err := os.Open(filePath)
|
||||
assert.NoError(t, err)
|
||||
defer file.Close()
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
assert.NoError(t, err)
|
||||
res, err := adapter.ReadString(len(data))
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, len(data), len(res))
|
||||
|
||||
for i := 0; i < len(res); i++ {
|
||||
assert.Equal(t, data[i], res[i])
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func Test_DecodeUtf32(t *testing.T) {
|
||||
// wrong input
|
||||
res, err := decodeUtf32([]byte{1, 2}, binary.LittleEndian)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, res)
|
||||
|
||||
// this string contains ascii characters and unicode characters
|
||||
str := "ad◤三百🎵ゐ↙"
|
||||
|
||||
// utf32 littleEndian of str
|
||||
src := []byte{97, 0, 0, 0, 100, 0, 0, 0, 228, 37, 0, 0, 9, 78, 0, 0, 126, 118, 0, 0, 181, 243, 1, 0, 144, 48, 0, 0, 153, 33, 0, 0}
|
||||
res, err = decodeUtf32(src, binary.LittleEndian)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, str, res)
|
||||
|
||||
// utf32 bigEndian of str
|
||||
src = []byte{0, 0, 0, 97, 0, 0, 0, 100, 0, 0, 37, 228, 0, 0, 78, 9, 0, 0, 118, 126, 0, 1, 243, 181, 0, 0, 48, 144, 0, 0, 33, 153}
|
||||
res, err = decodeUtf32(src, binary.BigEndian)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, str, res)
|
||||
}
|
|
@ -1,632 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type NumpyColumnReader struct {
|
||||
fieldName string // name of the target column
|
||||
fieldID storage.FieldID // ID of the target column
|
||||
dataType schemapb.DataType // data type of the target column
|
||||
rowCount int // how many rows need to be read
|
||||
dimension int // only for vector
|
||||
file storage.FileReader // file to be read
|
||||
reader *NumpyAdapter // data reader
|
||||
}
|
||||
|
||||
func closeReaders(columnReaders []*NumpyColumnReader) {
|
||||
for _, reader := range columnReaders {
|
||||
if reader.file != nil {
|
||||
err := reader.file.Close()
|
||||
if err != nil {
|
||||
log.Warn("Numper parser: failed to close numpy file", zap.String("fileName", reader.fieldName+NumpyFileExt))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type NumpyParser struct {
|
||||
ctx context.Context // for canceling parse process
|
||||
collectionInfo *CollectionInfo // collection details including schema
|
||||
rowIDAllocator *allocator.IDAllocator // autoid allocator
|
||||
blockSize int64 // maximum size of a read block(unit:byte)
|
||||
chunkManager storage.ChunkManager // storage interfaces to browse/read the files
|
||||
autoIDRange []int64 // auto-generated id range, for example: [1, 10, 20, 25] means id from 1 to 10 and 20 to 25
|
||||
callFlushFunc ImportFlushFunc // call back function to flush segment
|
||||
updateProgressFunc func(percent int64) // update working progress percent value
|
||||
}
|
||||
|
||||
// NewNumpyParser is helper function to create a NumpyParser
|
||||
func NewNumpyParser(ctx context.Context,
|
||||
collectionInfo *CollectionInfo,
|
||||
idAlloc *allocator.IDAllocator,
|
||||
blockSize int64,
|
||||
chunkManager storage.ChunkManager,
|
||||
flushFunc ImportFlushFunc,
|
||||
updateProgressFunc func(percent int64),
|
||||
) (*NumpyParser, error) {
|
||||
if collectionInfo == nil {
|
||||
log.Warn("Numper parser: collection schema is nil")
|
||||
return nil, merr.WrapErrImportFailed("collection schema is nil")
|
||||
}
|
||||
|
||||
if idAlloc == nil {
|
||||
log.Warn("Numper parser: id allocator is nil")
|
||||
return nil, merr.WrapErrImportFailed("id allocator is nil")
|
||||
}
|
||||
|
||||
if chunkManager == nil {
|
||||
log.Warn("Numper parser: chunk manager pointer is nil")
|
||||
return nil, merr.WrapErrImportFailed("chunk manager pointer is nil")
|
||||
}
|
||||
|
||||
if flushFunc == nil {
|
||||
log.Warn("Numper parser: flush function is nil")
|
||||
return nil, merr.WrapErrImportFailed("flush function is nil")
|
||||
}
|
||||
|
||||
parser := &NumpyParser{
|
||||
ctx: ctx,
|
||||
collectionInfo: collectionInfo,
|
||||
rowIDAllocator: idAlloc,
|
||||
blockSize: blockSize,
|
||||
chunkManager: chunkManager,
|
||||
autoIDRange: make([]int64, 0),
|
||||
callFlushFunc: flushFunc,
|
||||
updateProgressFunc: updateProgressFunc,
|
||||
}
|
||||
|
||||
return parser, nil
|
||||
}
|
||||
|
||||
func (p *NumpyParser) IDRange() []int64 {
|
||||
return p.autoIDRange
|
||||
}
|
||||
|
||||
// Parse is the function entry
|
||||
func (p *NumpyParser) Parse(filePaths []string) error {
|
||||
// check redundant files for column-based import
|
||||
// if the field is primary key and autoID is false, the file is required
|
||||
// any redundant file is not allowed
|
||||
err := p.validateFileNames(filePaths)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// open files and verify file header
|
||||
readers, err := p.createReaders(filePaths)
|
||||
// make sure all the files are closed finally, must call this method before the function return
|
||||
defer closeReaders(readers)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// read all data from the numpy files
|
||||
err = p.consume(readers)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// validateFileNames is to check redundant file and missed file
|
||||
func (p *NumpyParser) validateFileNames(filePaths []string) error {
|
||||
dynamicFieldName := ""
|
||||
requiredFieldNames := make(map[string]interface{})
|
||||
for _, schema := range p.collectionInfo.Schema.Fields {
|
||||
if schema.GetIsDynamic() && p.collectionInfo.Schema.GetEnableDynamicField() {
|
||||
dynamicFieldName = schema.GetName()
|
||||
}
|
||||
if schema.GetIsPrimaryKey() {
|
||||
if !schema.GetAutoID() {
|
||||
requiredFieldNames[schema.GetName()] = nil
|
||||
}
|
||||
} else {
|
||||
requiredFieldNames[schema.GetName()] = nil
|
||||
}
|
||||
}
|
||||
|
||||
// check redundant file
|
||||
fileNames := make(map[string]interface{})
|
||||
for _, filePath := range filePaths {
|
||||
name, _ := GetFileNameAndExt(filePath)
|
||||
fileNames[name] = nil
|
||||
_, ok := requiredFieldNames[name]
|
||||
if !ok {
|
||||
log.Warn("Numpy parser: the file has no corresponding field in collection", zap.String("fieldName", name))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("the file '%s' has no corresponding field in collection", filePath))
|
||||
}
|
||||
}
|
||||
|
||||
// check missed file
|
||||
for name := range requiredFieldNames {
|
||||
if name == dynamicFieldName {
|
||||
// dynamic schema field file is not required
|
||||
continue
|
||||
}
|
||||
_, ok := fileNames[name]
|
||||
if !ok {
|
||||
log.Warn("Numpy parser: there is no file corresponding to field", zap.String("fieldName", name))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("there is no file corresponding to field '%s'", name))
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// createReaders open the files and verify file header
|
||||
func (p *NumpyParser) createReaders(filePaths []string) ([]*NumpyColumnReader, error) {
|
||||
readers := make([]*NumpyColumnReader, 0)
|
||||
|
||||
for _, filePath := range filePaths {
|
||||
fileName, _ := GetFileNameAndExt(filePath)
|
||||
|
||||
// check existence of the target field
|
||||
var schema *schemapb.FieldSchema
|
||||
for i := 0; i < len(p.collectionInfo.Schema.Fields); i++ {
|
||||
tmpSchema := p.collectionInfo.Schema.Fields[i]
|
||||
if tmpSchema.GetName() == fileName {
|
||||
schema = tmpSchema
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if schema == nil {
|
||||
log.Warn("Numpy parser: the field is not found in collection schema", zap.String("fileName", fileName))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the field name '%s' is not found in collection schema", fileName))
|
||||
}
|
||||
|
||||
file, err := p.chunkManager.Reader(p.ctx, filePath)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read the file", zap.String("filePath", filePath), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read the file '%s', error: %s", filePath, err.Error()))
|
||||
}
|
||||
|
||||
adapter, err := NewNumpyAdapter(file)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read the file header", zap.String("filePath", filePath), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read the file header '%s', error: %s", filePath, err.Error()))
|
||||
}
|
||||
|
||||
if file == nil || adapter == nil {
|
||||
log.Warn("Numpy parser: failed to open file", zap.String("filePath", filePath))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to open file '%s'", filePath))
|
||||
}
|
||||
|
||||
dim, _ := getFieldDimension(schema)
|
||||
columnReader := &NumpyColumnReader{
|
||||
fieldName: schema.GetName(),
|
||||
fieldID: schema.GetFieldID(),
|
||||
dataType: schema.GetDataType(),
|
||||
dimension: dim,
|
||||
file: file,
|
||||
reader: adapter,
|
||||
}
|
||||
|
||||
// the validation method only check the file header information
|
||||
err = p.validateHeader(columnReader)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
readers = append(readers, columnReader)
|
||||
}
|
||||
|
||||
// row count of each file should be equal
|
||||
if len(readers) > 0 {
|
||||
firstReader := readers[0]
|
||||
rowCount := firstReader.rowCount
|
||||
for i := 1; i < len(readers); i++ {
|
||||
compareReader := readers[i]
|
||||
if rowCount != compareReader.rowCount {
|
||||
log.Warn("Numpy parser: the row count of files are not equal",
|
||||
zap.String("firstFile", firstReader.fieldName), zap.Int("firstRowCount", firstReader.rowCount),
|
||||
zap.String("compareFile", compareReader.fieldName), zap.Int("compareRowCount", compareReader.rowCount))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the row count(%d) of file '%s.npy' is not equal to row count(%d) of file '%s.npy'",
|
||||
firstReader.rowCount, firstReader.fieldName, compareReader.rowCount, compareReader.fieldName))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return readers, nil
|
||||
}
|
||||
|
||||
// validateHeader is to verify numpy file header, file header information should match field's schema
|
||||
func (p *NumpyParser) validateHeader(columnReader *NumpyColumnReader) error {
|
||||
if columnReader == nil || columnReader.reader == nil {
|
||||
log.Warn("Numpy parser: numpy reader is nil")
|
||||
return merr.WrapErrImportFailed("numpy adapter is nil")
|
||||
}
|
||||
|
||||
elementType := columnReader.reader.GetType()
|
||||
shape := columnReader.reader.GetShape()
|
||||
// if user only save an element in a numpy file, the shape list will be empty
|
||||
if len(shape) == 0 {
|
||||
log.Warn("Numpy parser: the content stored in numpy file is not valid numpy array",
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("the content stored in numpy file is not valid numpy array for field '%s'", columnReader.fieldName))
|
||||
}
|
||||
columnReader.rowCount = shape[0]
|
||||
|
||||
// 1. field data type should be consist to numpy data type
|
||||
// 2. vector field dimension should be consist to numpy shape
|
||||
if schemapb.DataType_FloatVector == columnReader.dataType {
|
||||
// float32/float64 numpy file can be used for float vector file, 2 reasons:
|
||||
// 1. for float vector, we support float32 and float64 numpy file because python float value is 64 bit
|
||||
// 2. for float64 numpy file, the performance is worse than float32 numpy file
|
||||
if elementType != schemapb.DataType_Float && elementType != schemapb.DataType_Double {
|
||||
log.Warn("Numpy parser: illegal data type of numpy file for float vector field", zap.Any("dataType", elementType),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal data type %s of numpy file for float vector field '%s'", getTypeName(elementType),
|
||||
columnReader.fieldName))
|
||||
}
|
||||
|
||||
// vector field, the shape should be 2
|
||||
if len(shape) != 2 {
|
||||
log.Warn("Numpy parser: illegal shape of numpy file for float vector field, shape should be 2", zap.Int("shape", len(shape)),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal shape %d of numpy file for float vector field '%s', shape should be 2", shape,
|
||||
columnReader.fieldName))
|
||||
}
|
||||
|
||||
if shape[1] != columnReader.dimension {
|
||||
log.Warn("Numpy parser: illegal dimension of numpy file for float vector field", zap.String("fieldName", columnReader.fieldName),
|
||||
zap.Int("numpyDimension", shape[1]), zap.Int("fieldDimension", columnReader.dimension))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal dimension %d of numpy file for float vector field '%s', dimension should be %d",
|
||||
shape[1], columnReader.fieldName, columnReader.dimension))
|
||||
}
|
||||
} else if schemapb.DataType_BinaryVector == columnReader.dataType {
|
||||
if elementType != schemapb.DataType_BinaryVector {
|
||||
log.Warn("Numpy parser: illegal data type of numpy file for binary vector field", zap.Any("dataType", elementType),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal data type %s of numpy file for binary vector field '%s'", getTypeName(elementType),
|
||||
columnReader.fieldName))
|
||||
}
|
||||
|
||||
// vector field, the shape should be 2
|
||||
if len(shape) != 2 {
|
||||
log.Warn("Numpy parser: illegal shape of numpy file for binary vector field, shape should be 2", zap.Int("shape", len(shape)),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal shape %d of numpy file for binary vector field '%s', shape should be 2", shape,
|
||||
columnReader.fieldName))
|
||||
}
|
||||
|
||||
if shape[1] != columnReader.dimension/8 {
|
||||
log.Warn("Numpy parser: illegal dimension of numpy file for float vector field", zap.String("fieldName", columnReader.fieldName),
|
||||
zap.Int("numpyDimension", shape[1]*8), zap.Int("fieldDimension", columnReader.dimension))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal dimension %d of numpy file for binary vector field '%s', dimension should be %d",
|
||||
shape[1]*8, columnReader.fieldName, columnReader.dimension))
|
||||
}
|
||||
} else {
|
||||
// JSON field and VARCHAR field are using string type numpy
|
||||
// legal input if columnReader.dataType is JSON and elementType is VARCHAR
|
||||
if elementType != schemapb.DataType_VarChar && columnReader.dataType != schemapb.DataType_JSON {
|
||||
if elementType != columnReader.dataType {
|
||||
log.Warn("Numpy parser: illegal data type of numpy file for scalar field", zap.Any("numpyDataType", elementType),
|
||||
zap.String("fieldName", columnReader.fieldName), zap.Any("fieldDataType", columnReader.dataType))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal data type %s of numpy file for scalar field '%s' with type %s",
|
||||
getTypeName(elementType), columnReader.fieldName, getTypeName(columnReader.dataType)))
|
||||
}
|
||||
}
|
||||
|
||||
// scalar field, the shape should be 1
|
||||
if len(shape) != 1 {
|
||||
log.Warn("Numpy parser: illegal shape of numpy file for scalar field, shape should be 1", zap.Int("shape", len(shape)),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("illegal shape %d of numpy file for scalar field '%s', shape should be 1", shape, columnReader.fieldName))
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// calcRowCountPerBlock calculates a proper value for a batch row count to read file
|
||||
func (p *NumpyParser) calcRowCountPerBlock() (int64, error) {
|
||||
sizePerRecord, err := typeutil.EstimateSizePerRecord(p.collectionInfo.Schema)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to estimate size of each row", zap.Error(err))
|
||||
return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to estimate size of each row: %s", err.Error()))
|
||||
}
|
||||
|
||||
if sizePerRecord <= 0 {
|
||||
log.Warn("Numpy parser: failed to estimate size of each row, the collection schema might be empty")
|
||||
return 0, merr.WrapErrImportFailed("failed to estimate size of each row: the collection schema might be empty")
|
||||
}
|
||||
|
||||
// the sizePerRecord is estimate value, if the schema contains varchar field, the value is not accurate
|
||||
// we will read data block by block, by default, each block size is 16MB
|
||||
// rowCountPerBlock is the estimated row count for a block
|
||||
rowCountPerBlock := p.blockSize / int64(sizePerRecord)
|
||||
if rowCountPerBlock <= 0 {
|
||||
rowCountPerBlock = 1 // make sure the value is positive
|
||||
}
|
||||
|
||||
log.Info("Numper parser: calculate row count per block to read file", zap.Int64("rowCountPerBlock", rowCountPerBlock),
|
||||
zap.Int64("blockSize", p.blockSize), zap.Int("sizePerRecord", sizePerRecord))
|
||||
return rowCountPerBlock, nil
|
||||
}
|
||||
|
||||
// consume method reads numpy data section into a storage.FieldData
|
||||
// please note it will require a large memory block(the memory size is almost equal to numpy file size)
|
||||
func (p *NumpyParser) consume(columnReaders []*NumpyColumnReader) error {
|
||||
rowCountPerBlock, err := p.calcRowCountPerBlock()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
updateProgress := func(readRowCount int) {
|
||||
if p.updateProgressFunc != nil && len(columnReaders) != 0 && columnReaders[0].rowCount > 0 {
|
||||
percent := (readRowCount * ProgressValueForPersist) / columnReaders[0].rowCount
|
||||
log.Debug("Numper parser: working progress", zap.Int("readRowCount", readRowCount),
|
||||
zap.Int("totalRowCount", columnReaders[0].rowCount), zap.Int("percent", percent))
|
||||
p.updateProgressFunc(int64(percent))
|
||||
}
|
||||
}
|
||||
|
||||
// prepare shards
|
||||
shards := make([]ShardData, 0, p.collectionInfo.ShardNum)
|
||||
for i := 0; i < int(p.collectionInfo.ShardNum); i++ {
|
||||
shardData := initShardData(p.collectionInfo.Schema, p.collectionInfo.PartitionIDs)
|
||||
if shardData == nil {
|
||||
log.Warn("Numper parser: failed to initialize FieldData list")
|
||||
return merr.WrapErrImportFailed("failed to initialize FieldData list")
|
||||
}
|
||||
shards = append(shards, shardData)
|
||||
}
|
||||
tr := timerecord.NewTimeRecorder("consume performance")
|
||||
defer tr.Elapse("end")
|
||||
// read data from files, batch by batch
|
||||
totalRead := 0
|
||||
for {
|
||||
readRowCount := 0
|
||||
segmentData := make(BlockData)
|
||||
for _, reader := range columnReaders {
|
||||
fieldData, err := p.readData(reader, int(rowCountPerBlock))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if readRowCount == 0 {
|
||||
readRowCount = fieldData.RowNum()
|
||||
} else if readRowCount != fieldData.RowNum() {
|
||||
log.Warn("Numpy parser: data block's row count mismatch", zap.Int("firstBlockRowCount", readRowCount),
|
||||
zap.Int("thisBlockRowCount", fieldData.RowNum()), zap.Int64("rowCountPerBlock", rowCountPerBlock))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("data block's row count mismatch: %d vs %d", readRowCount, fieldData.RowNum()))
|
||||
}
|
||||
|
||||
segmentData[reader.fieldID] = fieldData
|
||||
}
|
||||
|
||||
// nothing to read
|
||||
if readRowCount == 0 {
|
||||
break
|
||||
}
|
||||
totalRead += readRowCount
|
||||
updateProgress(totalRead)
|
||||
tr.Record("readData")
|
||||
// split data to shards
|
||||
p.autoIDRange, err = splitFieldsData(p.collectionInfo, segmentData, shards, p.rowIDAllocator)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
tr.Record("splitFieldsData")
|
||||
// when the estimated size is close to blockSize, save to binlog
|
||||
err = tryFlushBlocks(p.ctx, shards, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, Params.DataNodeCfg.BulkInsertMaxMemorySize.GetAsInt64(), false)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
tr.Record("tryFlushBlocks")
|
||||
}
|
||||
|
||||
// force flush at the end
|
||||
return tryFlushBlocks(p.ctx, shards, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, Params.DataNodeCfg.BulkInsertMaxMemorySize.GetAsInt64(), true)
|
||||
}
|
||||
|
||||
// readData method reads numpy data section into a storage.FieldData
|
||||
func (p *NumpyParser) readData(columnReader *NumpyColumnReader, rowCount int) (storage.FieldData, error) {
|
||||
switch columnReader.dataType {
|
||||
case schemapb.DataType_Bool:
|
||||
data, err := columnReader.reader.ReadBool(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read bool array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read bool array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.BoolFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int8:
|
||||
data, err := columnReader.reader.ReadInt8(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read int8 array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int8 array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.Int8FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int16:
|
||||
data, err := columnReader.reader.ReadInt16(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to int16 array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int16 array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.Int16FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int32:
|
||||
data, err := columnReader.reader.ReadInt32(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read int32 array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int32 array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.Int32FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int64:
|
||||
data, err := columnReader.reader.ReadInt64(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read int64 array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read int64 array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.Int64FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Float:
|
||||
data, err := columnReader.reader.ReadFloat32(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read float array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float array: %s", err.Error()))
|
||||
}
|
||||
|
||||
err = typeutil.VerifyFloats32(data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: illegal value in float array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("illegal value in float array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.FloatFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Double:
|
||||
data, err := columnReader.reader.ReadFloat64(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read double array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read double array: %s", err.Error()))
|
||||
}
|
||||
|
||||
err = typeutil.VerifyFloats64(data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: illegal value in double array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("illegal value in double array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.DoubleFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_VarChar:
|
||||
data, err := columnReader.reader.ReadString(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read varchar array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read varchar array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.StringFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_JSON:
|
||||
// JSON field read data from string array numpy
|
||||
data, err := columnReader.reader.ReadString(rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read json string array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read json string array: %s", err.Error()))
|
||||
}
|
||||
|
||||
byteArr := make([][]byte, 0)
|
||||
for _, str := range data {
|
||||
var dummy interface{}
|
||||
err := json.Unmarshal([]byte(str), &dummy)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: illegal string value for JSON field",
|
||||
zap.String("value", str), zap.String("FieldName", columnReader.fieldName), zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to parse value '%v' for JSON field '%s', error: %v",
|
||||
str, columnReader.fieldName, err))
|
||||
}
|
||||
byteArr = append(byteArr, []byte(str))
|
||||
}
|
||||
|
||||
return &storage.JSONFieldData{
|
||||
Data: byteArr,
|
||||
}, nil
|
||||
case schemapb.DataType_BinaryVector:
|
||||
data, err := columnReader.reader.ReadUint8(rowCount * (columnReader.dimension / 8))
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read binary vector array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read binary vector array: %s", err.Error()))
|
||||
}
|
||||
|
||||
return &storage.BinaryVectorFieldData{
|
||||
Data: data,
|
||||
Dim: columnReader.dimension,
|
||||
}, nil
|
||||
case schemapb.DataType_FloatVector:
|
||||
// float32/float64 numpy file can be used for float vector file, 2 reasons:
|
||||
// 1. for float vector, we support float32 and float64 numpy file because python float value is 64 bit
|
||||
// 2. for float64 numpy file, the performance is worse than float32 numpy file
|
||||
elementType := columnReader.reader.GetType()
|
||||
|
||||
var data []float32
|
||||
var err error
|
||||
if elementType == schemapb.DataType_Float {
|
||||
data, err = columnReader.reader.ReadFloat32(rowCount * columnReader.dimension)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read float vector array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float vector array: %s", err.Error()))
|
||||
}
|
||||
|
||||
err = typeutil.VerifyFloats32(data)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: illegal value in float vector array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("illegal value in float vector array: %s", err.Error()))
|
||||
}
|
||||
} else if elementType == schemapb.DataType_Double {
|
||||
data = make([]float32, 0, columnReader.rowCount)
|
||||
data64, err := columnReader.reader.ReadFloat64(rowCount * columnReader.dimension)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: failed to read float vector array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("failed to read float vector array: %s", err.Error()))
|
||||
}
|
||||
|
||||
for _, f64 := range data64 {
|
||||
err = typeutil.VerifyFloat(f64)
|
||||
if err != nil {
|
||||
log.Warn("Numpy parser: illegal value in float vector array", zap.Error(err))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("illegal value in float vector array: %s", err.Error()))
|
||||
}
|
||||
|
||||
data = append(data, float32(f64))
|
||||
}
|
||||
}
|
||||
|
||||
return &storage.FloatVectorFieldData{
|
||||
Data: data,
|
||||
Dim: columnReader.dimension,
|
||||
}, nil
|
||||
default:
|
||||
log.Warn("Numpy parser: unsupported data type of field", zap.Any("dataType", columnReader.dataType),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("unsupported data type %s of field '%s'", getTypeName(columnReader.dataType),
|
||||
columnReader.fieldName))
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -1,311 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/apache/arrow/go/v12/arrow"
|
||||
"github.com/apache/arrow/go/v12/arrow/array"
|
||||
"github.com/apache/arrow/go/v12/parquet/pqarrow"
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/exp/constraints"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type ParquetColumnReader struct {
|
||||
fieldName string
|
||||
fieldID int64
|
||||
columnIndex int
|
||||
// columnSchema *parquet.SchemaElement
|
||||
dataType schemapb.DataType
|
||||
elementType schemapb.DataType
|
||||
columnReader *pqarrow.ColumnReader
|
||||
dimension int
|
||||
}
|
||||
|
||||
func ReadBoolData(pcr *ParquetColumnReader, count int64) ([]bool, error) {
|
||||
chunked, err := pcr.columnReader.NextBatch(count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
data := make([]bool, 0, count)
|
||||
for _, chunk := range chunked.Chunks() {
|
||||
dataNums := chunk.Data().Len()
|
||||
boolReader, ok := chunk.(*array.Boolean)
|
||||
if !ok {
|
||||
log.Warn("the column data in parquet is not bool", zap.String("fieldName", pcr.fieldName), zap.String("actual type", chunk.DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data in parquet is not bool of field: %s, but: %s", pcr.fieldName, chunk.DataType().Name()))
|
||||
}
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, boolReader.Value(i))
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func ReadIntegerOrFloatData[T constraints.Integer | constraints.Float](pcr *ParquetColumnReader, count int64) ([]T, error) {
|
||||
chunked, err := pcr.columnReader.NextBatch(count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
data := make([]T, 0, count)
|
||||
for _, chunk := range chunked.Chunks() {
|
||||
dataNums := chunk.Data().Len()
|
||||
switch chunk.DataType().ID() {
|
||||
case arrow.INT8:
|
||||
int8Reader := chunk.(*array.Int8)
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, T(int8Reader.Value(i)))
|
||||
}
|
||||
case arrow.INT16:
|
||||
int16Reader := chunk.(*array.Int16)
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, T(int16Reader.Value(i)))
|
||||
}
|
||||
case arrow.INT32:
|
||||
int32Reader := chunk.(*array.Int32)
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, T(int32Reader.Value(i)))
|
||||
}
|
||||
case arrow.INT64:
|
||||
int64Reader := chunk.(*array.Int64)
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, T(int64Reader.Value(i)))
|
||||
}
|
||||
case arrow.FLOAT32:
|
||||
float32Reader := chunk.(*array.Float32)
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, T(float32Reader.Value(i)))
|
||||
}
|
||||
case arrow.FLOAT64:
|
||||
float64Reader := chunk.(*array.Float64)
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, T(float64Reader.Value(i)))
|
||||
}
|
||||
default:
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data type is not integer, neither float, but: %s", chunk.DataType().Name()))
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func ReadStringData(pcr *ParquetColumnReader, count int64) ([]string, error) {
|
||||
chunked, err := pcr.columnReader.NextBatch(count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
data := make([]string, 0, count)
|
||||
for _, chunk := range chunked.Chunks() {
|
||||
dataNums := chunk.Data().Len()
|
||||
stringReader, ok := chunk.(*array.String)
|
||||
if !ok {
|
||||
log.Warn("the column data in parquet is not string", zap.String("fieldName", pcr.fieldName), zap.String("actual type", chunk.DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data in parquet is not string of field: %s, but: %s", pcr.fieldName, chunk.DataType().Name()))
|
||||
}
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, stringReader.Value(i))
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func ReadBinaryData(pcr *ParquetColumnReader, count int64) ([]byte, error) {
|
||||
chunked, err := pcr.columnReader.NextBatch(count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
data := make([]byte, 0, count)
|
||||
for _, chunk := range chunked.Chunks() {
|
||||
dataNums := chunk.Data().Len()
|
||||
switch chunk.DataType().ID() {
|
||||
case arrow.BINARY:
|
||||
binaryReader := chunk.(*array.Binary)
|
||||
for i := 0; i < dataNums; i++ {
|
||||
data = append(data, binaryReader.Value(i)...)
|
||||
}
|
||||
case arrow.LIST:
|
||||
listReader := chunk.(*array.List)
|
||||
if !checkVectorIsRegular(listReader.Offsets(), pcr.dimension, true) {
|
||||
log.Warn("Parquet parser: binary vector is irregular", zap.Int("dim", pcr.dimension), zap.Int32s("offsets", listReader.Offsets()))
|
||||
return nil, merr.WrapErrImportFailed("binary vector is irregular")
|
||||
}
|
||||
uint8Reader, ok := listReader.ListValues().(*array.Uint8)
|
||||
if !ok {
|
||||
log.Warn("the column element data of array in parquet is not binary", zap.String("fieldName", pcr.fieldName))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column element data of array in parquet is not binary: %s", pcr.fieldName))
|
||||
}
|
||||
for i := 0; i < uint8Reader.Len(); i++ {
|
||||
data = append(data, uint8Reader.Value(i))
|
||||
}
|
||||
default:
|
||||
log.Warn("the column element data of array in parquet is not binary", zap.String("fieldName", pcr.fieldName), zap.String("actual data type", chunk.DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column element data of array in parquet is not binary: %s, it's: %s", pcr.fieldName, chunk.DataType().Name()))
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func checkVectorIsRegular(offsets []int32, dim int, isBinary bool) bool {
|
||||
if len(offsets) < 1 {
|
||||
return false
|
||||
}
|
||||
if isBinary {
|
||||
dim = dim / 8
|
||||
}
|
||||
start := offsets[0]
|
||||
for i := 1; i < len(offsets); i++ {
|
||||
if offsets[i]-start != int32(dim) {
|
||||
return false
|
||||
}
|
||||
start = offsets[i]
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func ReadBoolArrayData(pcr *ParquetColumnReader, count int64) ([][]bool, error) {
|
||||
chunked, err := pcr.columnReader.NextBatch(count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
data := make([][]bool, 0, count)
|
||||
for _, chunk := range chunked.Chunks() {
|
||||
listReader, ok := chunk.(*array.List)
|
||||
if !ok {
|
||||
log.Warn("the column data in parquet is not list", zap.String("fieldName", pcr.fieldName), zap.String("actual type", chunk.DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data in parquet is not list of field: %s, but: %s", pcr.fieldName, chunk.DataType().Name()))
|
||||
}
|
||||
boolReader, ok := listReader.ListValues().(*array.Boolean)
|
||||
if !ok {
|
||||
log.Warn("the column data in parquet is not bool array", zap.String("fieldName", pcr.fieldName),
|
||||
zap.String("actual type", listReader.ListValues().DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data in parquet is not bool array of field: %s, but: %s list", pcr.fieldName, listReader.ListValues().DataType().Name()))
|
||||
}
|
||||
offsets := listReader.Offsets()
|
||||
for i := 1; i < len(offsets); i++ {
|
||||
start, end := offsets[i-1], offsets[i]
|
||||
elementData := make([]bool, 0, end-start)
|
||||
for j := start; j < end; j++ {
|
||||
elementData = append(elementData, boolReader.Value(int(j)))
|
||||
}
|
||||
data = append(data, elementData)
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func ReadIntegerOrFloatArrayData[T constraints.Integer | constraints.Float](pcr *ParquetColumnReader, count int64) ([][]T, error) {
|
||||
chunked, err := pcr.columnReader.NextBatch(count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
data := make([][]T, 0, count)
|
||||
|
||||
getDataFunc := func(offsets []int32, getValue func(int) T) {
|
||||
for i := 1; i < len(offsets); i++ {
|
||||
start, end := offsets[i-1], offsets[i]
|
||||
elementData := make([]T, 0, end-start)
|
||||
for j := start; j < end; j++ {
|
||||
elementData = append(elementData, getValue(int(j)))
|
||||
}
|
||||
data = append(data, elementData)
|
||||
}
|
||||
}
|
||||
for _, chunk := range chunked.Chunks() {
|
||||
listReader, ok := chunk.(*array.List)
|
||||
if !ok {
|
||||
log.Warn("the column data in parquet is not list", zap.String("fieldName", pcr.fieldName), zap.String("actual type", chunk.DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data in parquet is not list of field: %s, but: %s", pcr.fieldName, chunk.DataType().Name()))
|
||||
}
|
||||
offsets := listReader.Offsets()
|
||||
if typeutil.IsVectorType(pcr.dataType) && !checkVectorIsRegular(offsets, pcr.dimension, pcr.dataType == schemapb.DataType_BinaryVector) {
|
||||
log.Warn("Parquet parser: float vector is irregular", zap.Int("dim", pcr.dimension), zap.Int32s("offsets", listReader.Offsets()))
|
||||
return nil, merr.WrapErrImportFailed("float vector is irregular")
|
||||
}
|
||||
valueReader := listReader.ListValues()
|
||||
switch valueReader.DataType().ID() {
|
||||
case arrow.INT8:
|
||||
int8Reader := valueReader.(*array.Int8)
|
||||
getDataFunc(offsets, func(i int) T {
|
||||
return T(int8Reader.Value(i))
|
||||
})
|
||||
case arrow.INT16:
|
||||
int16Reader := valueReader.(*array.Int16)
|
||||
getDataFunc(offsets, func(i int) T {
|
||||
return T(int16Reader.Value(i))
|
||||
})
|
||||
case arrow.INT32:
|
||||
int32Reader := valueReader.(*array.Int32)
|
||||
getDataFunc(offsets, func(i int) T {
|
||||
return T(int32Reader.Value(i))
|
||||
})
|
||||
case arrow.INT64:
|
||||
int64Reader := valueReader.(*array.Int64)
|
||||
getDataFunc(offsets, func(i int) T {
|
||||
return T(int64Reader.Value(i))
|
||||
})
|
||||
case arrow.FLOAT32:
|
||||
float32Reader := valueReader.(*array.Float32)
|
||||
getDataFunc(offsets, func(i int) T {
|
||||
return T(float32Reader.Value(i))
|
||||
})
|
||||
case arrow.FLOAT64:
|
||||
float64Reader := valueReader.(*array.Float64)
|
||||
getDataFunc(offsets, func(i int) T {
|
||||
return T(float64Reader.Value(i))
|
||||
})
|
||||
default:
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data type is not integer array, neither float array, but: %s", valueReader.DataType().Name()))
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func ReadStringArrayData(pcr *ParquetColumnReader, count int64) ([][]string, error) {
|
||||
chunked, err := pcr.columnReader.NextBatch(count)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
data := make([][]string, 0, count)
|
||||
for _, chunk := range chunked.Chunks() {
|
||||
listReader, ok := chunk.(*array.List)
|
||||
if !ok {
|
||||
log.Warn("the column data in parquet is not list", zap.String("fieldName", pcr.fieldName), zap.String("actual type", chunk.DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data in parquet is not list of field: %s, but: %s", pcr.fieldName, chunk.DataType().Name()))
|
||||
}
|
||||
stringReader, ok := listReader.ListValues().(*array.String)
|
||||
if !ok {
|
||||
log.Warn("the column data in parquet is not string array", zap.String("fieldName", pcr.fieldName),
|
||||
zap.String("actual type", listReader.ListValues().DataType().Name()))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("the column data in parquet is not string array of field: %s, but: %s list", pcr.fieldName, listReader.ListValues().DataType().Name()))
|
||||
}
|
||||
offsets := listReader.Offsets()
|
||||
for i := 1; i < len(offsets); i++ {
|
||||
start, end := offsets[i-1], offsets[i]
|
||||
elementData := make([]string, 0, end-start)
|
||||
for j := start; j < end; j++ {
|
||||
elementData = append(elementData, stringReader.Value(int(j)))
|
||||
}
|
||||
data = append(data, elementData)
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
|
@ -1,686 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package importutil
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
|
||||
"github.com/apache/arrow/go/v12/arrow"
|
||||
"github.com/apache/arrow/go/v12/arrow/memory"
|
||||
"github.com/apache/arrow/go/v12/parquet"
|
||||
"github.com/apache/arrow/go/v12/parquet/file"
|
||||
"github.com/apache/arrow/go/v12/parquet/pqarrow"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
// ParquetParser is analogous to the ParquetColumnReader, but for Parquet files
|
||||
type ParquetParser struct {
|
||||
ctx context.Context // for canceling parse process
|
||||
collectionInfo *CollectionInfo // collection details including schema
|
||||
rowIDAllocator *allocator.IDAllocator // autoid allocator
|
||||
blockSize int64 // maximum size of a read block(unit:byte)
|
||||
chunkManager storage.ChunkManager // storage interfaces to browse/read the files
|
||||
autoIDRange []int64 // auto-generated id range, for example: [1, 10, 20, 25] means id from 1 to 10 and 20 to 25
|
||||
callFlushFunc ImportFlushFunc // call back function to flush segment
|
||||
updateProgressFunc func(percent int64) // update working progress percent value
|
||||
columnMap map[string]*ParquetColumnReader
|
||||
reader *file.Reader
|
||||
fileReader *pqarrow.FileReader
|
||||
}
|
||||
|
||||
// NewParquetParser is helper function to create a ParquetParser
|
||||
func NewParquetParser(ctx context.Context,
|
||||
collectionInfo *CollectionInfo,
|
||||
idAlloc *allocator.IDAllocator,
|
||||
blockSize int64,
|
||||
chunkManager storage.ChunkManager,
|
||||
filePath string,
|
||||
flushFunc ImportFlushFunc,
|
||||
updateProgressFunc func(percent int64),
|
||||
) (*ParquetParser, error) {
|
||||
if collectionInfo == nil {
|
||||
log.Warn("Parquet parser: collection schema is nil")
|
||||
return nil, merr.WrapErrImportFailed("collection schema is nil")
|
||||
}
|
||||
|
||||
if idAlloc == nil {
|
||||
log.Warn("Parquet parser: id allocator is nil")
|
||||
return nil, merr.WrapErrImportFailed("id allocator is nil")
|
||||
}
|
||||
|
||||
if chunkManager == nil {
|
||||
log.Warn("Parquet parser: chunk manager pointer is nil")
|
||||
return nil, merr.WrapErrImportFailed("chunk manager pointer is nil")
|
||||
}
|
||||
|
||||
if flushFunc == nil {
|
||||
log.Warn("Parquet parser: flush function is nil")
|
||||
return nil, merr.WrapErrImportFailed("flush function is nil")
|
||||
}
|
||||
|
||||
cmReader, err := chunkManager.Reader(ctx, filePath)
|
||||
if err != nil {
|
||||
log.Warn("create chunk manager reader failed")
|
||||
return nil, err
|
||||
}
|
||||
|
||||
reader, err := file.NewParquetReader(cmReader, file.WithReadProps(&parquet.ReaderProperties{
|
||||
BufferSize: 32 * 1024 * 1024,
|
||||
BufferedStreamEnabled: true,
|
||||
}))
|
||||
if err != nil {
|
||||
log.Warn("create parquet reader failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
log.Info("create file reader done!", zap.Int("row group num", reader.NumRowGroups()), zap.Int64("num rows", reader.NumRows()))
|
||||
|
||||
fileReader, err := pqarrow.NewFileReader(reader, pqarrow.ArrowReadProperties{}, memory.DefaultAllocator)
|
||||
if err != nil {
|
||||
log.Warn("create arrow parquet file reader failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
parser := &ParquetParser{
|
||||
ctx: ctx,
|
||||
collectionInfo: collectionInfo,
|
||||
rowIDAllocator: idAlloc,
|
||||
blockSize: blockSize,
|
||||
chunkManager: chunkManager,
|
||||
autoIDRange: make([]int64, 0),
|
||||
callFlushFunc: flushFunc,
|
||||
updateProgressFunc: updateProgressFunc,
|
||||
columnMap: make(map[string]*ParquetColumnReader),
|
||||
fileReader: fileReader,
|
||||
reader: reader,
|
||||
}
|
||||
|
||||
return parser, nil
|
||||
}
|
||||
|
||||
func (p *ParquetParser) IDRange() []int64 {
|
||||
return p.autoIDRange
|
||||
}
|
||||
|
||||
// Parse is the function entry
|
||||
func (p *ParquetParser) Parse() error {
|
||||
err := p.createReaders()
|
||||
defer p.Close()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// read all data from the Parquet files
|
||||
err = p.consume()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *ParquetParser) checkFields() error {
|
||||
for _, field := range p.collectionInfo.Schema.GetFields() {
|
||||
if (field.GetIsPrimaryKey() && field.GetAutoID()) || field.GetIsDynamic() {
|
||||
continue
|
||||
}
|
||||
if _, ok := p.columnMap[field.GetName()]; !ok {
|
||||
log.Warn("there is no field in parquet file", zap.String("fieldName", field.GetName()))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("there is no field in parquet file of name: %s", field.GetName()))
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *ParquetParser) createReaders() error {
|
||||
schema, err := p.fileReader.Schema()
|
||||
if err != nil {
|
||||
log.Warn("can't schema from file", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
// The collection schema must be checked, so no errors will occur here.
|
||||
schemaHelper, _ := typeutil.CreateSchemaHelper(p.collectionInfo.Schema)
|
||||
parquetFields := schema.Fields()
|
||||
for i, field := range parquetFields {
|
||||
fieldSchema, err := schemaHelper.GetFieldFromName(field.Name)
|
||||
if err != nil {
|
||||
// TODO @cai.zhang: handle dynamic field
|
||||
log.Warn("the field is not in schema, if it's a dynamic field, please reformat data by bulk_writer", zap.String("fieldName", field.Name))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("the field: %s is not in schema, if it's a dynamic field, please reformat data by bulk_writer", field.Name))
|
||||
}
|
||||
if _, ok := p.columnMap[field.Name]; ok {
|
||||
log.Warn("there is multi field of fieldName", zap.String("fieldName", field.Name),
|
||||
zap.Ints("file fields indices", schema.FieldIndices(field.Name)))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("there is multi field of fieldName: %s", field.Name))
|
||||
}
|
||||
if fieldSchema.GetIsPrimaryKey() && fieldSchema.GetAutoID() {
|
||||
log.Warn("the field is primary key, and autoID is true, please remove it from file", zap.String("fieldName", field.Name))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("the field: %s is primary key, and autoID is true, please remove it from file", field.Name))
|
||||
}
|
||||
arrowType, isList := convertArrowSchemaToDataType(field, false)
|
||||
dataType := fieldSchema.GetDataType()
|
||||
if isList {
|
||||
if !typeutil.IsVectorType(dataType) && dataType != schemapb.DataType_Array {
|
||||
log.Warn("field schema is not match",
|
||||
zap.String("collection schema", dataType.String()),
|
||||
zap.String("file schema", field.Type.Name()))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("field schema is not match, collection field dataType: %s, file field dataType:%s", dataType.String(), field.Type.Name()))
|
||||
}
|
||||
if dataType == schemapb.DataType_Array {
|
||||
dataType = fieldSchema.GetElementType()
|
||||
}
|
||||
}
|
||||
if !isConvertible(arrowType, dataType, isList) {
|
||||
log.Warn("field schema is not match",
|
||||
zap.String("collection schema", dataType.String()),
|
||||
zap.String("file schema", field.Type.Name()))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("field schema is not match, collection field dataType: %s, file field dataType:%s", dataType.String(), field.Type.Name()))
|
||||
}
|
||||
// Here, the scalar column does not have a dim field,
|
||||
// and the dim type of the vector column must have been checked, so there is no error catch here.
|
||||
dim, _ := getFieldDimension(fieldSchema)
|
||||
parquetColumnReader := &ParquetColumnReader{
|
||||
fieldName: fieldSchema.GetName(),
|
||||
fieldID: fieldSchema.GetFieldID(),
|
||||
dataType: fieldSchema.GetDataType(),
|
||||
elementType: fieldSchema.GetElementType(),
|
||||
dimension: dim,
|
||||
}
|
||||
parquetColumnReader.columnIndex = i
|
||||
columnReader, err := p.fileReader.GetColumn(p.ctx, parquetColumnReader.columnIndex)
|
||||
if err != nil {
|
||||
log.Warn("get column reader failed", zap.String("fieldName", field.Name), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
parquetColumnReader.columnReader = columnReader
|
||||
p.columnMap[field.Name] = parquetColumnReader
|
||||
}
|
||||
if err = p.checkFields(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func convertArrowSchemaToDataType(field arrow.Field, isList bool) (schemapb.DataType, bool) {
|
||||
switch field.Type.ID() {
|
||||
case arrow.BOOL:
|
||||
return schemapb.DataType_Bool, false
|
||||
case arrow.UINT8:
|
||||
if isList {
|
||||
return schemapb.DataType_BinaryVector, false
|
||||
}
|
||||
return schemapb.DataType_None, false
|
||||
case arrow.INT8:
|
||||
return schemapb.DataType_Int8, false
|
||||
case arrow.INT16:
|
||||
return schemapb.DataType_Int16, false
|
||||
case arrow.INT32:
|
||||
return schemapb.DataType_Int32, false
|
||||
case arrow.INT64:
|
||||
return schemapb.DataType_Int64, false
|
||||
case arrow.FLOAT16:
|
||||
if isList {
|
||||
return schemapb.DataType_Float16Vector, false
|
||||
}
|
||||
return schemapb.DataType_None, false
|
||||
case arrow.FLOAT32:
|
||||
return schemapb.DataType_Float, false
|
||||
case arrow.FLOAT64:
|
||||
return schemapb.DataType_Double, false
|
||||
case arrow.STRING:
|
||||
return schemapb.DataType_VarChar, false
|
||||
case arrow.BINARY:
|
||||
return schemapb.DataType_BinaryVector, false
|
||||
case arrow.LIST:
|
||||
elementType, _ := convertArrowSchemaToDataType(field.Type.(*arrow.ListType).ElemField(), true)
|
||||
return elementType, true
|
||||
default:
|
||||
return schemapb.DataType_None, false
|
||||
}
|
||||
}
|
||||
|
||||
func isConvertible(src, dst schemapb.DataType, isList bool) bool {
|
||||
switch src {
|
||||
case schemapb.DataType_Bool:
|
||||
return typeutil.IsBoolType(dst)
|
||||
case schemapb.DataType_Int8:
|
||||
return typeutil.IsArithmetic(dst)
|
||||
case schemapb.DataType_Int16:
|
||||
return typeutil.IsArithmetic(dst) && dst != schemapb.DataType_Int8
|
||||
case schemapb.DataType_Int32:
|
||||
return typeutil.IsArithmetic(dst) && dst != schemapb.DataType_Int8 && dst != schemapb.DataType_Int16
|
||||
case schemapb.DataType_Int64:
|
||||
return typeutil.IsFloatingType(dst) || dst == schemapb.DataType_Int64
|
||||
case schemapb.DataType_Float:
|
||||
if isList && dst == schemapb.DataType_FloatVector {
|
||||
return true
|
||||
}
|
||||
return typeutil.IsFloatingType(dst)
|
||||
case schemapb.DataType_Double:
|
||||
if isList && dst == schemapb.DataType_FloatVector {
|
||||
return true
|
||||
}
|
||||
return dst == schemapb.DataType_Double
|
||||
case schemapb.DataType_String, schemapb.DataType_VarChar:
|
||||
return typeutil.IsStringType(dst) || typeutil.IsJSONType(dst)
|
||||
case schemapb.DataType_JSON:
|
||||
return typeutil.IsJSONType(dst)
|
||||
case schemapb.DataType_BinaryVector:
|
||||
return dst == schemapb.DataType_BinaryVector
|
||||
case schemapb.DataType_Float16Vector:
|
||||
return dst == schemapb.DataType_Float16Vector
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// Close closes the parquet file reader
|
||||
func (p *ParquetParser) Close() {
|
||||
p.reader.Close()
|
||||
}
|
||||
|
||||
// calcRowCountPerBlock calculates a proper value for a batch row count to read file
|
||||
func (p *ParquetParser) calcRowCountPerBlock() (int64, error) {
|
||||
sizePerRecord, err := typeutil.EstimateSizePerRecord(p.collectionInfo.Schema)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to estimate size of each row", zap.Error(err))
|
||||
return 0, merr.WrapErrImportFailed(fmt.Sprintf("failed to estimate size of each row: %s", err.Error()))
|
||||
}
|
||||
|
||||
if sizePerRecord <= 0 {
|
||||
log.Warn("Parquet parser: failed to estimate size of each row, the collection schema might be empty")
|
||||
return 0, merr.WrapErrImportFailed("failed to estimate size of each row: the collection schema might be empty")
|
||||
}
|
||||
|
||||
// the sizePerRecord is estimate value, if the schema contains varchar field, the value is not accurate
|
||||
// we will read data block by block, by default, each block size is 16MB
|
||||
// rowCountPerBlock is the estimated row count for a block
|
||||
rowCountPerBlock := p.blockSize / int64(sizePerRecord)
|
||||
if rowCountPerBlock <= 0 {
|
||||
rowCountPerBlock = 1 // make sure the value is positive
|
||||
}
|
||||
|
||||
log.Info("Parquet parser: calculate row count per block to read file", zap.Int64("rowCountPerBlock", rowCountPerBlock),
|
||||
zap.Int64("blockSize", p.blockSize), zap.Int("sizePerRecord", sizePerRecord))
|
||||
return rowCountPerBlock, nil
|
||||
}
|
||||
|
||||
// consume method reads Parquet data section into a storage.FieldData
|
||||
// please note it will require a large memory block(the memory size is almost equal to Parquet file size)
|
||||
func (p *ParquetParser) consume() error {
|
||||
rowCountPerBlock, err := p.calcRowCountPerBlock()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
updateProgress := func(readRowCount int64) {
|
||||
if p.updateProgressFunc != nil && p.reader != nil && p.reader.NumRows() > 0 {
|
||||
percent := (readRowCount * ProgressValueForPersist) / p.reader.NumRows()
|
||||
log.Info("Parquet parser: working progress", zap.Int64("readRowCount", readRowCount),
|
||||
zap.Int64("totalRowCount", p.reader.NumRows()), zap.Int64("percent", percent))
|
||||
p.updateProgressFunc(percent)
|
||||
}
|
||||
}
|
||||
|
||||
// prepare shards
|
||||
shards := make([]ShardData, 0, p.collectionInfo.ShardNum)
|
||||
for i := 0; i < int(p.collectionInfo.ShardNum); i++ {
|
||||
shardData := initShardData(p.collectionInfo.Schema, p.collectionInfo.PartitionIDs)
|
||||
if shardData == nil {
|
||||
log.Warn("Parquet parser: failed to initialize FieldData list")
|
||||
return merr.WrapErrImportFailed("failed to initialize FieldData list")
|
||||
}
|
||||
shards = append(shards, shardData)
|
||||
}
|
||||
tr := timerecord.NewTimeRecorder("consume performance")
|
||||
defer tr.Elapse("end")
|
||||
// read data from files, batch by batch
|
||||
totalRead := 0
|
||||
for {
|
||||
readRowCount := 0
|
||||
segmentData := make(BlockData)
|
||||
for _, reader := range p.columnMap {
|
||||
fieldData, err := p.readData(reader, rowCountPerBlock)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if readRowCount == 0 {
|
||||
readRowCount = fieldData.RowNum()
|
||||
} else if readRowCount != fieldData.RowNum() {
|
||||
log.Warn("Parquet parser: data block's row count mismatch", zap.Int("firstBlockRowCount", readRowCount),
|
||||
zap.Int("thisBlockRowCount", fieldData.RowNum()), zap.Int64("rowCountPerBlock", rowCountPerBlock),
|
||||
zap.String("current field", reader.fieldName))
|
||||
return merr.WrapErrImportFailed(fmt.Sprintf("data block's row count mismatch: %d vs %d", readRowCount, fieldData.RowNum()))
|
||||
}
|
||||
|
||||
segmentData[reader.fieldID] = fieldData
|
||||
}
|
||||
|
||||
// nothing to read
|
||||
if readRowCount == 0 {
|
||||
break
|
||||
}
|
||||
totalRead += readRowCount
|
||||
updateProgress(int64(totalRead))
|
||||
tr.Record("readData")
|
||||
// split data to shards
|
||||
p.autoIDRange, err = splitFieldsData(p.collectionInfo, segmentData, shards, p.rowIDAllocator)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
tr.Record("splitFieldsData")
|
||||
// when the estimated size is close to blockSize, save to binlog
|
||||
err = tryFlushBlocks(p.ctx, shards, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, Params.DataNodeCfg.BulkInsertMaxMemorySize.GetAsInt64(), false)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
tr.Record("tryFlushBlocks")
|
||||
}
|
||||
|
||||
// force flush at the end
|
||||
return tryFlushBlocks(p.ctx, shards, p.collectionInfo.Schema, p.callFlushFunc, p.blockSize, Params.DataNodeCfg.BulkInsertMaxMemorySize.GetAsInt64(), true)
|
||||
}
|
||||
|
||||
// readData method reads Parquet data section into a storage.FieldData
|
||||
func (p *ParquetParser) readData(columnReader *ParquetColumnReader, rowCount int64) (storage.FieldData, error) {
|
||||
switch columnReader.dataType {
|
||||
case schemapb.DataType_Bool:
|
||||
data, err := ReadBoolData(columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read bool array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.BoolFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int8:
|
||||
data, err := ReadIntegerOrFloatData[int8](columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read int8 array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.Int8FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int16:
|
||||
data, err := ReadIntegerOrFloatData[int16](columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to int16 array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.Int16FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int32:
|
||||
data, err := ReadIntegerOrFloatData[int32](columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read int32 array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.Int32FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Int64:
|
||||
data, err := ReadIntegerOrFloatData[int64](columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read int64 array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.Int64FieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Float:
|
||||
data, err := ReadIntegerOrFloatData[float32](columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read float array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = typeutil.VerifyFloats32(data)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: illegal value in float array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.FloatFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_Double:
|
||||
data, err := ReadIntegerOrFloatData[float64](columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read double array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = typeutil.VerifyFloats64(data)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: illegal value in double array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.DoubleFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_VarChar, schemapb.DataType_String:
|
||||
data, err := ReadStringData(columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read varchar array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.StringFieldData{
|
||||
Data: data,
|
||||
}, nil
|
||||
case schemapb.DataType_JSON:
|
||||
// JSON field read data from string array Parquet
|
||||
data, err := ReadStringData(columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read json string array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
byteArr := make([][]byte, 0)
|
||||
for _, str := range data {
|
||||
var dummy interface{}
|
||||
err := json.Unmarshal([]byte(str), &dummy)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: illegal string value for JSON field",
|
||||
zap.String("value", str), zap.String("fieldName", columnReader.fieldName), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
byteArr = append(byteArr, []byte(str))
|
||||
}
|
||||
|
||||
return &storage.JSONFieldData{
|
||||
Data: byteArr,
|
||||
}, nil
|
||||
case schemapb.DataType_BinaryVector:
|
||||
binaryData, err := ReadBinaryData(columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read binary vector array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &storage.BinaryVectorFieldData{
|
||||
Data: binaryData,
|
||||
Dim: columnReader.dimension,
|
||||
}, nil
|
||||
case schemapb.DataType_FloatVector:
|
||||
arrayData, err := ReadIntegerOrFloatArrayData[float32](columnReader, rowCount)
|
||||
if err != nil {
|
||||
log.Warn("Parquet parser: failed to read float vector array", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
data := make([]float32, 0, len(arrayData)*columnReader.dimension)
|
||||
for _, arr := range arrayData {
|
||||
data = append(data, arr...)
|
||||
}
|
||||
|
||||
return &storage.FloatVectorFieldData{
|
||||
Data: data,
|
||||
Dim: columnReader.dimension,
|
||||
}, nil
|
||||
case schemapb.DataType_Array:
|
||||
data := make([]*schemapb.ScalarField, 0, rowCount)
|
||||
switch columnReader.elementType {
|
||||
case schemapb.DataType_Bool:
|
||||
boolArray, err := ReadBoolArrayData(columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range boolArray {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_BoolData{
|
||||
BoolData: &schemapb.BoolArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
case schemapb.DataType_Int8:
|
||||
int8Array, err := ReadIntegerOrFloatArrayData[int32](columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range int8Array {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
case schemapb.DataType_Int16:
|
||||
int16Array, err := ReadIntegerOrFloatArrayData[int32](columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range int16Array {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
case schemapb.DataType_Int32:
|
||||
int32Array, err := ReadIntegerOrFloatArrayData[int32](columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range int32Array {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
case schemapb.DataType_Int64:
|
||||
int64Array, err := ReadIntegerOrFloatArrayData[int64](columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range int64Array {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_LongData{
|
||||
LongData: &schemapb.LongArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
case schemapb.DataType_Float:
|
||||
float32Array, err := ReadIntegerOrFloatArrayData[float32](columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range float32Array {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_FloatData{
|
||||
FloatData: &schemapb.FloatArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
case schemapb.DataType_Double:
|
||||
float64Array, err := ReadIntegerOrFloatArrayData[float64](columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range float64Array {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_DoubleData{
|
||||
DoubleData: &schemapb.DoubleArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
case schemapb.DataType_VarChar, schemapb.DataType_String:
|
||||
stringArray, err := ReadStringArrayData(columnReader, rowCount)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for _, elementArray := range stringArray {
|
||||
data = append(data, &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_StringData{
|
||||
StringData: &schemapb.StringArray{
|
||||
Data: elementArray,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
default:
|
||||
log.Warn("unsupported element type", zap.String("element type", columnReader.elementType.String()),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("unsupported data type: %s of array field: %s", columnReader.elementType.String(), columnReader.fieldName))
|
||||
}
|
||||
return &storage.ArrayFieldData{
|
||||
ElementType: columnReader.elementType,
|
||||
Data: data,
|
||||
}, nil
|
||||
default:
|
||||
log.Warn("Parquet parser: unsupported data type of field",
|
||||
zap.String("dataType", columnReader.dataType.String()),
|
||||
zap.String("fieldName", columnReader.fieldName))
|
||||
return nil, merr.WrapErrImportFailed(fmt.Sprintf("unsupported data type: %s of field: %s", columnReader.elementType.String(), columnReader.fieldName))
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -65,18 +65,10 @@ func (m *GrpcDataNodeClient) GetCompactionState(ctx context.Context, in *datapb.
|
|||
return &datapb.CompactionStateResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcDataNodeClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcDataNodeClient) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest, opts ...grpc.CallOption) (*datapb.ResendSegmentStatsResponse, error) {
|
||||
return &datapb.ResendSegmentStatsResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcDataNodeClient) AddImportSegment(ctx context.Context, in *datapb.AddImportSegmentRequest, opts ...grpc.CallOption) (*datapb.AddImportSegmentResponse, error) {
|
||||
return &datapb.AddImportSegmentResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcDataNodeClient) SyncSegments(ctx context.Context, in *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
|
|
@ -230,22 +230,6 @@ func (m *GrpcRootCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMe
|
|||
return &milvuspb.GetMetricsResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcRootCoordClient) Import(ctx context.Context, req *milvuspb.ImportRequest, opts ...grpc.CallOption) (*milvuspb.ImportResponse, error) {
|
||||
return &milvuspb.ImportResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcRootCoordClient) GetImportState(ctx context.Context, req *milvuspb.GetImportStateRequest, opts ...grpc.CallOption) (*milvuspb.GetImportStateResponse, error) {
|
||||
return &milvuspb.GetImportStateResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcRootCoordClient) ListImportTasks(ctx context.Context, req *milvuspb.ListImportTasksRequest, opts ...grpc.CallOption) (*milvuspb.ListImportTasksResponse, error) {
|
||||
return &milvuspb.ListImportTasksResponse{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcRootCoordClient) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
||||
func (m *GrpcRootCoordClient) CreateCredential(ctx context.Context, in *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{}, m.Err
|
||||
}
|
||||
|
|
|
@ -230,8 +230,6 @@ type commonConfig struct {
|
|||
|
||||
JSONMaxLength ParamItem `refreshable:"false"`
|
||||
|
||||
ImportMaxFileSize ParamItem `refreshable:"true"`
|
||||
|
||||
MetricsPort ParamItem `refreshable:"false"`
|
||||
|
||||
// lock related params
|
||||
|
@ -633,13 +631,6 @@ like the old password verification when updating the credential`,
|
|||
}
|
||||
p.JSONMaxLength.Init(base.mgr)
|
||||
|
||||
p.ImportMaxFileSize = ParamItem{
|
||||
Key: "common.ImportMaxFileSize",
|
||||
Version: "2.2.9",
|
||||
DefaultValue: fmt.Sprint(16 << 30),
|
||||
}
|
||||
p.ImportMaxFileSize.Init(base.mgr)
|
||||
|
||||
p.MetricsPort = ParamItem{
|
||||
Key: "common.MetricsPort",
|
||||
Version: "2.3.0",
|
||||
|
@ -891,10 +882,6 @@ type rootCoordConfig struct {
|
|||
DmlChannelNum ParamItem `refreshable:"false"`
|
||||
MaxPartitionNum ParamItem `refreshable:"true"`
|
||||
MinSegmentSizeToEnableIndex ParamItem `refreshable:"true"`
|
||||
ImportTaskExpiration ParamItem `refreshable:"true"`
|
||||
ImportTaskRetention ParamItem `refreshable:"true"`
|
||||
ImportMaxPendingTaskCount ParamItem `refreshable:"true"`
|
||||
ImportTaskSubPath ParamItem `refreshable:"true"`
|
||||
EnableActiveStandby ParamItem `refreshable:"false"`
|
||||
MaxDatabaseNum ParamItem `refreshable:"false"`
|
||||
MaxGeneralCapacity ParamItem `refreshable:"true"`
|
||||
|
@ -930,38 +917,6 @@ func (p *rootCoordConfig) init(base *BaseTable) {
|
|||
}
|
||||
p.MinSegmentSizeToEnableIndex.Init(base.mgr)
|
||||
|
||||
p.ImportTaskExpiration = ParamItem{
|
||||
Key: "rootCoord.importTaskExpiration",
|
||||
Version: "2.2.0",
|
||||
DefaultValue: "900", // 15 * 60 seconds
|
||||
Doc: "(in seconds) Duration after which an import task will expire (be killed). Default 900 seconds (15 minutes).",
|
||||
Export: true,
|
||||
}
|
||||
p.ImportTaskExpiration.Init(base.mgr)
|
||||
|
||||
p.ImportTaskRetention = ParamItem{
|
||||
Key: "rootCoord.importTaskRetention",
|
||||
Version: "2.2.0",
|
||||
DefaultValue: strconv.Itoa(24 * 60 * 60),
|
||||
Doc: "(in seconds) Milvus will keep the record of import tasks for at least `importTaskRetention` seconds. Default 86400, seconds (24 hours).",
|
||||
Export: true,
|
||||
}
|
||||
p.ImportTaskRetention.Init(base.mgr)
|
||||
|
||||
p.ImportTaskSubPath = ParamItem{
|
||||
Key: "rootCoord.ImportTaskSubPath",
|
||||
Version: "2.2.0",
|
||||
DefaultValue: "importtask",
|
||||
}
|
||||
p.ImportTaskSubPath.Init(base.mgr)
|
||||
|
||||
p.ImportMaxPendingTaskCount = ParamItem{
|
||||
Key: "rootCoord.importMaxPendingTaskCount",
|
||||
Version: "2.2.2",
|
||||
DefaultValue: strconv.Itoa(65535),
|
||||
}
|
||||
p.ImportMaxPendingTaskCount.Init(base.mgr)
|
||||
|
||||
p.EnableActiveStandby = ParamItem{
|
||||
Key: "rootCoord.enableActiveStandby",
|
||||
Version: "2.2.0",
|
||||
|
@ -3164,11 +3119,6 @@ type dataNodeConfig struct {
|
|||
// DataNode send timetick interval per collection
|
||||
DataNodeTimeTickInterval ParamItem `refreshable:"false"`
|
||||
|
||||
// timeout for bulkinsert
|
||||
BulkInsertTimeoutSeconds ParamItem `refreshable:"true"`
|
||||
BulkInsertReadBufferSize ParamItem `refreshable:"true"`
|
||||
BulkInsertMaxMemorySize ParamItem `refreshable:"true"`
|
||||
|
||||
// Skip BF
|
||||
SkipBFStatsLoad ParamItem `refreshable:"true"`
|
||||
|
||||
|
@ -3382,30 +3332,6 @@ func (p *dataNodeConfig) init(base *BaseTable) {
|
|||
}
|
||||
p.SkipBFStatsLoad.Init(base.mgr)
|
||||
|
||||
p.BulkInsertTimeoutSeconds = ParamItem{
|
||||
Key: "datanode.bulkinsert.timeout.seconds",
|
||||
Version: "2.3.0",
|
||||
PanicIfEmpty: false,
|
||||
DefaultValue: "18000",
|
||||
}
|
||||
p.BulkInsertTimeoutSeconds.Init(base.mgr)
|
||||
|
||||
p.BulkInsertReadBufferSize = ParamItem{
|
||||
Key: "datanode.bulkinsert.readBufferSize",
|
||||
Version: "2.3.4",
|
||||
PanicIfEmpty: false,
|
||||
DefaultValue: "16777216",
|
||||
}
|
||||
p.BulkInsertReadBufferSize.Init(base.mgr)
|
||||
|
||||
p.BulkInsertMaxMemorySize = ParamItem{
|
||||
Key: "datanode.bulkinsert.maxMemorySize",
|
||||
Version: "2.3.4",
|
||||
PanicIfEmpty: false,
|
||||
DefaultValue: "6442450944",
|
||||
}
|
||||
p.BulkInsertMaxMemorySize.Init(base.mgr)
|
||||
|
||||
p.ChannelWorkPoolSize = ParamItem{
|
||||
Key: "datanode.channel.workPoolSize",
|
||||
Version: "2.3.2",
|
||||
|
|
|
@ -117,8 +117,6 @@ func TestComponentParam(t *testing.T) {
|
|||
t.Logf("master MaxPartitionNum = %d", Params.MaxPartitionNum.GetAsInt64())
|
||||
assert.NotEqual(t, Params.MinSegmentSizeToEnableIndex.GetAsInt64(), 0)
|
||||
t.Logf("master MinSegmentSizeToEnableIndex = %d", Params.MinSegmentSizeToEnableIndex.GetAsInt64())
|
||||
assert.NotEqual(t, Params.ImportTaskExpiration.GetAsFloat(), 0)
|
||||
t.Logf("master ImportTaskRetention = %f", Params.ImportTaskRetention.GetAsFloat())
|
||||
assert.Equal(t, Params.EnableActiveStandby.GetAsBool(), false)
|
||||
t.Logf("rootCoord EnableActiveStandby = %t", Params.EnableActiveStandby.GetAsBool())
|
||||
|
||||
|
@ -423,10 +421,6 @@ func TestComponentParam(t *testing.T) {
|
|||
t.Logf("SyncPeriod: %v", period)
|
||||
assert.Equal(t, 10*time.Minute, Params.SyncPeriod.GetAsDuration(time.Second))
|
||||
|
||||
bulkinsertTimeout := &Params.BulkInsertTimeoutSeconds
|
||||
t.Logf("BulkInsertTimeoutSeconds: %v", bulkinsertTimeout)
|
||||
assert.Equal(t, "18000", Params.BulkInsertTimeoutSeconds.GetValue())
|
||||
|
||||
channelWorkPoolSize := Params.ChannelWorkPoolSize.GetAsInt()
|
||||
t.Logf("channelWorkPoolSize: %d", channelWorkPoolSize)
|
||||
assert.Equal(t, -1, Params.ChannelWorkPoolSize.GetAsInt())
|
||||
|
|
|
@ -1,288 +0,0 @@
|
|||
// 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 bulkinsert
|
||||
|
||||
import (
|
||||
"context"
|
||||
"os"
|
||||
"strconv"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/util/importutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/metric"
|
||||
"github.com/milvus-io/milvus/tests/integration"
|
||||
)
|
||||
|
||||
const (
|
||||
TempFilesPath = "/tmp/integration_test/import/"
|
||||
Dim = 128
|
||||
)
|
||||
|
||||
type BulkInsertSuite struct {
|
||||
integration.MiniClusterSuite
|
||||
}
|
||||
|
||||
// test bulk insert E2E
|
||||
// 1, create collection with a vector column and a varchar column
|
||||
// 2, generate numpy files
|
||||
// 3, import
|
||||
// 4, create index
|
||||
// 5, load
|
||||
// 6, search
|
||||
func (s *BulkInsertSuite) TestBulkInsert() {
|
||||
c := s.Cluster
|
||||
ctx, cancel := context.WithCancel(c.GetContext())
|
||||
defer cancel()
|
||||
|
||||
prefix := "TestBulkInsert"
|
||||
dbName := ""
|
||||
collectionName := prefix + funcutil.GenRandomStr()
|
||||
// floatVecField := floatVecField
|
||||
dim := 128
|
||||
|
||||
schema := integration.ConstructSchema(collectionName, dim, true,
|
||||
&schemapb.FieldSchema{Name: "id", DataType: schemapb.DataType_Int64, IsPrimaryKey: true, AutoID: true},
|
||||
&schemapb.FieldSchema{Name: "image_path", DataType: schemapb.DataType_VarChar, TypeParams: []*commonpb.KeyValuePair{{Key: common.MaxLengthKey, Value: "65535"}}},
|
||||
&schemapb.FieldSchema{Name: "embeddings", DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{{Key: common.DimKey, Value: "128"}}},
|
||||
)
|
||||
marshaledSchema, err := proto.Marshal(schema)
|
||||
s.NoError(err)
|
||||
|
||||
createCollectionStatus, err := c.Proxy.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
Schema: marshaledSchema,
|
||||
ShardsNum: common.DefaultShardsNum,
|
||||
})
|
||||
s.NoError(err)
|
||||
if createCollectionStatus.GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Warn("createCollectionStatus fail reason", zap.String("reason", createCollectionStatus.GetReason()))
|
||||
s.FailNow("failed to create collection")
|
||||
}
|
||||
s.Equal(createCollectionStatus.GetErrorCode(), commonpb.ErrorCode_Success)
|
||||
|
||||
log.Info("CreateCollection result", zap.Any("createCollectionStatus", createCollectionStatus))
|
||||
showCollectionsResp, err := c.Proxy.ShowCollections(ctx, &milvuspb.ShowCollectionsRequest{})
|
||||
s.NoError(err)
|
||||
s.Equal(showCollectionsResp.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
|
||||
log.Info("ShowCollections result", zap.Any("showCollectionsResp", showCollectionsResp))
|
||||
|
||||
err = os.MkdirAll(c.ChunkManager.RootPath(), os.ModePerm)
|
||||
s.NoError(err)
|
||||
|
||||
err = GenerateNumpyFile(c.ChunkManager.RootPath()+"/"+"embeddings.npy", 100, schemapb.DataType_FloatVector, []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DimKey,
|
||||
Value: strconv.Itoa(Dim),
|
||||
},
|
||||
})
|
||||
s.NoError(err)
|
||||
err = GenerateNumpyFile(c.ChunkManager.RootPath()+"/"+"image_path.npy", 100, schemapb.DataType_VarChar, []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.MaxLengthKey,
|
||||
Value: strconv.Itoa(65535),
|
||||
},
|
||||
})
|
||||
s.NoError(err)
|
||||
|
||||
bulkInsertFiles := []string{
|
||||
c.ChunkManager.RootPath() + "/" + "embeddings.npy",
|
||||
c.ChunkManager.RootPath() + "/" + "image_path.npy",
|
||||
}
|
||||
|
||||
health1, err := c.DataCoord.CheckHealth(ctx, &milvuspb.CheckHealthRequest{})
|
||||
s.NoError(err)
|
||||
log.Info("dataCoord health", zap.Any("health1", health1))
|
||||
importResp, err := c.Proxy.Import(ctx, &milvuspb.ImportRequest{
|
||||
CollectionName: collectionName,
|
||||
Files: bulkInsertFiles,
|
||||
})
|
||||
s.NoError(err)
|
||||
log.Info("Import result", zap.Any("importResp", importResp), zap.Int64s("tasks", importResp.GetTasks()))
|
||||
|
||||
tasks := importResp.GetTasks()
|
||||
for _, task := range tasks {
|
||||
loop:
|
||||
for {
|
||||
importTaskState, err := c.Proxy.GetImportState(ctx, &milvuspb.GetImportStateRequest{
|
||||
Task: task,
|
||||
})
|
||||
s.NoError(err)
|
||||
switch importTaskState.GetState() {
|
||||
case commonpb.ImportState_ImportCompleted:
|
||||
break loop
|
||||
case commonpb.ImportState_ImportFailed:
|
||||
break loop
|
||||
case commonpb.ImportState_ImportFailedAndCleaned:
|
||||
break loop
|
||||
default:
|
||||
log.Info("import task state", zap.Int64("id", task), zap.String("state", importTaskState.GetState().String()))
|
||||
time.Sleep(time.Second * time.Duration(3))
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// flush
|
||||
flushResp, err := c.Proxy.Flush(ctx, &milvuspb.FlushRequest{
|
||||
DbName: dbName,
|
||||
CollectionNames: []string{collectionName},
|
||||
})
|
||||
s.NoError(err)
|
||||
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
|
||||
ids := segmentIDs.GetData()
|
||||
s.Require().Empty(segmentIDs)
|
||||
s.Require().True(has)
|
||||
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
|
||||
s.True(has)
|
||||
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
|
||||
|
||||
health2, err := c.DataCoord.CheckHealth(ctx, &milvuspb.CheckHealthRequest{})
|
||||
s.NoError(err)
|
||||
log.Info("dataCoord health", zap.Any("health2", health2))
|
||||
|
||||
segments, err := c.MetaWatcher.ShowSegments()
|
||||
s.NoError(err)
|
||||
s.NotEmpty(segments)
|
||||
for _, segment := range segments {
|
||||
log.Info("ShowSegments result", zap.String("segment", segment.String()))
|
||||
}
|
||||
|
||||
// create index
|
||||
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
|
||||
CollectionName: collectionName,
|
||||
FieldName: "embeddings",
|
||||
IndexName: "_default",
|
||||
ExtraParams: integration.ConstructIndexParam(dim, integration.IndexHNSW, metric.L2),
|
||||
})
|
||||
if createIndexStatus.GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Warn("createIndexStatus fail reason", zap.String("reason", createIndexStatus.GetReason()))
|
||||
}
|
||||
s.NoError(err)
|
||||
s.Equal(commonpb.ErrorCode_Success, createIndexStatus.GetErrorCode())
|
||||
|
||||
s.WaitForIndexBuilt(ctx, collectionName, "embeddings")
|
||||
|
||||
// load
|
||||
loadStatus, err := c.Proxy.LoadCollection(ctx, &milvuspb.LoadCollectionRequest{
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
})
|
||||
s.NoError(err)
|
||||
if loadStatus.GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Warn("loadStatus fail reason", zap.String("reason", loadStatus.GetReason()))
|
||||
}
|
||||
s.Equal(commonpb.ErrorCode_Success, loadStatus.GetErrorCode())
|
||||
s.WaitForLoad(ctx, collectionName)
|
||||
|
||||
// search
|
||||
expr := "" // fmt.Sprintf("%s > 0", int64Field)
|
||||
nq := 10
|
||||
topk := 10
|
||||
roundDecimal := -1
|
||||
|
||||
params := integration.GetSearchParams(integration.IndexHNSW, metric.L2)
|
||||
searchReq := integration.ConstructSearchRequest("", collectionName, expr,
|
||||
"embeddings", schemapb.DataType_FloatVector, nil, metric.L2, params, nq, dim, topk, roundDecimal)
|
||||
|
||||
searchResult, err := c.Proxy.Search(ctx, searchReq)
|
||||
|
||||
if searchResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Warn("searchResult fail reason", zap.String("reason", searchResult.GetStatus().GetReason()))
|
||||
}
|
||||
s.NoError(err)
|
||||
s.Equal(commonpb.ErrorCode_Success, searchResult.GetStatus().GetErrorCode())
|
||||
|
||||
log.Info("======================")
|
||||
log.Info("======================")
|
||||
log.Info("TestBulkInsert succeed")
|
||||
log.Info("======================")
|
||||
log.Info("======================")
|
||||
}
|
||||
|
||||
func TestBulkInsert(t *testing.T) {
|
||||
suite.Run(t, new(BulkInsertSuite))
|
||||
}
|
||||
|
||||
func GenerateNumpyFile(filePath string, rowCount int, dType schemapb.DataType, typeParams []*commonpb.KeyValuePair) error {
|
||||
if dType == schemapb.DataType_VarChar {
|
||||
var data []string
|
||||
for i := 0; i < rowCount; i++ {
|
||||
data = append(data, "str")
|
||||
}
|
||||
err := importutil.CreateNumpyFile(filePath, data)
|
||||
if err != nil {
|
||||
log.Warn("failed to create numpy file", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
if dType == schemapb.DataType_FloatVector {
|
||||
dimStr, ok := funcutil.KeyValuePair2Map(typeParams)[common.DimKey]
|
||||
if !ok {
|
||||
return errors.New("FloatVector field needs dim parameter")
|
||||
}
|
||||
dim, err := strconv.Atoi(dimStr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// data := make([][]float32, rowCount)
|
||||
var data [][Dim]float32
|
||||
for i := 0; i < rowCount; i++ {
|
||||
vec := [Dim]float32{}
|
||||
for j := 0; j < dim; j++ {
|
||||
vec[j] = 1.1
|
||||
}
|
||||
// v := reflect.Indirect(reflect.ValueOf(vec))
|
||||
// log.Info("type", zap.Any("type", v.Kind()))
|
||||
data = append(data, vec)
|
||||
// v2 := reflect.Indirect(reflect.ValueOf(data))
|
||||
// log.Info("type", zap.Any("type", v2.Kind()))
|
||||
}
|
||||
err = importutil.CreateNumpyFile(filePath, data)
|
||||
if err != nil {
|
||||
log.Warn("failed to create numpy file", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestGenerateNumpyFile(t *testing.T) {
|
||||
err := os.MkdirAll(TempFilesPath, os.ModePerm)
|
||||
require.NoError(t, err)
|
||||
err = GenerateNumpyFile(TempFilesPath+"embeddings.npy", 100, schemapb.DataType_FloatVector, []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.DimKey,
|
||||
Value: strconv.Itoa(Dim),
|
||||
},
|
||||
})
|
||||
assert.NoError(t, err)
|
||||
}
|
Loading…
Reference in New Issue