Decoupling client and server API in types interface (#27186)

Co-authored-by:: aoiasd <zhicheng.yue@zilliz.com>

Signed-off-by: jaime <yun.zhang@zilliz.com>
This commit is contained in:
jaime 2023-09-26 09:57:25 +08:00 committed by GitHub
parent 24354b166c
commit 7f7c71ea7d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
170 changed files with 17880 additions and 7274 deletions

View File

@ -374,6 +374,15 @@ generate-mockery-types: getdeps
# IndexNode
$(INSTALL_PATH)/mockery --name=IndexNodeComponent --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_indexnode.go --with-expecter --structname=MockIndexNode
# Clients
$(INSTALL_PATH)/mockery --name=RootCoordClient --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_rootcoord_client.go --with-expecter --structname=MockRootCoordClient
$(INSTALL_PATH)/mockery --name=QueryCoordClient --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_querycoord_client.go --with-expecter --structname=MockQueryCoordClient
$(INSTALL_PATH)/mockery --name=DataCoordClient --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_datacoord_client.go --with-expecter --structname=MockDataCoordClient
$(INSTALL_PATH)/mockery --name=QueryNodeClient --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_querynode_client.go --with-expecter --structname=MockQueryNodeClient
$(INSTALL_PATH)/mockery --name=DataNodeClient --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_datanode_client.go --with-expecter --structname=MockDataNodeClient
$(INSTALL_PATH)/mockery --name=IndexNodeClient --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_indexnode_client.go --with-expecter --structname=MockIndexNodeClient
$(INSTALL_PATH)/mockery --name=ProxyClient --dir=$(PWD)/internal/types --output=$(PWD)/internal/mocks --filename=mock_proxy_client.go --with-expecter --structname=MockProxyClient
generate-mockery-rootcoord: getdeps
$(INSTALL_PATH)/mockery --name=IMetaTable --dir=$(PWD)/internal/rootcoord --output=$(PWD)/internal/rootcoord/mocks --filename=meta_table.go --with-expecter --outpkg=mockrootcoord
$(INSTALL_PATH)/mockery --name=GarbageCollector --dir=$(PWD)/internal/rootcoord --output=$(PWD)/internal/rootcoord/mocks --filename=garbage_collector.go --with-expecter --outpkg=mockrootcoord

View File

@ -22,6 +22,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/pkg/util/merr"
@ -29,7 +30,7 @@ import (
type mockIDAllocator struct{}
func (tso *mockIDAllocator) AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
func (tso *mockIDAllocator) AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocIDResponse, error) {
return &rootcoordpb.AllocIDResponse{
Status: merr.Status(nil),
ID: int64(1),

View File

@ -19,9 +19,11 @@ package allocator
import (
"context"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
)
type remoteInterface interface {
AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error)
AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocIDResponse, error)
}

View File

@ -37,13 +37,13 @@ var _ allocator = (*rootCoordAllocator)(nil)
// rootCoordAllocator use RootCoord as allocator
type rootCoordAllocator struct {
types.RootCoord
types.RootCoordClient
}
// newRootCoordAllocator gets an allocator from RootCoord
func newRootCoordAllocator(rootCoordClient types.RootCoord) allocator {
func newRootCoordAllocator(rootCoordClient types.RootCoordClient) allocator {
return &rootCoordAllocator{
RootCoord: rootCoordClient,
RootCoordClient: rootCoordClient,
}
}

View File

@ -27,7 +27,7 @@ import (
func TestAllocator_Basic(t *testing.T) {
paramtable.Init()
ms := newMockRootCoordService()
ms := newMockRootCoordClient()
allocator := newRootCoordAllocator(ms)
ctx := context.Background()
@ -42,7 +42,7 @@ func TestAllocator_Basic(t *testing.T) {
})
t.Run("Test Unhealthy Root", func(t *testing.T) {
ms := newMockRootCoordService()
ms := newMockRootCoordClient()
allocator := newRootCoordAllocator(ms)
err := ms.Stop()
assert.NoError(t, err)

View File

@ -367,7 +367,7 @@ func (suite *ClusterSuite) TestUnregister() {
ctx, cancel := context.WithCancel(context.TODO())
defer cancel()
mockSessionCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
mockSessionCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(1, nil)
}
sessionManager := NewSessionManager(withSessionCreator(mockSessionCreator))
@ -414,7 +414,7 @@ func TestWatchIfNeeded(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO())
defer cancel()
mockSessionCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
mockSessionCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(1, nil)
}
sessionManager := NewSessionManager(withSessionCreator(mockSessionCreator))
@ -629,7 +629,7 @@ func TestCluster_ReCollectSegmentStats(t *testing.T) {
t.Run("recollect succeed", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO())
defer cancel()
mockSessionCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
mockSessionCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(1, nil)
}
sessionManager := NewSessionManager(withSessionCreator(mockSessionCreator))

View File

@ -27,6 +27,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
"go.uber.org/zap"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
mockkv "github.com/milvus-io/milvus/internal/kv/mocks"
@ -196,7 +197,7 @@ func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
}
func Test_compactionPlanHandler_execWithParallels(t *testing.T) {
mockDataNode := &mocks.MockDataNode{}
mockDataNode := &mocks.MockDataNodeClient{}
paramtable.Get().Save(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key, "1")
defer paramtable.Get().Reset(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key)
c := &compactionPlanHandler{
@ -232,11 +233,12 @@ func Test_compactionPlanHandler_execWithParallels(t *testing.T) {
var mut sync.RWMutex
called := 0
mockDataNode.EXPECT().Compaction(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.CompactionPlan) {
mut.Lock()
defer mut.Unlock()
called++
}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil).Times(3)
mockDataNode.EXPECT().Compaction(mock.Anything, mock.Anything, mock.Anything).
Run(func(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) {
mut.Lock()
defer mut.Unlock()
called++
}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil).Times(3)
go func() {
c.execCompactionPlan(signal, plan1)
c.execCompactionPlan(signal, plan2)
@ -283,8 +285,10 @@ func getDeltaLogPath(rootPath string, segmentID typeutil.UniqueID) string {
}
func TestCompactionPlanHandler_handleMergeCompactionResult(t *testing.T) {
mockDataNode := &mocks.MockDataNode{}
call := mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
mockDataNode := &mocks.MockDataNodeClient{}
call := mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything, mock.Anything).
Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) {}).
Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
dataNodeID := UniqueID(111)
@ -417,7 +421,8 @@ func TestCompactionPlanHandler_handleMergeCompactionResult(t *testing.T) {
require.True(t, has)
call.Unset()
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}, nil)
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything, mock.Anything).
Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}, nil)
err = c.handleMergeCompactionResult(plan, compactionResult2)
assert.Error(t, err)
}
@ -439,8 +444,10 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) {
})
t.Run("test complete merge compaction task", func(t *testing.T) {
mockDataNode := &mocks.MockDataNode{}
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
mockDataNode := &mocks.MockDataNodeClient{}
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything, mock.Anything).
Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) {}).
Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
dataNodeID := UniqueID(111)
@ -532,8 +539,10 @@ func TestCompactionPlanHandler_completeCompaction(t *testing.T) {
})
t.Run("test empty result merge compaction task", func(t *testing.T) {
mockDataNode := &mocks.MockDataNode{}
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
mockDataNode := &mocks.MockDataNodeClient{}
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything, mock.Anything).
Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) {}).
Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
dataNodeID := UniqueID(111)

View File

@ -44,10 +44,10 @@ type Broker interface {
}
type CoordinatorBroker struct {
rootCoord types.RootCoord
rootCoord types.RootCoordClient
}
func NewCoordinatorBroker(rootCoord types.RootCoord) *CoordinatorBroker {
func NewCoordinatorBroker(rootCoord types.RootCoordClient) *CoordinatorBroker {
return &CoordinatorBroker{
rootCoord: rootCoord,
}

View File

@ -420,7 +420,7 @@ func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
// assignTask sends the index task to the IndexNode, it has a timeout interval, if the IndexNode doesn't respond within
// the interval, it is considered that the task sending failed.
func (ib *indexBuilder) assignTask(builderClient types.IndexNode, req *indexpb.CreateJobRequest) error {
func (ib *indexBuilder) assignTask(builderClient types.IndexNodeClient, req *indexpb.CreateJobRequest) error {
ctx, cancel := context.WithTimeout(context.Background(), reqTimeoutInterval)
defer cancel()
resp, err := builderClient.CreateJob(ctx, req)

View File

@ -24,9 +24,9 @@ import (
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/metastore"
catalogmocks "github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/metastore/model"
@ -34,6 +34,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
mclient "github.com/milvus-io/milvus/internal/util/mock"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
@ -615,11 +616,51 @@ func TestIndexBuilder(t *testing.T) {
mock.Anything,
).Return(nil)
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().GetJobStats(mock.Anything, mock.Anything, mock.Anything).
Return(&indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TotalJobNum: 1,
EnqueueJobNum: 0,
InProgressJobNum: 1,
TaskSlots: 1,
JobInfos: []*indexpb.JobInfo{
{
NumRows: 1024,
Dim: 128,
StartTime: 1,
EndTime: 10,
PodID: 1,
},
},
}, nil)
ic.EXPECT().QueryJobs(mock.Anything, mock.Anything, mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, in *indexpb.QueryJobsRequest, option ...grpc.CallOption) (*indexpb.QueryJobsResponse, error) {
indexInfos := make([]*indexpb.IndexTaskInfo, 0)
for _, buildID := range in.BuildIDs {
indexInfos = append(indexInfos, &indexpb.IndexTaskInfo{
BuildID: buildID,
State: commonpb.IndexState_Finished,
IndexFileKeys: []string{"file1", "file2"},
})
}
return &indexpb.QueryJobsResponse{
Status: merr.Status(nil),
ClusterID: in.ClusterID,
IndexInfos: indexInfos,
}, nil
})
ic.EXPECT().CreateJob(mock.Anything, mock.Anything, mock.Anything, mock.Anything).
Return(merr.Status(nil), nil)
ic.EXPECT().DropJobs(mock.Anything, mock.Anything, mock.Anything).
Return(merr.Status(nil), nil)
mt := createMetaTable(catalog)
nodeManager := &IndexNodeManager{
ctx: ctx,
nodeClients: map[UniqueID]types.IndexNode{
4: indexnode.NewIndexNodeMock(),
nodeClients: map[UniqueID]types.IndexNodeClient{
4: ic,
},
}
chunkManager := &mocks.ChunkManager{}
@ -719,7 +760,7 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("peek client fail", func(t *testing.T) {
ib.tasks[buildID] = indexTaskInit
ib.nodeManager = &IndexNodeManager{nodeClients: map[UniqueID]types.IndexNode{}}
ib.nodeManager = &IndexNodeManager{nodeClients: map[UniqueID]types.IndexNodeClient{}}
ib.process(buildID)
state, ok := ib.tasks[buildID]
@ -730,7 +771,7 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("update version fail", func(t *testing.T) {
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{1: indexnode.NewIndexNodeMock()},
nodeClients: map[UniqueID]types.IndexNodeClient{1: &mclient.GrpcIndexNodeClient{Err: nil}},
}
ib.process(buildID)
@ -765,20 +806,18 @@ func TestIndexBuilder_Error(t *testing.T) {
paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "local")
ib.tasks[buildID] = indexTaskInit
ib.meta.catalog = sc
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().CreateJob(mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("error"))
ic.EXPECT().GetJobStats(mock.Anything, mock.Anything, mock.Anything).Return(&indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
}, nil)
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallCreateJob: func(ctx context.Context, req *indexpb.CreateJobRequest) (*commonpb.Status, error) {
return nil, errors.New("error")
},
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
1: ic,
},
}
ib.process(buildID)
@ -790,23 +829,20 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("assign task fail", func(t *testing.T) {
paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "local")
ib.meta.catalog = sc
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().CreateJob(mock.Anything, mock.Anything, mock.Anything).Return(&commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "mock fail",
}, nil)
ic.EXPECT().GetJobStats(mock.Anything, mock.Anything, mock.Anything).Return(&indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
}, nil)
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallCreateJob: func(ctx context.Context, req *indexpb.CreateJobRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "mock fail",
}, nil
},
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
1: ic,
},
}
ib.tasks[buildID] = indexTaskInit
@ -820,16 +856,15 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("drop job error", func(t *testing.T) {
ib.meta.buildID2SegmentIndex[buildID].NodeID = nodeID
ib.meta.catalog = sc
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().DropJobs(mock.Anything, mock.Anything, mock.Anything).Return(&commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
}, errors.New("error"))
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
nodeID: &indexnode.Mock{
CallDropJobs: func(ctx context.Context, in *indexpb.DropJobsRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
}, errors.New("error")
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
nodeID: ic,
},
}
ib.tasks[buildID] = indexTaskDone
@ -850,17 +885,16 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("drop job fail", func(t *testing.T) {
ib.meta.buildID2SegmentIndex[buildID].NodeID = nodeID
ib.meta.catalog = sc
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().DropJobs(mock.Anything, mock.Anything, mock.Anything).Return(&commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "mock fail",
}, nil)
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
nodeID: &indexnode.Mock{
CallDropJobs: func(ctx context.Context, in *indexpb.DropJobsRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "mock fail",
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
nodeID: ic,
},
}
ib.tasks[buildID] = indexTaskDone
@ -881,14 +915,12 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("get state error", func(t *testing.T) {
ib.meta.buildID2SegmentIndex[buildID].NodeID = nodeID
ib.meta.catalog = sc
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().QueryJobs(mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("error"))
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
nodeID: &indexnode.Mock{
CallQueryJobs: func(ctx context.Context, in *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
return nil, errors.New("error")
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
nodeID: ic,
},
}
@ -903,19 +935,17 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("get state fail", func(t *testing.T) {
ib.meta.buildID2SegmentIndex[buildID].NodeID = nodeID
ib.meta.catalog = sc
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().QueryJobs(mock.Anything, mock.Anything, mock.Anything).Return(&indexpb.QueryJobsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_BuildIndexError,
Reason: "mock fail",
},
}, nil)
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
nodeID: &indexnode.Mock{
CallQueryJobs: func(ctx context.Context, in *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
return &indexpb.QueryJobsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_BuildIndexError,
Reason: "mock fail",
},
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
nodeID: ic,
},
}
@ -930,25 +960,24 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("finish task fail", func(t *testing.T) {
ib.meta.buildID2SegmentIndex[buildID].NodeID = nodeID
ib.meta.catalog = ec
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().QueryJobs(mock.Anything, mock.Anything, mock.Anything).Return(&indexpb.QueryJobsResponse{
Status: merr.Status(nil),
IndexInfos: []*indexpb.IndexTaskInfo{
{
BuildID: buildID,
State: commonpb.IndexState_Finished,
IndexFileKeys: []string{"file1", "file2"},
SerializedSize: 1024,
FailReason: "",
},
},
}, nil)
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
nodeID: &indexnode.Mock{
CallQueryJobs: func(ctx context.Context, in *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
return &indexpb.QueryJobsResponse{
Status: merr.Status(nil),
IndexInfos: []*indexpb.IndexTaskInfo{
{
BuildID: buildID,
State: commonpb.IndexState_Finished,
IndexFileKeys: []string{"file1", "file2"},
SerializedSize: 1024,
FailReason: "",
},
},
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
nodeID: ic,
},
}
@ -963,25 +992,24 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("task still in progress", func(t *testing.T) {
ib.meta.buildID2SegmentIndex[buildID].NodeID = nodeID
ib.meta.catalog = ec
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().QueryJobs(mock.Anything, mock.Anything, mock.Anything).Return(&indexpb.QueryJobsResponse{
Status: merr.Status(nil),
IndexInfos: []*indexpb.IndexTaskInfo{
{
BuildID: buildID,
State: commonpb.IndexState_InProgress,
IndexFileKeys: nil,
SerializedSize: 0,
FailReason: "",
},
},
}, nil)
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
nodeID: &indexnode.Mock{
CallQueryJobs: func(ctx context.Context, in *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
return &indexpb.QueryJobsResponse{
Status: merr.Status(nil),
IndexInfos: []*indexpb.IndexTaskInfo{
{
BuildID: buildID,
State: commonpb.IndexState_InProgress,
IndexFileKeys: nil,
SerializedSize: 0,
FailReason: "",
},
},
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
nodeID: ic,
},
}
@ -996,17 +1024,15 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("indexNode has no task", func(t *testing.T) {
ib.meta.buildID2SegmentIndex[buildID].NodeID = nodeID
ib.meta.catalog = sc
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().QueryJobs(mock.Anything, mock.Anything, mock.Anything).Return(&indexpb.QueryJobsResponse{
Status: merr.Status(nil),
IndexInfos: nil,
}, nil)
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{
nodeID: &indexnode.Mock{
CallQueryJobs: func(ctx context.Context, in *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
return &indexpb.QueryJobsResponse{
Status: merr.Status(nil),
IndexInfos: nil,
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
nodeID: ic,
},
}
@ -1023,7 +1049,7 @@ func TestIndexBuilder_Error(t *testing.T) {
ib.meta.catalog = sc
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNode{},
nodeClients: map[UniqueID]types.IndexNodeClient{},
}
ib.tasks[buildID] = indexTaskInProgress

View File

@ -33,7 +33,7 @@ import (
// IndexNodeManager is used to manage the client of IndexNode.
type IndexNodeManager struct {
nodeClients map[UniqueID]types.IndexNode
nodeClients map[UniqueID]types.IndexNodeClient
stoppingNodes map[UniqueID]struct{}
lock sync.RWMutex
ctx context.Context
@ -43,7 +43,7 @@ type IndexNodeManager struct {
// NewNodeManager is used to create a new IndexNodeManager.
func NewNodeManager(ctx context.Context, indexNodeCreator indexNodeCreatorFunc) *IndexNodeManager {
return &IndexNodeManager{
nodeClients: make(map[UniqueID]types.IndexNode),
nodeClients: make(map[UniqueID]types.IndexNodeClient),
stoppingNodes: make(map[UniqueID]struct{}),
lock: sync.RWMutex{},
ctx: ctx,
@ -52,7 +52,7 @@ func NewNodeManager(ctx context.Context, indexNodeCreator indexNodeCreatorFunc)
}
// setClient sets IndexNode client to node manager.
func (nm *IndexNodeManager) setClient(nodeID UniqueID, client types.IndexNode) {
func (nm *IndexNodeManager) setClient(nodeID UniqueID, client types.IndexNodeClient) {
log.Debug("set IndexNode client", zap.Int64("nodeID", nodeID))
nm.lock.Lock()
defer nm.lock.Unlock()
@ -82,7 +82,7 @@ func (nm *IndexNodeManager) StoppingNode(nodeID UniqueID) {
func (nm *IndexNodeManager) AddNode(nodeID UniqueID, address string) error {
log.Debug("add IndexNode", zap.Int64("nodeID", nodeID), zap.String("node address", address))
var (
nodeClient types.IndexNode
nodeClient types.IndexNodeClient
err error
)
@ -97,7 +97,7 @@ func (nm *IndexNodeManager) AddNode(nodeID UniqueID, address string) error {
}
// PeekClient peeks the client with the least load.
func (nm *IndexNodeManager) PeekClient(meta *model.SegmentIndex) (UniqueID, types.IndexNode) {
func (nm *IndexNodeManager) PeekClient(meta *model.SegmentIndex) (UniqueID, types.IndexNodeClient) {
allClients := nm.GetAllClients()
if len(allClients) == 0 {
log.Error("there is no IndexNode online")
@ -207,11 +207,11 @@ func (nm *IndexNodeManager) ClientSupportDisk() bool {
return false
}
func (nm *IndexNodeManager) GetAllClients() map[UniqueID]types.IndexNode {
func (nm *IndexNodeManager) GetAllClients() map[UniqueID]types.IndexNodeClient {
nm.lock.RLock()
defer nm.lock.RUnlock()
allClients := make(map[UniqueID]types.IndexNode, len(nm.nodeClients))
allClients := make(map[UniqueID]types.IndexNodeClient, len(nm.nodeClients))
for nodeID, client := range nm.nodeClients {
if _, ok := nm.stoppingNodes[nodeID]; !ok {
allClients[nodeID] = client
@ -221,7 +221,7 @@ func (nm *IndexNodeManager) GetAllClients() map[UniqueID]types.IndexNode {
return allClients
}
func (nm *IndexNodeManager) GetClientByID(nodeID UniqueID) (types.IndexNode, bool) {
func (nm *IndexNodeManager) GetClientByID(nodeID UniqueID) (types.IndexNodeClient, bool) {
nm.lock.RLock()
defer nm.lock.RUnlock()
@ -237,7 +237,7 @@ type indexNodeGetMetricsResponse struct {
// getMetrics get metrics information of all IndexNode.
func (nm *IndexNodeManager) getMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) []indexNodeGetMetricsResponse {
var clients []types.IndexNode
var clients []types.IndexNodeClient
nm.lock.RLock()
for _, node := range nm.nodeClients {
clients = append(clients, node)

View File

@ -23,10 +23,11 @@ import (
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/util/merr"
@ -50,92 +51,62 @@ func TestIndexNodeManager_AddNode(t *testing.T) {
}
func TestIndexNodeManager_PeekClient(t *testing.T) {
getMockedGetJobStatsClient := func(resp *indexpb.GetJobStatsResponse, err error) types.IndexNodeClient {
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().GetJobStats(mock.Anything, mock.Anything, mock.Anything).Return(resp, err)
return ic
}
t.Run("multiple unavailable IndexNode", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.TODO(),
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
nodeClients: map[UniqueID]types.IndexNodeClient{
1: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
},
2: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
}, errors.New("error")),
2: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
},
3: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
}, errors.New("error")),
3: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
},
4: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
}, errors.New("error")),
4: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
},
5: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
}, errors.New("error")),
5: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
},
6: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
}, nil),
6: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
},
7: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
}, nil),
7: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
},
8: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
TaskSlots: 1,
Status: merr.Status(nil),
}, nil
},
},
9: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
TaskSlots: 10,
Status: merr.Status(nil),
}, nil
},
},
}, nil),
8: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
TaskSlots: 1,
Status: merr.Status(nil),
}, nil),
9: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
TaskSlots: 10,
Status: merr.Status(nil),
}, nil),
},
}
@ -146,21 +117,23 @@ func TestIndexNodeManager_PeekClient(t *testing.T) {
}
func TestIndexNodeManager_ClientSupportDisk(t *testing.T) {
getMockedGetJobStatsClient := func(resp *indexpb.GetJobStatsResponse, err error) types.IndexNodeClient {
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().GetJobStats(mock.Anything, mock.Anything, mock.Anything).Return(resp, err)
return ic
}
t.Run("support", func(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
JobInfos: nil,
EnableDisk: true,
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
1: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
JobInfos: nil,
EnableDisk: true,
}, nil),
},
}
@ -172,17 +145,13 @@ func TestIndexNodeManager_ClientSupportDisk(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
JobInfos: nil,
EnableDisk: false,
}, nil
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
1: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: merr.Status(nil),
TaskSlots: 1,
JobInfos: nil,
EnableDisk: false,
}, nil),
},
}
@ -194,7 +163,7 @@ func TestIndexNodeManager_ClientSupportDisk(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{},
nodeClients: map[UniqueID]types.IndexNodeClient{},
}
support := nm.ClientSupportDisk()
@ -205,12 +174,8 @@ func TestIndexNodeManager_ClientSupportDisk(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return nil, errors.New("error")
},
},
nodeClients: map[UniqueID]types.IndexNodeClient{
1: getMockedGetJobStatsClient(nil, errors.New("error")),
},
}
@ -222,20 +187,16 @@ func TestIndexNodeManager_ClientSupportDisk(t *testing.T) {
nm := &IndexNodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
TaskSlots: 0,
JobInfos: nil,
EnableDisk: false,
}, nil
nodeClients: map[UniqueID]types.IndexNodeClient{
1: getMockedGetJobStatsClient(&indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
},
TaskSlots: 0,
JobInfos: nil,
EnableDisk: false,
}, nil),
},
}

View File

@ -183,7 +183,7 @@ func (s *Server) getDataNodeMetrics(ctx context.Context, req *milvuspb.GetMetric
return infos, nil
}
func (s *Server) getIndexNodeMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, node types.IndexNode) (metricsinfo.IndexNodeInfos, error) {
func (s *Server) getIndexNodeMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, node types.IndexNodeClient) (metricsinfo.IndexNodeInfos, error) {
infos := metricsinfo.IndexNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
HasError: true,

View File

@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -32,25 +33,25 @@ import (
)
type mockMetricDataNodeClient struct {
types.DataNode
types.DataNodeClient
mock func() (*milvuspb.GetMetricsResponse, error)
}
func (c *mockMetricDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *mockMetricDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
if c.mock == nil {
return c.DataNode.GetMetrics(ctx, req)
return c.DataNodeClient.GetMetrics(ctx, req)
}
return c.mock()
}
type mockMetricIndexNodeClient struct {
types.IndexNode
types.IndexNodeClient
mock func() (*milvuspb.GetMetricsResponse, error)
}
func (m *mockMetricIndexNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (m *mockMetricIndexNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
if m.mock == nil {
return m.IndexNode.GetMetrics(ctx, req)
return m.IndexNodeClient.GetMetrics(ctx, req)
}
return m.mock()
}
@ -69,7 +70,7 @@ func TestGetDataNodeMetrics(t *testing.T) {
_, err = svr.getDataNodeMetrics(ctx, req, NewSession(&NodeInfo{}, nil))
assert.Error(t, err)
creator := func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
creator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(100, nil)
}
@ -81,10 +82,10 @@ func TestGetDataNodeMetrics(t *testing.T) {
assert.Equal(t, metricsinfo.ConstructComponentName(typeutil.DataNodeRole, 100), info.BaseComponentInfos.Name)
getMockFailedClientCreator := func(mockFunc func() (*milvuspb.GetMetricsResponse, error)) dataNodeCreatorFunc {
return func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
return func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
cli, err := creator(ctx, addr, nodeID)
assert.NoError(t, err)
return &mockMetricDataNodeClient{DataNode: cli, mock: mockFunc}, nil
return &mockMetricDataNodeClient{DataNodeClient: cli, mock: mockFunc}, nil
}
}

View File

@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/errors"
clientv3 "go.etcd.io/etcd/client/v3"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -183,20 +184,11 @@ func newMockIndexNodeClient(id int64) (*mockIndexNodeClient, error) {
}, nil
}
func (c *mockDataNodeClient) Init() error {
func (c *mockDataNodeClient) Close() error {
return nil
}
func (c *mockDataNodeClient) Start() error {
c.state = commonpb.StateCode_Healthy
return nil
}
func (c *mockDataNodeClient) Register() error {
return nil
}
func (c *mockDataNodeClient) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *mockDataNodeClient) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: c.id,
@ -205,34 +197,34 @@ func (c *mockDataNodeClient) GetComponentStates(ctx context.Context) (*milvuspb.
}, nil
}
func (c *mockDataNodeClient) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *mockDataNodeClient) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return nil, nil
}
func (c *mockDataNodeClient) WatchDmChannels(ctx context.Context, in *datapb.WatchDmChannelsRequest) (*commonpb.Status, error) {
func (c *mockDataNodeClient) WatchDmChannels(ctx context.Context, in *datapb.WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) FlushSegments(ctx context.Context, in *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
func (c *mockDataNodeClient) FlushSegments(ctx context.Context, in *datapb.FlushSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
if c.ch != nil {
c.ch <- in
}
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
func (c *mockDataNodeClient) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest, opts ...grpc.CallOption) (*datapb.ResendSegmentStatsResponse, error) {
return &datapb.ResendSegmentStatsResponse{
Status: merr.Status(nil),
}, nil
}
func (c *mockDataNodeClient) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (c *mockDataNodeClient) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
return &internalpb.ShowConfigurationsResponse{
Status: merr.Status(nil),
}, nil
}
func (c *mockDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *mockDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
// TODO(dragondriver): change the id, though it's not important in ut
nodeID := c.id
@ -261,7 +253,7 @@ func (c *mockDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMe
}, nil
}
func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
if c.ch != nil {
c.ch <- struct{}{}
if c.compactionResp != nil {
@ -275,31 +267,31 @@ func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.Compact
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError, Reason: "not implemented"}, nil
}
func (c *mockDataNodeClient) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
func (c *mockDataNodeClient) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest, opts ...grpc.CallOption) (*datapb.CompactionStateResponse, error) {
return c.compactionStateResp, nil
}
func (c *mockDataNodeClient) Import(ctx context.Context, in *datapb.ImportTaskRequest) (*commonpb.Status, error) {
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) (*datapb.AddImportSegmentResponse, error) {
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) (*commonpb.Status, error) {
func (c *mockDataNodeClient) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
func (c *mockDataNodeClient) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest) (*commonpb.Status, error) {
func (c *mockDataNodeClient) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (c *mockDataNodeClient) CheckChannelOperationProgress(ctx context.Context, req *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error) {
func (c *mockDataNodeClient) CheckChannelOperationProgress(ctx context.Context, req *datapb.ChannelWatchInfo, opts ...grpc.CallOption) (*datapb.ChannelOperationProgressResponse, error) {
return &datapb.ChannelOperationProgressResponse{Status: merr.Status(nil)}, nil
}
@ -308,58 +300,55 @@ func (c *mockDataNodeClient) Stop() error {
return nil
}
type mockRootCoordService struct {
type mockRootCoordClient struct {
state commonpb.StateCode
cnt int64
}
func (m *mockRootCoordService) RenameCollection(ctx context.Context, req *milvuspb.RenameCollectionRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) Close() error {
// TODO implement me
panic("implement me")
}
func (m *mockRootCoordService) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
func (m *mockRootCoordClient) RenameCollection(ctx context.Context, req *milvuspb.RenameCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
// TODO implement me
panic("implement me")
}
func (m *mockRootCoordService) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) {
panic("implement me")
}
func (m *mockRootCoordService) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) AlterAlias(ctx context.Context, req *milvuspb.AlterAliasRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func newMockRootCoordService() *mockRootCoordService {
return &mockRootCoordService{state: commonpb.StateCode_Healthy}
func (m *mockRootCoordClient) AlterAlias(ctx context.Context, req *milvuspb.AlterAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func newMockRootCoordClient() *mockRootCoordClient {
return &mockRootCoordClient{state: commonpb.StateCode_Healthy}
}
func (m *mockRootCoordClient) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return nil, nil
}
func (m *mockRootCoordService) Init() error {
return nil
}
func (m *mockRootCoordService) Start() error {
return nil
}
func (m *mockRootCoordService) Stop() error {
func (m *mockRootCoordClient) Stop() error {
m.state = commonpb.StateCode_Abnormal
return nil
}
func (m *mockRootCoordService) Register() error {
func (m *mockRootCoordClient) Register() error {
return nil
}
func (m *mockRootCoordService) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *mockRootCoordClient) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: 0,
@ -372,24 +361,24 @@ func (m *mockRootCoordService) GetComponentStates(ctx context.Context) (*milvusp
}, nil
}
func (m *mockRootCoordService) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *mockRootCoordClient) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
panic("not implemented") // TODO: Implement
}
// DDL request
func (m *mockRootCoordService) CreateCollection(ctx context.Context, req *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) CreateCollection(ctx context.Context, req *milvuspb.CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) DropCollection(ctx context.Context, req *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) DropCollection(ctx context.Context, req *milvuspb.DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) HasCollection(ctx context.Context, req *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
func (m *mockRootCoordClient) HasCollection(ctx context.Context, req *milvuspb.HasCollectionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) DescribeCollection(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (m *mockRootCoordClient) DescribeCollection(ctx context.Context, req *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
// return not exist
if req.CollectionID == -1 {
err := merr.WrapErrCollectionNotFound(req.GetCollectionID())
@ -407,46 +396,46 @@ func (m *mockRootCoordService) DescribeCollection(ctx context.Context, req *milv
}, nil
}
func (m *mockRootCoordService) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (m *mockRootCoordClient) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
return m.DescribeCollection(ctx, req)
}
func (m *mockRootCoordService) ShowCollections(ctx context.Context, req *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
func (m *mockRootCoordClient) ShowCollections(ctx context.Context, req *milvuspb.ShowCollectionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionsResponse, error) {
return &milvuspb.ShowCollectionsResponse{
Status: merr.Status(nil),
CollectionNames: []string{"test"},
}, nil
}
func (m *mockRootCoordService) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) ListDatabases(ctx context.Context, in *milvuspb.ListDatabasesRequest) (*milvuspb.ListDatabasesResponse, error) {
func (m *mockRootCoordClient) ListDatabases(ctx context.Context, in *milvuspb.ListDatabasesRequest, opts ...grpc.CallOption) (*milvuspb.ListDatabasesResponse, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) CreatePartition(ctx context.Context, req *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) CreatePartition(ctx context.Context, req *milvuspb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) DropPartition(ctx context.Context, req *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) DropPartition(ctx context.Context, req *milvuspb.DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) HasPartition(ctx context.Context, req *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
func (m *mockRootCoordClient) HasPartition(ctx context.Context, req *milvuspb.HasPartitionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
func (m *mockRootCoordClient) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) {
return &milvuspb.ShowPartitionsResponse{
Status: merr.Status(nil),
PartitionNames: []string{"_default"},
@ -454,12 +443,12 @@ func (m *mockRootCoordService) ShowPartitions(ctx context.Context, req *milvuspb
}, nil
}
func (m *mockRootCoordService) ShowPartitionsInternal(ctx context.Context, req *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
func (m *mockRootCoordClient) ShowPartitionsInternal(ctx context.Context, req *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) {
return m.ShowPartitions(ctx, req)
}
// global timestamp allocator
func (m *mockRootCoordService) AllocTimestamp(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
func (m *mockRootCoordClient) AllocTimestamp(ctx context.Context, req *rootcoordpb.AllocTimestampRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocTimestampResponse, error) {
if m.state != commonpb.StateCode_Healthy {
return &rootcoordpb.AllocTimestampResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}}, nil
}
@ -474,7 +463,7 @@ func (m *mockRootCoordService) AllocTimestamp(ctx context.Context, req *rootcoor
}, nil
}
func (m *mockRootCoordService) AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
func (m *mockRootCoordClient) AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocIDResponse, error) {
if m.state != commonpb.StateCode_Healthy {
return &rootcoordpb.AllocIDResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}}, nil
}
@ -487,48 +476,48 @@ func (m *mockRootCoordService) AllocID(ctx context.Context, req *rootcoordpb.All
}
// segment
func (m *mockRootCoordService) DescribeSegment(ctx context.Context, req *milvuspb.DescribeSegmentRequest) (*milvuspb.DescribeSegmentResponse, error) {
func (m *mockRootCoordClient) DescribeSegment(ctx context.Context, req *milvuspb.DescribeSegmentRequest, opts ...grpc.CallOption) (*milvuspb.DescribeSegmentResponse, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) ShowSegments(ctx context.Context, req *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) {
func (m *mockRootCoordClient) ShowSegments(ctx context.Context, req *milvuspb.ShowSegmentsRequest, opts ...grpc.CallOption) (*milvuspb.ShowSegmentsResponse, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) DescribeSegments(ctx context.Context, req *rootcoordpb.DescribeSegmentsRequest) (*rootcoordpb.DescribeSegmentsResponse, error) {
func (m *mockRootCoordClient) DescribeSegments(ctx context.Context, req *rootcoordpb.DescribeSegmentsRequest, opts ...grpc.CallOption) (*rootcoordpb.DescribeSegmentsResponse, error) {
panic("implement me")
}
func (m *mockRootCoordService) GetDdChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *mockRootCoordClient) GetDdChannel(ctx context.Context, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
Value: "ddchannel",
}, nil
}
func (m *mockRootCoordService) UpdateChannelTimeTick(ctx context.Context, req *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) {
func (m *mockRootCoordClient) UpdateChannelTimeTick(ctx context.Context, req *internalpb.ChannelTimeTickMsg, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) InvalidateCollectionMetaCache(ctx context.Context, req *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) InvalidateCollectionMetaCache(ctx context.Context, req *proxypb.InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) SegmentFlushCompleted(ctx context.Context, in *datapb.SegmentFlushCompletedMsg) (*commonpb.Status, error) {
func (m *mockRootCoordClient) SegmentFlushCompleted(ctx context.Context, in *datapb.SegmentFlushCompletedMsg, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (m *mockRootCoordService) AddNewSegment(ctx context.Context, in *datapb.SegmentMsg) (*commonpb.Status, error) {
func (m *mockRootCoordClient) AddNewSegment(ctx context.Context, in *datapb.SegmentMsg, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (m *mockRootCoordClient) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
return &internalpb.ShowConfigurationsResponse{
Status: merr.Status(nil),
}, nil
}
func (m *mockRootCoordService) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (m *mockRootCoordClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
// TODO(dragondriver): change the id, though it's not important in ut
nodeID := UniqueID(20210901)
@ -565,21 +554,21 @@ func (m *mockRootCoordService) GetMetrics(ctx context.Context, req *milvuspb.Get
}, nil
}
func (m *mockRootCoordService) Import(ctx context.Context, req *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error) {
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 *mockRootCoordService) GetImportState(ctx context.Context, req *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error) {
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 *mockRootCoordService) ListImportTasks(ctx context.Context, in *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error) {
func (m *mockRootCoordClient) ListImportTasks(ctx context.Context, in *milvuspb.ListImportTasksRequest, opts ...grpc.CallOption) (*milvuspb.ListImportTasksResponse, error) {
panic("not implemented") // TODO: Implement
}
func (m *mockRootCoordService) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) (*commonpb.Status, error) {
func (m *mockRootCoordClient) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
@ -721,55 +710,55 @@ func (t *mockCompactionTrigger) stop() {
panic("not implemented")
}
func (m *mockRootCoordService) CreateCredential(ctx context.Context, req *internalpb.CredentialInfo) (*commonpb.Status, error) {
func (m *mockRootCoordClient) CreateCredential(ctx context.Context, req *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) UpdateCredential(ctx context.Context, req *internalpb.CredentialInfo) (*commonpb.Status, error) {
func (m *mockRootCoordClient) UpdateCredential(ctx context.Context, req *internalpb.CredentialInfo, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCredentialRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCredentialRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) {
func (m *mockRootCoordClient) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest, opts ...grpc.CallOption) (*milvuspb.ListCredUsersResponse, error) {
panic("implement me")
}
func (m *mockRootCoordService) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest) (*rootcoordpb.GetCredentialResponse, error) {
func (m *mockRootCoordClient) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest, opts ...grpc.CallOption) (*rootcoordpb.GetCredentialResponse, error) {
panic("implement me")
}
func (m *mockRootCoordService) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserRoleRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest) (*milvuspb.SelectRoleResponse, error) {
func (m *mockRootCoordClient) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest, opts ...grpc.CallOption) (*milvuspb.SelectRoleResponse, error) {
panic("implement me")
}
func (m *mockRootCoordService) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest) (*milvuspb.SelectUserResponse, error) {
func (m *mockRootCoordClient) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest, opts ...grpc.CallOption) (*milvuspb.SelectUserResponse, error) {
panic("implement me")
}
func (m *mockRootCoordService) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePrivilegeRequest) (*commonpb.Status, error) {
func (m *mockRootCoordClient) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePrivilegeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (m *mockRootCoordService) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) {
func (m *mockRootCoordClient) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantRequest, opts ...grpc.CallOption) (*milvuspb.SelectGrantResponse, error) {
panic("implement me")
}
func (m *mockRootCoordService) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) {
func (m *mockRootCoordClient) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest, opts ...grpc.CallOption) (*internalpb.ListPolicyResponse, error) {
return &internalpb.ListPolicyResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}}, nil
}

View File

@ -149,7 +149,7 @@ func TestLastExpireReset(t *testing.T) {
paramtable.Init()
Params.Save(Params.DataCoordCfg.AllocLatestExpireAttempt.Key, "1")
Params.Save(Params.DataCoordCfg.SegmentMaxSize.Key, "1")
mockAllocator := newRootCoordAllocator(newMockRootCoordService())
mockAllocator := newRootCoordAllocator(newMockRootCoordClient())
etcdCli, _ := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),

View File

@ -82,11 +82,11 @@ type (
Timestamp = typeutil.Timestamp
)
type dataNodeCreatorFunc func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error)
type dataNodeCreatorFunc func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error)
type indexNodeCreatorFunc func(ctx context.Context, addr string, nodeID int64) (types.IndexNode, error)
type indexNodeCreatorFunc func(ctx context.Context, addr string, nodeID int64) (types.IndexNodeClient, error)
type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdClient *clientv3.Client) (types.RootCoord, error)
type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdClient *clientv3.Client) (types.RootCoordClient, error)
// makes sure Server implements `DataCoord`
var _ types.DataCoord = (*Server)(nil)
@ -115,7 +115,7 @@ type Server struct {
cluster *Cluster
sessionManager *SessionManager
channelManager *ChannelManager
rootCoordClient types.RootCoord
rootCoordClient types.RootCoordClient
garbageCollector *garbageCollector
gcOpt GcOption
handler Handler
@ -227,15 +227,15 @@ func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Optio
return s
}
func defaultDataNodeCreatorFunc(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
func defaultDataNodeCreatorFunc(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return datanodeclient.NewClient(ctx, addr, nodeID)
}
func defaultIndexNodeCreatorFunc(ctx context.Context, addr string, nodeID int64) (types.IndexNode, error) {
func defaultIndexNodeCreatorFunc(ctx context.Context, addr string, nodeID int64) (types.IndexNodeClient, error) {
return indexnodeclient.NewClient(ctx, addr, nodeID, Params.DataCoordCfg.WithCredential.GetAsBool())
}
func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, client *clientv3.Client) (types.RootCoord, error) {
func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, client *clientv3.Client) (types.RootCoordClient, error) {
return rootcoordclient.NewClient(ctx, metaRootPath, client)
}
@ -432,15 +432,15 @@ func (s *Server) SetTiKVClient(client *txnkv.Client) {
s.tikvCli = client
}
func (s *Server) SetRootCoord(rootCoord types.RootCoord) {
func (s *Server) SetRootCoordClient(rootCoord types.RootCoordClient) {
s.rootCoordClient = rootCoord
}
func (s *Server) SetDataNodeCreator(f func(context.Context, string, int64) (types.DataNode, error)) {
func (s *Server) SetDataNodeCreator(f func(context.Context, string, int64) (types.DataNodeClient, error)) {
s.dataNodeCreator = f
}
func (s *Server) SetIndexNodeCreator(f func(context.Context, string, int64) (types.IndexNode, error)) {
func (s *Server) SetIndexNodeCreator(f func(context.Context, string, int64) (types.IndexNodeClient, error)) {
s.indexNodeCreator = f
}
@ -1014,10 +1014,7 @@ func (s *Server) initRootCoordClient() error {
return err
}
}
if err = s.rootCoordClient.Init(); err != nil {
return err
}
return s.rootCoordClient.Start()
return nil
}
// Stop do the Server finalize processes

View File

@ -36,12 +36,12 @@ import (
"github.com/stretchr/testify/require"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
"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/indexnode"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -51,6 +51,7 @@ import (
"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"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
@ -94,7 +95,7 @@ func TestGetSegmentInfoChannel(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
t.Run("get segment info channel", func(t *testing.T) {
resp, err := svr.GetSegmentInfoChannel(context.TODO())
resp, err := svr.GetSegmentInfoChannel(context.TODO(), nil)
assert.NoError(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.EqualValues(t, Params.CommonCfg.DataCoordSegmentInfo.GetValue(), resp.Value)
@ -193,8 +194,8 @@ func TestAssignSegmentID(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClient = &mockRootCoord{
RootCoord: svr.rootCoordClient,
collID: collID,
RootCoordClient: svr.rootCoordClient,
collID: collID,
}
schema := newTestSchema()
@ -221,11 +222,11 @@ func TestAssignSegmentID(t *testing.T) {
}
type mockRootCoord struct {
types.RootCoord
types.RootCoordClient
collID UniqueID
}
func (r *mockRootCoord) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (r *mockRootCoord) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
if req.CollectionID != r.collID {
return &milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{
@ -234,10 +235,10 @@ func (r *mockRootCoord) DescribeCollectionInternal(ctx context.Context, req *mil
},
}, nil
}
return r.RootCoord.DescribeCollection(ctx, req)
return r.RootCoordClient.DescribeCollection(ctx, req)
}
func (r *mockRootCoord) ReportImport(context.Context, *rootcoordpb.ImportResult) (*commonpb.Status, error) {
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",
@ -365,7 +366,7 @@ func TestFlush(t *testing.T) {
func TestGetTimeTickChannel(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
resp, err := svr.GetTimeTickChannel(context.TODO())
resp, err := svr.GetTimeTickChannel(context.TODO(), nil)
assert.NoError(t, err)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.EqualValues(t, Params.CommonCfg.DataCoordTimeTick.GetValue(), resp.Value)
@ -716,7 +717,7 @@ func TestGetSegmentInfo(t *testing.T) {
func TestGetComponentStates(t *testing.T) {
svr := &Server{}
resp, err := svr.GetComponentStates(context.Background())
resp, err := svr.GetComponentStates(context.Background(), nil)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, common.NotRegisteredID, resp.State.NodeID)
@ -733,7 +734,7 @@ func TestGetComponentStates(t *testing.T) {
}
for _, tc := range cases {
svr.stateCode.Store(tc.state)
resp, err := svr.GetComponentStates(context.Background())
resp, err := svr.GetComponentStates(context.Background(), nil)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, tc.code, resp.GetState().GetStateCode())
@ -2421,11 +2422,9 @@ func TestGetQueryVChanPositions_Retrieve_unIndexed(t *testing.T) {
func TestShouldDropChannel(t *testing.T) {
type myRootCoord struct {
mocks.RootCoord
mocks.MockRootCoordClient
}
myRoot := &myRootCoord{}
myRoot.EXPECT().Init().Return(nil)
myRoot.EXPECT().Start().Return(nil)
myRoot.EXPECT().AllocTimestamp(mock.Anything, mock.Anything).Return(&rootcoordpb.AllocTimestampResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Timestamp: tsoutil.ComposeTSByTime(time.Now(), 0),
@ -2438,7 +2437,7 @@ func TestShouldDropChannel(t *testing.T) {
Count: 1,
}, nil)
var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, etcdClient *clientv3.Client) (types.RootCoord, error) {
var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, etcdClient *clientv3.Client) (types.RootCoordClient, error) {
return myRoot, nil
}
@ -2521,8 +2520,8 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
req := &datapb.GetRecoveryInfoRequest{
@ -2565,8 +2564,8 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -2670,8 +2669,8 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -2749,8 +2748,8 @@ func TestGetRecoveryInfo(t *testing.T) {
Schema: newTestSchema(),
})
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
binlogReq := &datapb.SaveBinlogPathsRequest{
@ -2846,8 +2845,8 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -2888,8 +2887,8 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -2929,8 +2928,8 @@ func TestGetRecoveryInfo(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -3206,7 +3205,7 @@ func TestOptions(t *testing.T) {
t.Run("WithRootCoordCreator", func(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, etcdClient *clientv3.Client) (types.RootCoord, error) {
var crt rootCoordCreatorFunc = func(ctx context.Context, metaRoot string, etcdClient *clientv3.Client) (types.RootCoordClient, error) {
return nil, errors.New("dummy")
}
opt := WithRootCoordCreator(crt)
@ -3237,7 +3236,7 @@ func TestOptions(t *testing.T) {
t.Run("WithDataNodeCreator", func(t *testing.T) {
var target int64
val := rand.Int63()
opt := WithDataNodeCreator(func(context.Context, string, int64) (types.DataNode, error) {
opt := WithDataNodeCreator(func(context.Context, string, int64) (types.DataNodeClient, error) {
target = val
return nil, nil
})
@ -3341,7 +3340,7 @@ func TestHandleSessionEvent(t *testing.T) {
}
type rootCoordSegFlushComplete struct {
mockRootCoordService
mockRootCoordClient
flag bool
}
@ -3670,15 +3669,15 @@ func TestGetFlushAllState(t *testing.T) {
}
var err error
svr.meta = &meta{}
svr.rootCoordClient = mocks.NewRootCoord(t)
svr.rootCoordClient = mocks.NewMockRootCoordClient(t)
svr.broker = NewCoordinatorBroker(svr.rootCoordClient)
if test.ListDatabaseFailed {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError},
}, nil).Maybe()
} else {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{
DbNames: []string{"db1"},
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
@ -3686,12 +3685,12 @@ func TestGetFlushAllState(t *testing.T) {
}
if test.ShowCollectionFailed {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().ShowCollections(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().ShowCollections(mock.Anything, mock.Anything).
Return(&milvuspb.ShowCollectionsResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError},
}, nil).Maybe()
} else {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().ShowCollections(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().ShowCollections(mock.Anything, mock.Anything).
Return(&milvuspb.ShowCollectionsResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
CollectionIds: []int64{collection},
@ -3699,12 +3698,12 @@ func TestGetFlushAllState(t *testing.T) {
}
if test.DescribeCollectionFailed {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).
Return(&milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError},
}, nil).Maybe()
} else {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).
Return(&milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
VirtualChannelNames: vchannels,
@ -3754,30 +3753,30 @@ func TestGetFlushAllStateWithDB(t *testing.T) {
svr.stateCode.Store(commonpb.StateCode_Healthy)
var err error
svr.meta = &meta{}
svr.rootCoordClient = mocks.NewRootCoord(t)
svr.rootCoordClient = mocks.NewMockRootCoordClient(t)
svr.broker = NewCoordinatorBroker(svr.rootCoordClient)
if test.DbExist {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{
DbNames: []string{dbName},
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
}, nil).Maybe()
} else {
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{
DbNames: []string{},
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
}, nil).Maybe()
}
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().ShowCollections(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().ShowCollections(mock.Anything, mock.Anything).
Return(&milvuspb.ShowCollectionsResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
CollectionIds: []int64{collectionID},
}, nil).Maybe()
svr.rootCoordClient.(*mocks.RootCoord).EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).
svr.rootCoordClient.(*mocks.MockRootCoordClient).EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).
Return(&milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
VirtualChannelNames: vchannels,
@ -4200,11 +4199,11 @@ func newTestServer(t *testing.T, receiveCh chan any, opts ...Option) *Server {
svr.SetEtcdClient(etcdCli)
svr.SetTiKVClient(global_test_tikv)
svr.dataNodeCreator = func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
svr.dataNodeCreator = func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(0, receiveCh)
}
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
for _, opt := range opts {
@ -4254,11 +4253,11 @@ func newTestServerWithMeta(t *testing.T, receiveCh chan any, meta *meta, opts ..
svr.SetEtcdClient(etcdCli)
svr.SetTiKVClient(global_test_tikv)
svr.dataNodeCreator = func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
svr.dataNodeCreator = func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(0, receiveCh)
}
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
// indexCoord := mocks.NewMockIndexCoord(t)
// indexCoord.EXPECT().GetIndexInfos(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
@ -4311,11 +4310,11 @@ func newTestServer2(t *testing.T, receiveCh chan any, opts ...Option) *Server {
svr.SetEtcdClient(etcdCli)
svr.SetTiKVClient(global_test_tikv)
svr.dataNodeCreator = func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
svr.dataNodeCreator = func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(0, receiveCh)
}
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
err = svr.Init()
@ -4357,7 +4356,7 @@ func Test_CheckHealth(t *testing.T) {
data map[int64]*Session
}{data: map[int64]*Session{1: {
client: healthClient,
clientCreator: func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
clientCreator: func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return healthClient, nil
},
}}}
@ -4383,7 +4382,7 @@ func Test_CheckHealth(t *testing.T) {
data map[int64]*Session
}{data: map[int64]*Session{1: {
client: unhealthClient,
clientCreator: func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
clientCreator: func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return unhealthClient, nil
},
}}}
@ -4507,13 +4506,13 @@ func testDataCoordBase(t *testing.T, opts ...Option) *Server {
svr.SetEtcdClient(etcdCli)
svr.SetTiKVClient(global_test_tikv)
svr.SetDataNodeCreator(func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
svr.SetDataNodeCreator(func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return newMockDataNodeClient(0, nil)
})
svr.SetIndexNodeCreator(func(ctx context.Context, addr string, nodeID int64) (types.IndexNode, error) {
return indexnode.NewMockIndexNodeComponent(ctx)
svr.SetIndexNodeCreator(func(ctx context.Context, addr string, nodeID int64) (types.IndexNodeClient, error) {
return &grpcmock.GrpcIndexNodeClient{Err: nil}, nil
})
svr.SetRootCoord(newMockRootCoordService())
svr.SetRootCoordClient(newMockRootCoordClient())
err = svr.Init()
assert.NoError(t, err)
@ -4522,7 +4521,7 @@ func testDataCoordBase(t *testing.T, opts ...Option) *Server {
err = svr.Register()
assert.NoError(t, err)
resp, err := svr.GetComponentStates(context.Background())
resp, err := svr.GetComponentStates(context.Background(), nil)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.Equal(t, commonpb.StateCode_Healthy, resp.GetState().GetStateCode())

View File

@ -53,7 +53,7 @@ func (s *Server) isClosed() bool {
}
// GetTimeTickChannel legacy API, returns time tick channel name
func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (s *Server) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
Value: Params.CommonCfg.DataCoordTimeTick.GetValue(),
@ -61,7 +61,7 @@ func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringRespon
}
// GetStatisticsChannel legacy API, returns statistics channel name
func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -357,7 +357,7 @@ func (s *Server) GetPartitionStatistics(ctx context.Context, req *datapb.GetPart
}
// GetSegmentInfoChannel legacy API, returns segment info statistics channel
func (s *Server) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (s *Server) GetSegmentInfoChannel(ctx context.Context, req *datapb.GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
Value: Params.CommonCfg.DataCoordSegmentInfo.GetValue(),
@ -618,7 +618,7 @@ func (s *Server) GetStateCode() commonpb.StateCode {
}
// GetComponentStates returns DataCoord's current state
func (s *Server) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
code := s.GetStateCode()
nodeID := common.NotRegisteredID
if s.session != nil && s.session.Registered() {
@ -1726,7 +1726,7 @@ func (s *Server) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthReque
return err
}
sta, err := cli.GetComponentStates(ctx)
sta, err := cli.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
isHealthy, reason := errorutil.UnHealthReasonWithComponentStatesOrErr("datanode", nodeID, sta, err)
if !isHealthy {
mu.Lock()

View File

@ -102,8 +102,8 @@ func TestGetRecoveryInfoV2(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
req := &datapb.GetRecoveryInfoRequestV2{
@ -145,8 +145,8 @@ func TestGetRecoveryInfoV2(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -249,8 +249,8 @@ func TestGetRecoveryInfoV2(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -326,8 +326,8 @@ func TestGetRecoveryInfoV2(t *testing.T) {
Schema: newTestSchema(),
})
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
binlogReq := &datapb.SaveBinlogPathsRequest{
@ -423,8 +423,8 @@ func TestGetRecoveryInfoV2(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -464,8 +464,8 @@ func TestGetRecoveryInfoV2(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{
@ -504,8 +504,8 @@ func TestGetRecoveryInfoV2(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) {
return newMockRootCoordService(), nil
svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoordClient, error) {
return newMockRootCoordClient(), nil
}
svr.meta.AddCollection(&collectionInfo{

View File

@ -38,7 +38,7 @@ type NodeInfo struct {
type Session struct {
sync.Mutex
info *NodeInfo
client types.DataNode
client types.DataNodeClient
clientCreator dataNodeCreatorFunc
isDisposed bool
}
@ -52,7 +52,7 @@ func NewSession(info *NodeInfo, creator dataNodeCreatorFunc) *Session {
}
// GetOrCreateClient gets or creates a new client for session
func (n *Session) GetOrCreateClient(ctx context.Context) (types.DataNode, error) {
func (n *Session) GetOrCreateClient(ctx context.Context) (types.DataNodeClient, error) {
n.Lock()
defer n.Unlock()
@ -76,10 +76,7 @@ func (n *Session) initClient(ctx context.Context) (err error) {
if n.client, err = n.clientCreator(ctx, n.info.Address, n.info.NodeID); err != nil {
return
}
if err = n.client.Init(); err != nil {
return
}
return n.client.Start()
return nil
}
// Dispose releases client connection
@ -88,7 +85,7 @@ func (n *Session) Dispose() {
defer n.Unlock()
if n.client != nil {
n.client.Stop()
n.client.Close()
n.client = nil
}
n.isDisposed = true

View File

@ -61,7 +61,7 @@ func withSessionCreator(creator dataNodeCreatorFunc) SessionOpt {
}
func defaultSessionCreator() dataNodeCreatorFunc {
return func(ctx context.Context, addr string, nodeID int64) (types.DataNode, error) {
return func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) {
return grpcdatanodeclient.NewClient(ctx, addr, nodeID)
}
}
@ -323,7 +323,7 @@ func (c *SessionManager) FlushChannels(ctx context.Context, nodeID int64, req *d
return nil
}
func (c *SessionManager) getClient(ctx context.Context, nodeID int64) (types.DataNode, error) {
func (c *SessionManager) getClient(ctx context.Context, nodeID int64) (types.DataNodeClient, error) {
c.sessions.RLock()
session, ok := c.sessions.data[nodeID]
c.sessions.RUnlock()

View File

@ -45,7 +45,7 @@ type Impl struct {
*gAllocator.IDAllocator
}
func New(ctx context.Context, rootCoord types.RootCoord, peerID UniqueID) (Allocator, error) {
func New(ctx context.Context, rootCoord types.RootCoordClient, peerID UniqueID) (Allocator, error) {
idAlloc, err := gAllocator.NewIDAllocator(ctx, rootCoord, peerID)
if err != nil {
return nil, err

View File

@ -22,6 +22,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/types"
@ -77,11 +78,11 @@ func TestGetGenerator(t *testing.T) {
}
type RootCoordFactory struct {
types.RootCoord
types.RootCoordClient
ID UniqueID
}
func (m *RootCoordFactory) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
func (m *RootCoordFactory) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocIDResponse, error) {
resp := &rootcoordpb.AllocIDResponse{
ID: m.ID,
Count: in.GetCount(),

View File

@ -149,7 +149,7 @@ type addSegmentReq struct {
var _ Channel = &ChannelMeta{}
func newChannel(channelName string, collID UniqueID, schema *schemapb.CollectionSchema, rc types.RootCoord, cm storage.ChunkManager) *ChannelMeta {
func newChannel(channelName string, collID UniqueID, schema *schemapb.CollectionSchema, rc types.RootCoordClient, cm storage.ChunkManager) *ChannelMeta {
metaService := newMetaService(rc, collID)
channel := ChannelMeta{

View File

@ -274,7 +274,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
collectionID := int64(1)
meta := NewMetaFactory().GetCollectionMeta(collectionID, "test", schemapb.DataType_Int64)
rc := &mocks.RootCoord{}
rc := &mocks.MockRootCoordClient{}
rc.EXPECT().DescribeCollection(mock.Anything, mock.Anything).
Return(&milvuspb.DescribeCollectionResponse{
Schema: meta.GetSchema(),

View File

@ -95,8 +95,8 @@ type DataNode struct {
etcdCli *clientv3.Client
address string
rootCoord types.RootCoord
dataCoord types.DataCoord
rootCoord types.RootCoordClient
dataCoord types.DataCoordClient
// call once
initOnce sync.Once
@ -155,8 +155,8 @@ func (node *DataNode) SetEtcdClient(etcdCli *clientv3.Client) {
node.etcdCli = etcdCli
}
// SetRootCoord sets RootCoord's grpc client, error is returned if repeatedly set.
func (node *DataNode) SetRootCoord(rc types.RootCoord) error {
// SetRootCoordClient sets RootCoord's grpc client, error is returned if repeatedly set.
func (node *DataNode) SetRootCoordClient(rc types.RootCoordClient) error {
switch {
case rc == nil, node.rootCoord != nil:
return errors.New("nil parameter or repeatedly set")
@ -166,8 +166,8 @@ func (node *DataNode) SetRootCoord(rc types.RootCoord) error {
}
}
// SetDataCoord sets data service's grpc client, error is returned if repeatedly set.
func (node *DataNode) SetDataCoord(ds types.DataCoord) error {
// SetDataCoordClient sets data service's grpc client, error is returned if repeatedly set.
func (node *DataNode) SetDataCoordClient(ds types.DataCoordClient) error {
switch {
case ds == nil, node.dataCoord != nil:
return errors.New("nil parameter or repeatedly set")

View File

@ -108,10 +108,10 @@ func TestDataNode(t *testing.T) {
paramtable.SetNodeID(1)
defer cancel()
t.Run("Test SetRootCoord", func(t *testing.T) {
t.Run("Test SetRootCoordClient", func(t *testing.T) {
emptyDN := &DataNode{}
tests := []struct {
inrc types.RootCoord
inrc types.RootCoordClient
isvalid bool
description string
}{
@ -121,7 +121,7 @@ func TestDataNode(t *testing.T) {
for _, test := range tests {
t.Run(test.description, func(t *testing.T) {
err := emptyDN.SetRootCoord(test.inrc)
err := emptyDN.SetRootCoordClient(test.inrc)
if test.isvalid {
assert.NoError(t, err)
} else {
@ -131,10 +131,10 @@ func TestDataNode(t *testing.T) {
}
})
t.Run("Test SetDataCoord", func(t *testing.T) {
t.Run("Test SetDataCoordClient", func(t *testing.T) {
emptyDN := &DataNode{}
tests := []struct {
inrc types.DataCoord
inrc types.DataCoordClient
isvalid bool
description string
}{
@ -144,7 +144,7 @@ func TestDataNode(t *testing.T) {
for _, test := range tests {
t.Run(test.description, func(t *testing.T) {
err := emptyDN.SetDataCoord(test.inrc)
err := emptyDN.SetDataCoordClient(test.inrc)
if test.isvalid {
assert.NoError(t, err)
} else {

View File

@ -54,8 +54,8 @@ type dataSyncService struct {
msFactory msgstream.Factory
collectionID UniqueID // collection id of vchan for which this data sync service serves
vchannelName string
dataCoord types.DataCoord // DataCoord instance to interact with
clearSignal chan<- string // signal channel to notify flowgraph close for collection/partition drop msg consumed
dataCoord types.DataCoordClient // DataCoord instance to interact with
clearSignal chan<- string // signal channel to notify flowgraph close for collection/partition drop msg consumed
delBufferManager *DeltaBufferManager
flushingSegCache *Cache // a guarding cache stores currently flushing segment ids
@ -79,7 +79,7 @@ func newDataSyncService(
factory msgstream.Factory,
vchan *datapb.VchannelInfo,
clearSignal chan<- string,
dataCoord types.DataCoord,
dataCoord types.DataCoordClient,
flushingSegCache *Cache,
chunkManager storage.ChunkManager,
compactor *compactionExecutor,

View File

@ -31,6 +31,7 @@ import (
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
"go.uber.org/atomic"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -941,11 +942,11 @@ func TestInsertBufferNodeSuite(t *testing.T) {
// CompactedRootCoord has meta info compacted at ts
type CompactedRootCoord struct {
types.RootCoord
types.RootCoordClient
compactTs Timestamp
}
func (m *CompactedRootCoord) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (m *CompactedRootCoord) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
if in.TimeStamp != 0 && in.GetTimeStamp() <= m.compactTs {
return &milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{
@ -954,7 +955,7 @@ func (m *CompactedRootCoord) DescribeCollection(ctx context.Context, in *milvusp
},
}, nil
}
return m.RootCoord.DescribeCollection(ctx, in)
return m.RootCoordClient.DescribeCollection(ctx, in)
}
func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
@ -986,8 +987,8 @@ func TestInsertBufferNode_bufferInsertMsg(t *testing.T) {
pkType: test.pkType,
}
mockRootCoord := &CompactedRootCoord{
RootCoord: rcf,
compactTs: 100,
RootCoordClient: rcf,
compactTs: 100,
}
channel := newChannel(insertChannelName, collMeta.ID, collMeta.Schema, mockRootCoord, cm)

View File

@ -51,7 +51,7 @@ type ttNode struct {
vChannelName string
channel Channel
lastUpdateTime *atomic.Time
dataCoord types.DataCoord
dataCoord types.DataCoordClient
updateCPLock sync.Mutex
notifyChannel chan checkPoint
@ -149,7 +149,7 @@ func (ttn *ttNode) updateChannelCP(channelPos *msgpb.MsgPosition, curTs time.Tim
return nil
}
func newTTNode(config *nodeConfig, dc types.DataCoord) (*ttNode, error) {
func newTTNode(config *nodeConfig, dc types.DataCoordClient) (*ttNode, error) {
baseNode := BaseNode{}
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())

View File

@ -39,11 +39,11 @@ import (
type metaService struct {
channel Channel
collectionID UniqueID
rootCoord types.RootCoord
rootCoord types.RootCoordClient
}
// newMetaService creates a new metaService with provided RootCoord and collectionID.
func newMetaService(rc types.RootCoord, collectionID UniqueID) *metaService {
func newMetaService(rc types.RootCoordClient, collectionID UniqueID) *metaService {
return &metaService{
rootCoord: rc,
collectionID: collectionID,

View File

@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -66,7 +67,7 @@ type RootCoordFails1 struct {
}
// DescribeCollectionInternal override method that will fails
func (rc *RootCoordFails1) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (rc *RootCoordFails1) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
return nil, errors.New("always fail")
}
@ -76,7 +77,7 @@ type RootCoordFails2 struct {
}
// DescribeCollectionInternal override method that will fails
func (rc *RootCoordFails2) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (rc *RootCoordFails2) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
return &milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError},
}, nil

View File

@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -159,7 +160,7 @@ type DataFactory struct {
}
type RootCoordFactory struct {
types.RootCoord
types.RootCoordClient
ID UniqueID
collectionName string
collectionID UniqueID
@ -175,7 +176,7 @@ type RootCoordFactory struct {
}
type DataCoordFactory struct {
types.DataCoord
types.DataCoordClient
SaveBinlogPathError bool
SaveBinlogPathStatus commonpb.ErrorCode
@ -198,7 +199,7 @@ type DataCoordFactory struct {
ReportDataNodeTtMsgsNotSuccess bool
}
func (ds *DataCoordFactory) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
func (ds *DataCoordFactory) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegmentIDResponse, error) {
if ds.AddSegmentError {
return nil, errors.New("Error")
}
@ -220,7 +221,7 @@ func (ds *DataCoordFactory) AssignSegmentID(ctx context.Context, req *datapb.Ass
return res, nil
}
func (ds *DataCoordFactory) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) {
func (ds *DataCoordFactory) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
if ds.CompleteCompactionError {
return nil, errors.New("Error")
}
@ -231,14 +232,14 @@ func (ds *DataCoordFactory) CompleteCompaction(ctx context.Context, req *datapb.
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (ds *DataCoordFactory) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
if ds.SaveBinlogPathError {
return nil, errors.New("Error")
}
return &commonpb.Status{ErrorCode: ds.SaveBinlogPathStatus}, nil
}
func (ds *DataCoordFactory) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) {
func (ds *DataCoordFactory) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest, opts ...grpc.CallOption) (*datapb.DropVirtualChannelResponse, error) {
if ds.DropVirtualChannelError {
return nil, errors.New("error")
}
@ -249,15 +250,15 @@ func (ds *DataCoordFactory) DropVirtualChannel(ctx context.Context, req *datapb.
}, nil
}
func (ds *DataCoordFactory) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (ds *DataCoordFactory) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (ds *DataCoordFactory) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
if ds.ReportDataNodeTtMsgsError {
return nil, errors.New("mock ReportDataNodeTtMsgs error")
}
@ -269,29 +270,29 @@ func (ds *DataCoordFactory) ReportDataNodeTtMsgs(ctx context.Context, req *datap
return merr.Status(nil), nil
}
func (ds *DataCoordFactory) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (ds *DataCoordFactory) UnsetIsImportingState(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (ds *DataCoordFactory) MarkSegmentsDropped(context.Context, *datapb.MarkSegmentsDroppedRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (ds *DataCoordFactory) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
func (ds *DataCoordFactory) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (ds *DataCoordFactory) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
func (ds *DataCoordFactory) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) {
return &milvuspb.CheckHealthResponse{
IsHealthy: true,
}, nil
}
func (ds *DataCoordFactory) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
func (ds *DataCoordFactory) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest, opts ...grpc.CallOption) (*datapb.GetSegmentInfoResponse, error) {
if ds.GetSegmentInfosError {
return nil, errors.New("mock get segment info error")
}
@ -945,7 +946,7 @@ func (m *RootCoordFactory) setCollectionName(name string) {
m.collectionName = name
}
func (m *RootCoordFactory) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
func (m *RootCoordFactory) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocIDResponse, error) {
resp := &rootcoordpb.AllocIDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -973,7 +974,7 @@ func (m *RootCoordFactory) AllocID(ctx context.Context, in *rootcoordpb.AllocIDR
return resp, nil
}
func (m *RootCoordFactory) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
func (m *RootCoordFactory) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimestampRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocTimestampResponse, error) {
resp := &rootcoordpb.AllocTimestampResponse{
Status: &commonpb.Status{},
Timestamp: 1000,
@ -988,7 +989,7 @@ func (m *RootCoordFactory) AllocTimestamp(ctx context.Context, in *rootcoordpb.A
return resp, nil
}
func (m *RootCoordFactory) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
func (m *RootCoordFactory) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionsResponse, error) {
resp := &milvuspb.ShowCollectionsResponse{
Status: &commonpb.Status{},
CollectionNames: []string{m.collectionName},
@ -996,7 +997,7 @@ func (m *RootCoordFactory) ShowCollections(ctx context.Context, in *milvuspb.Sho
return resp, nil
}
func (m *RootCoordFactory) DescribeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (m *RootCoordFactory) DescribeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
f := MetaFactory{}
meta := f.GetCollectionMeta(m.collectionID, m.collectionName, m.pkType)
resp := &milvuspb.DescribeCollectionResponse{
@ -1022,7 +1023,7 @@ func (m *RootCoordFactory) DescribeCollectionInternal(ctx context.Context, in *m
return resp, nil
}
func (m *RootCoordFactory) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
func (m *RootCoordFactory) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) {
if m.ShowPartitionsErr {
return &milvuspb.ShowPartitionsResponse{
Status: merr.Status(nil),
@ -1045,7 +1046,7 @@ func (m *RootCoordFactory) ShowPartitions(ctx context.Context, req *milvuspb.Sho
}, nil
}
func (m *RootCoordFactory) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *RootCoordFactory) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{},
SubcomponentStates: make([]*milvuspb.ComponentInfo, 0),
@ -1053,7 +1054,7 @@ func (m *RootCoordFactory) GetComponentStates(ctx context.Context) (*milvuspb.Co
}, nil
}
func (m *RootCoordFactory) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) (*commonpb.Status, error) {
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")

View File

@ -66,7 +66,7 @@ func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmCha
}
// GetComponentStates will return current state of DataNode
func (node *DataNode) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (node *DataNode) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
log.Debug("DataNode current state", zap.Any("State", node.stateCode.Load()))
nodeID := common.NotRegisteredID
if node.GetSession() != nil && node.session.Registered() {
@ -179,14 +179,14 @@ func (node *DataNode) ResendSegmentStats(ctx context.Context, req *datapb.Resend
}
// GetTimeTickChannel currently do nothing
func (node *DataNode) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (node *DataNode) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
}, nil
}
// GetStatisticsChannel currently do nothing
func (node *DataNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (node *DataNode) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
}, nil

View File

@ -121,25 +121,25 @@ func (s *DataNodeServicesSuite) TestNotInUseAPIs() {
s.Assert().True(merr.Ok(status))
})
s.Run("GetTimeTickChannel", func() {
_, err := s.node.GetTimeTickChannel(s.ctx)
_, err := s.node.GetTimeTickChannel(s.ctx, nil)
s.Assert().NoError(err)
})
s.Run("GetStatisticsChannel", func() {
_, err := s.node.GetStatisticsChannel(s.ctx)
_, err := s.node.GetStatisticsChannel(s.ctx, nil)
s.Assert().NoError(err)
})
}
func (s *DataNodeServicesSuite) TestGetComponentStates() {
resp, err := s.node.GetComponentStates(s.ctx)
resp, err := s.node.GetComponentStates(s.ctx, nil)
s.Assert().NoError(err)
s.Assert().True(merr.Ok(resp.GetStatus()))
s.Assert().Equal(common.NotRegisteredID, resp.State.NodeID)
s.node.SetSession(&sessionutil.Session{})
s.node.session.UpdateRegistered(true)
resp, err = s.node.GetComponentStates(context.Background())
resp, err = s.node.GetComponentStates(context.Background(), nil)
s.Assert().NoError(err)
s.Assert().True(merr.Ok(resp.GetStatus()))
}

View File

@ -39,7 +39,7 @@ import (
// after send succeeds will clean the cache earlier than the sended timestamp
type timeTickSender struct {
nodeID int64
dataCoord types.DataCoord
dataCoord types.DataCoordClient
mu sync.Mutex
channelStatesCaches map[string]*segmentStatesSequence // string -> *segmentStatesSequence
@ -50,7 +50,7 @@ type segmentStatesSequence struct {
data map[uint64][]*commonpb.SegmentStats // ts -> segmentStats
}
func newTimeTickSender(dataCoord types.DataCoord, nodeID int64) *timeTickSender {
func newTimeTickSender(dataCoord types.DataCoordClient, nodeID int64) *timeTickSender {
return &timeTickSender{
nodeID: nodeID,
dataCoord: dataCoord,

View File

@ -24,6 +24,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"go.uber.org/atomic"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/mocks"
@ -167,11 +168,11 @@ func TestTimetickManagerSendNotSuccess(t *testing.T) {
func TestTimetickManagerSendReport(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
mockDataCoord := mocks.NewMockDataCoord(t)
mockDataCoord := mocks.NewMockDataCoordClient(t)
tsInMill := time.Now().UnixMilli()
validTs := atomic.NewBool(false)
mockDataCoord.EXPECT().ReportDataNodeTtMsgs(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) {
mockDataCoord.EXPECT().ReportDataNodeTtMsgs(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest, opt ...grpc.CallOption) {
if req.GetBase().Timestamp > uint64(tsInMill) {
validTs.Store(true)
}

View File

@ -41,7 +41,7 @@ import (
var Params *paramtable.ComponentParam = paramtable.Get()
var _ types.DataCoord = (*Client)(nil)
var _ types.DataCoordClient = (*Client)(nil)
// Client is the datacoord grpc client
type Client struct {
@ -93,26 +93,11 @@ func (c *Client) getDataCoordAddr() (string, error) {
return ms.Address, nil
}
// Init initializes the client
func (c *Client) Init() error {
return nil
}
// Start enables the client
func (c *Client) Start() error {
return nil
}
// Stop stops the client
func (c *Client) Stop() error {
func (c *Client) Close() error {
return c.grpcClient.Close()
}
// Register dummy
func (c *Client) Register() error {
return nil
}
func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(coordClient datapb.DataCoordClient) (*T, error)) (*T, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
@ -127,28 +112,28 @@ func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(coordClient d
}
// GetComponentStates calls DataCoord GetComponentStates services
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *Client) GetComponentStates(ctx context.Context, _ *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.ComponentStates, error) {
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
}
// GetTimeTickChannel return the name of time tick channel.
func (c *Client) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetTimeTickChannel(ctx context.Context, _ *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.StringResponse, error) {
return client.GetTimeTickChannel(ctx, &internalpb.GetTimeTickChannelRequest{})
})
}
// GetStatisticsChannel return the name of statistics channel.
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetStatisticsChannel(ctx context.Context, _ *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.StringResponse, error) {
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
}
// Flush flushes a collection's data
func (c *Client) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
func (c *Client) Flush(ctx context.Context, req *datapb.FlushRequest, opts ...grpc.CallOption) (*datapb.FlushResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -172,7 +157,7 @@ func (c *Client) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F
// `AssignSegmentID` will applies current configured allocation policies for each request
// if the VChannel is newly used, `WatchDmlChannels` will be invoked to notify a `DataNode`(selected by policy) to watch it
// if there is anything make the allocation impossible, the response will not contain the corresponding result
func (c *Client) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
func (c *Client) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegmentIDResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*datapb.AssignSegmentIDResponse, error) {
return client.AssignSegmentID(ctx, req)
})
@ -189,7 +174,7 @@ func (c *Client) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
// otherwise the Segment State and Start position information will be returned
//
// error is returned only when some communication issue occurs
func (c *Client) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
func (c *Client) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest, opts ...grpc.CallOption) (*datapb.GetSegmentStatesResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -210,7 +195,7 @@ func (c *Client) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentSta
// and corresponding binlog path list
//
// error is returned only when some communication issue occurs
func (c *Client) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
func (c *Client) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*datapb.GetInsertBinlogPathsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -231,7 +216,7 @@ func (c *Client) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsert
// only row count for now
//
// error is returned only when some communication issue occurs
func (c *Client) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) {
func (c *Client) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetCollectionStatisticsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -252,7 +237,7 @@ func (c *Client) GetCollectionStatistics(ctx context.Context, req *datapb.GetCol
// only row count for now
//
// error is returned only when some communication issue occurs
func (c *Client) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error) {
func (c *Client) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetPartitionStatisticsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -265,7 +250,7 @@ func (c *Client) GetPartitionStatistics(ctx context.Context, req *datapb.GetPart
// GetSegmentInfoChannel DEPRECATED
// legacy api to get SegmentInfo Channel name
func (c *Client) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetSegmentInfoChannel(ctx context.Context, _ *datapb.GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.StringResponse, error) {
return client.GetSegmentInfoChannel(ctx, &datapb.GetSegmentInfoChannelRequest{})
})
@ -278,7 +263,7 @@ func (c *Client) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringRes
//
// response struct `GetSegmentInfoResponse` contains the list of segment info
// error is returned only when some communication issue occurs
func (c *Client) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
func (c *Client) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest, opts ...grpc.CallOption) (*datapb.GetSegmentInfoResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -303,7 +288,7 @@ func (c *Client) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoR
//
// the root reason is each `SaveBinlogPaths` will overwrite the checkpoint position
// if the constraint is broken, the checkpoint position will not be monotonically increasing and the integrity will be compromised
func (c *Client) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
func (c *Client) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
// use Call here on purpose
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
@ -322,7 +307,7 @@ func (c *Client) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
//
// response struct `GetRecoveryInfoResponse` contains the list of segments info and corresponding vchannel info
// error is returned only when some communication issue occurs
func (c *Client) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
func (c *Client) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest, opts ...grpc.CallOption) (*datapb.GetRecoveryInfoResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -340,7 +325,7 @@ func (c *Client) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInf
//
// response struct `GetRecoveryInfoResponseV2` contains the list of segments info and corresponding vchannel info
// error is returned only when some communication issue occurs
func (c *Client) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryInfoRequestV2) (*datapb.GetRecoveryInfoResponseV2, error) {
func (c *Client) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryInfoRequestV2, opts ...grpc.CallOption) (*datapb.GetRecoveryInfoResponseV2, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -360,7 +345,7 @@ func (c *Client) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryI
//
// response struct `GetFlushedSegmentsResponse` contains flushed segment id list
// error is returned only when some communication issue occurs
func (c *Client) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
func (c *Client) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*datapb.GetFlushedSegmentsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -379,7 +364,7 @@ func (c *Client) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedS
//
// response struct `GetSegmentsByStatesResponse` contains segment id list
// error is returned only when some communication issue occurs
func (c *Client) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegmentsByStatesRequest) (*datapb.GetSegmentsByStatesResponse, error) {
func (c *Client) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegmentsByStatesRequest, opts ...grpc.CallOption) (*datapb.GetSegmentsByStatesResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -391,7 +376,7 @@ func (c *Client) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegment
}
// ShowConfigurations gets specified configurations para of DataCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -403,7 +388,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
}
// GetMetrics gets all metrics of datacoord
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -415,49 +400,49 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
}
// ManualCompaction triggers a compaction for a collection
func (c *Client) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
func (c *Client) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.ManualCompactionResponse, error) {
return client.ManualCompaction(ctx, req)
})
}
// GetCompactionState gets the state of a compaction
func (c *Client) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) {
func (c *Client) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.GetCompactionStateResponse, error) {
return client.GetCompactionState(ctx, req)
})
}
// GetCompactionStateWithPlans gets the state of a compaction by plan
func (c *Client) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error) {
func (c *Client) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.GetCompactionPlansResponse, error) {
return client.GetCompactionStateWithPlans(ctx, req)
})
}
// WatchChannels notifies DataCoord to watch vchannels of a collection
func (c *Client) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest) (*datapb.WatchChannelsResponse, error) {
func (c *Client) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest, opts ...grpc.CallOption) (*datapb.WatchChannelsResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*datapb.WatchChannelsResponse, error) {
return client.WatchChannels(ctx, req)
})
}
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
func (c *Client) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
func (c *Client) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.GetFlushStateResponse, error) {
return client.GetFlushState(ctx, req)
})
}
// GetFlushAllState checks if all DML messages before `FlushAllTs` have been flushed.
func (c *Client) GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error) {
func (c *Client) GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushAllStateResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.GetFlushAllStateResponse, error) {
return client.GetFlushAllState(ctx, req)
})
}
// DropVirtualChannel drops virtual channel in datacoord.
func (c *Client) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) {
func (c *Client) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest, opts ...grpc.CallOption) (*datapb.DropVirtualChannelResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -469,7 +454,7 @@ func (c *Client) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtual
}
// SetSegmentState sets the state of a given segment.
func (c *Client) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest) (*datapb.SetSegmentStateResponse, error) {
func (c *Client) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest, opts ...grpc.CallOption) (*datapb.SetSegmentStateResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -481,7 +466,7 @@ 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) (*datapb.ImportTaskResponse, error) {
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -493,7 +478,7 @@ func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*da
}
// UpdateSegmentStatistics is the client side caller of UpdateSegmentStatistics.
func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -505,7 +490,7 @@ func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update
}
// UpdateChannelCheckpoint updates channel checkpoint in dataCoord.
func (c *Client) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) {
func (c *Client) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -517,7 +502,7 @@ 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) (*commonpb.Status, error) {
func (c *Client) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -528,7 +513,7 @@ func (c *Client) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSe
})
}
func (c *Client) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
func (c *Client) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -539,7 +524,7 @@ func (c *Client) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsI
})
}
func (c *Client) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest) (*commonpb.Status, error) {
func (c *Client) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -551,82 +536,82 @@ func (c *Client) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmen
}
// BroadcastAlteredCollection is the DataCoord client side code for BroadcastAlteredCollection call.
func (c *Client) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
func (c *Client) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*commonpb.Status, error) {
return client.BroadcastAlteredCollection(ctx, req)
})
}
func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*milvuspb.CheckHealthResponse, error) {
return client.CheckHealth(ctx, req)
})
}
func (c *Client) GcConfirm(ctx context.Context, req *datapb.GcConfirmRequest) (*datapb.GcConfirmResponse, error) {
func (c *Client) GcConfirm(ctx context.Context, req *datapb.GcConfirmRequest, opts ...grpc.CallOption) (*datapb.GcConfirmResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*datapb.GcConfirmResponse, error) {
return client.GcConfirm(ctx, req)
})
}
// CreateIndex sends the build index request to IndexCoord.
func (c *Client) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
func (c *Client) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*commonpb.Status, error) {
return client.CreateIndex(ctx, req)
})
}
// GetIndexState gets the index states from IndexCoord.
func (c *Client) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
func (c *Client) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStateResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*indexpb.GetIndexStateResponse, error) {
return client.GetIndexState(ctx, req)
})
}
// GetSegmentIndexState gets the index states from IndexCoord.
func (c *Client) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
func (c *Client) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetSegmentIndexStateResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*indexpb.GetSegmentIndexStateResponse, error) {
return client.GetSegmentIndexState(ctx, req)
})
}
// GetIndexInfos gets the index file paths from IndexCoord.
func (c *Client) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
func (c *Client) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest, opts ...grpc.CallOption) (*indexpb.GetIndexInfoResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*indexpb.GetIndexInfoResponse, error) {
return client.GetIndexInfos(ctx, req)
})
}
// DescribeIndex describe the index info of the collection.
func (c *Client) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
func (c *Client) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest, opts ...grpc.CallOption) (*indexpb.DescribeIndexResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*indexpb.DescribeIndexResponse, error) {
return client.DescribeIndex(ctx, req)
})
}
// GetIndexStatistics get the statistics of the index.
func (c *Client) GetIndexStatistics(ctx context.Context, req *indexpb.GetIndexStatisticsRequest) (*indexpb.GetIndexStatisticsResponse, error) {
func (c *Client) GetIndexStatistics(ctx context.Context, req *indexpb.GetIndexStatisticsRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStatisticsResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*indexpb.GetIndexStatisticsResponse, error) {
return client.GetIndexStatistics(ctx, req)
})
}
// GetIndexBuildProgress describe the progress of the index.
func (c *Client) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
func (c *Client) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*indexpb.GetIndexBuildProgressResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*indexpb.GetIndexBuildProgressResponse, error) {
return client.GetIndexBuildProgress(ctx, req)
})
}
// DropIndex sends the drop index request to IndexCoord.
func (c *Client) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
func (c *Client) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*commonpb.Status, error) {
return client.DropIndex(ctx, req)
})
}
func (c *Client) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
func (c *Client) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*commonpb.Status, error) {
return client.ReportDataNodeTtMsgs(ctx, req)
})

View File

@ -70,15 +70,6 @@ func Test_NewClient(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, client)
err = client.Init()
assert.NoError(t, err)
err = client.Start()
assert.NoError(t, err)
err = client.Register()
assert.NoError(t, err)
checkFunc := func(retNotNil bool) {
retCheck := func(notNil bool, ret any, err error) {
if notNil {
@ -90,13 +81,13 @@ func Test_NewClient(t *testing.T) {
}
}
r1, err := client.GetComponentStates(ctx)
r1, err := client.GetComponentStates(ctx, nil)
retCheck(retNotNil, r1, err)
r2, err := client.GetTimeTickChannel(ctx)
r2, err := client.GetTimeTickChannel(ctx, nil)
retCheck(retNotNil, r2, err)
r3, err := client.GetStatisticsChannel(ctx)
r3, err := client.GetStatisticsChannel(ctx, nil)
retCheck(retNotNil, r3, err)
r4, err := client.Flush(ctx, nil)
@ -120,7 +111,7 @@ func Test_NewClient(t *testing.T) {
r10, err := client.GetPartitionStatistics(ctx, nil)
retCheck(retNotNil, r10, err)
r11, err := client.GetSegmentInfoChannel(ctx)
r11, err := client.GetSegmentInfoChannel(ctx, nil)
retCheck(retNotNil, r11, err)
// r12, err := client.SaveBinlogPaths(ctx, nil)
@ -259,6 +250,6 @@ func Test_NewClient(t *testing.T) {
assert.NotNil(t, ret)
assert.NoError(t, err)
err = client.Stop()
err = client.Close()
assert.NoError(t, err)
}

View File

@ -259,17 +259,17 @@ func (s *Server) Run() error {
// GetComponentStates gets states of datacoord and datanodes
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.dataCoord.GetComponentStates(ctx)
return s.dataCoord.GetComponentStates(ctx, req)
}
// GetTimeTickChannel gets timetick channel
func (s *Server) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return s.dataCoord.GetTimeTickChannel(ctx)
return s.dataCoord.GetTimeTickChannel(ctx, req)
}
// GetStatisticsChannel gets statistics channel
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.dataCoord.GetStatisticsChannel(ctx)
return s.dataCoord.GetStatisticsChannel(ctx, req)
}
// GetSegmentInfo gets segment information according to segment id
@ -309,7 +309,7 @@ func (s *Server) GetPartitionStatistics(ctx context.Context, req *datapb.GetPart
// GetSegmentInfoChannel gets channel to which datacoord sends segment information
func (s *Server) GetSegmentInfoChannel(ctx context.Context, req *datapb.GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) {
return s.dataCoord.GetSegmentInfoChannel(ctx)
return s.dataCoord.GetSegmentInfoChannel(ctx, req)
}
// SaveBinlogPaths implement DataCoordServer, saves segment, collection binlog according to datanode request

View File

@ -111,24 +111,24 @@ func (m *MockDataCoord) SetEtcdClient(etcdClient *clientv3.Client) {
func (m *MockDataCoord) SetTiKVClient(client *txnkv.Client) {
}
func (m *MockDataCoord) SetRootCoord(rootCoord types.RootCoord) {
func (m *MockDataCoord) SetRootCoordClient(rootCoord types.RootCoordClient) {
}
func (m *MockDataCoord) SetDataNodeCreator(func(context.Context, string, int64) (types.DataNode, error)) {
func (m *MockDataCoord) SetDataNodeCreator(func(context.Context, string, int64) (types.DataNodeClient, error)) {
}
func (m *MockDataCoord) SetIndexNodeCreator(func(context.Context, string, int64) (types.IndexNode, error)) {
func (m *MockDataCoord) SetIndexNodeCreator(func(context.Context, string, int64) (types.IndexNodeClient, error)) {
}
func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *MockDataCoord) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return m.states, m.err
}
func (m *MockDataCoord) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *MockDataCoord) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return m.strResp, m.err
}
func (m *MockDataCoord) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *MockDataCoord) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return m.strResp, m.err
}
@ -160,7 +160,7 @@ func (m *MockDataCoord) GetPartitionStatistics(ctx context.Context, req *datapb.
return m.partStatResp, m.err
}
func (m *MockDataCoord) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *MockDataCoord) GetSegmentInfoChannel(ctx context.Context, req *datapb.GetSegmentInfoChannelRequest) (*milvuspb.StringResponse, error) {
return m.strResp, m.err
}

View File

@ -60,28 +60,12 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (*Client, error)
return client, nil
}
// Init initializes the client.
func (c *Client) Init() error {
return nil
}
// Start starts the client.
// Currently, it does nothing.
func (c *Client) Start() error {
return nil
}
// Stop stops the client.
// Close stops the client.
// Currently, it closes the grpc connection with the DataNode.
func (c *Client) Stop() error {
func (c *Client) Close() error {
return c.grpcClient.Close()
}
// Register does nothing.
func (c *Client) Register() error {
return nil
}
func (c *Client) newGrpcClient(cc *grpc.ClientConn) datapb.DataNodeClient {
return datapb.NewDataNodeClient(cc)
}
@ -104,7 +88,7 @@ func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(grpcClient da
}
// GetComponentStates returns ComponentStates
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *Client) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*milvuspb.ComponentStates, error) {
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
@ -112,7 +96,7 @@ func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentSta
// GetStatisticsChannel return the statistics channel in string
// Statistics channel contains statistics infos of query nodes, such as segment infos, memory infos
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*milvuspb.StringResponse, error) {
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
@ -120,7 +104,7 @@ func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResp
// Deprecated
// WatchDmChannels create consumers on dmChannels to reveive Incremental data
func (c *Client) WatchDmChannels(ctx context.Context, req *datapb.WatchDmChannelsRequest) (*commonpb.Status, error) {
func (c *Client) WatchDmChannels(ctx context.Context, req *datapb.WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -142,7 +126,7 @@ func (c *Client) WatchDmChannels(ctx context.Context, req *datapb.WatchDmChannel
// Return Success code in status and trigers background flush:
//
// Log an info log if a segment is under flushing
func (c *Client) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
func (c *Client) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -153,7 +137,7 @@ func (c *Client) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsReq
}
// ShowConfigurations gets specified configurations para of DataNode
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -164,7 +148,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
}
// GetMetrics returns metrics
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -175,13 +159,13 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
}
// Compaction return compaction by given plan
func (c *Client) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
func (c *Client) Compaction(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*commonpb.Status, error) {
return client.Compaction(ctx, req)
})
}
func (c *Client) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
func (c *Client) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest, opts ...grpc.CallOption) (*datapb.CompactionStateResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -192,7 +176,7 @@ 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) (*commonpb.Status, error) {
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -202,7 +186,7 @@ func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*co
})
}
func (c *Client) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
func (c *Client) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest, opts ...grpc.CallOption) (*datapb.ResendSegmentStatsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -213,7 +197,7 @@ 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) (*datapb.AddImportSegmentResponse, error) {
func (c *Client) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest, opts ...grpc.CallOption) (*datapb.AddImportSegmentResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -224,26 +208,26 @@ func (c *Client) AddImportSegment(ctx context.Context, req *datapb.AddImportSegm
}
// SyncSegments is the DataNode client side code for SyncSegments call.
func (c *Client) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
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) {
return client.SyncSegments(ctx, req)
})
}
// FlushChannels notifies DataNode to sync all the segments belongs to the target channels.
func (c *Client) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
func (c *Client) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*commonpb.Status, error) {
return client.FlushChannels(ctx, req)
})
}
func (c *Client) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest) (*commonpb.Status, error) {
func (c *Client) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*commonpb.Status, error) {
return client.NotifyChannelOperation(ctx, req)
})
}
func (c *Client) CheckChannelOperationProgress(ctx context.Context, req *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error) {
func (c *Client) CheckChannelOperationProgress(ctx context.Context, req *datapb.ChannelWatchInfo, opts ...grpc.CallOption) (*datapb.ChannelOperationProgressResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*datapb.ChannelOperationProgressResponse, error) {
return client.CheckChannelOperationProgress(ctx, req)
})

View File

@ -40,15 +40,6 @@ func Test_NewClient(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, client)
err = client.Init()
assert.NoError(t, err)
err = client.Start()
assert.NoError(t, err)
err = client.Register()
assert.NoError(t, err)
checkFunc := func(retNotNil bool) {
retCheck := func(notNil bool, ret interface{}, err error) {
if notNil {
@ -60,10 +51,10 @@ func Test_NewClient(t *testing.T) {
}
}
r1, err := client.GetComponentStates(ctx)
r1, err := client.GetComponentStates(ctx, nil)
retCheck(retNotNil, r1, err)
r2, err := client.GetStatisticsChannel(ctx)
r2, err := client.GetStatisticsChannel(ctx, nil)
retCheck(retNotNil, r2, err)
r3, err := client.WatchDmChannels(ctx, nil)
@ -134,6 +125,6 @@ func Test_NewClient(t *testing.T) {
checkFunc(true)
err = client.Stop()
err = client.Close()
assert.NoError(t, err)
}

View File

@ -68,8 +68,8 @@ type Server struct {
rootCoord types.RootCoord
dataCoord types.DataCoord
newRootCoordClient func(string, *clientv3.Client) (types.RootCoord, error)
newDataCoordClient func(string, *clientv3.Client) (types.DataCoord, error)
newRootCoordClient func(string, *clientv3.Client) (types.RootCoordClient, error)
newDataCoordClient func(string, *clientv3.Client) (types.DataCoordClient, error)
}
// NewServer new DataNode grpc server
@ -80,10 +80,10 @@ func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error)
cancel: cancel,
factory: factory,
grpcErrChan: make(chan error),
newRootCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.RootCoord, error) {
newRootCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.RootCoordClient, error) {
return rcc.NewClient(ctx1, etcdMetaRoot, client)
},
newDataCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.DataCoord, error) {
newDataCoordClient: func(etcdMetaRoot string, client *clientv3.Client) (types.DataCoordClient, error) {
return dcc.NewClient(ctx1, etcdMetaRoot, client)
},
}
@ -173,12 +173,12 @@ func (s *Server) SetEtcdClient(client *clientv3.Client) {
s.datanode.SetEtcdClient(client)
}
func (s *Server) SetRootCoordInterface(ms types.RootCoord) error {
return s.datanode.SetRootCoord(ms)
func (s *Server) SetRootCoordInterface(ms types.RootCoordClient) error {
return s.datanode.SetRootCoordClient(ms)
}
func (s *Server) SetDataCoordInterface(ds types.DataCoord) error {
return s.datanode.SetDataCoord(ds)
func (s *Server) SetDataCoordInterface(ds types.DataCoordClient) error {
return s.datanode.SetDataCoordClient(ds)
}
// Run initializes and starts Datanode's grpc service.
@ -271,14 +271,7 @@ func (s *Server) init() error {
log.Error("failed to create new RootCoord client", zap.Error(err))
panic(err)
}
if err = rootCoordClient.Init(); err != nil {
log.Error("failed to init RootCoord client", zap.Error(err))
panic(err)
}
if err = rootCoordClient.Start(); err != nil {
log.Error("failed to start RootCoord client", zap.Error(err))
panic(err)
}
if err = componentutil.WaitForComponentHealthy(ctx, rootCoordClient, "RootCoord", 1000000, time.Millisecond*200); err != nil {
log.Error("failed to wait for RootCoord client to be ready", zap.Error(err))
panic(err)
@ -297,14 +290,7 @@ func (s *Server) init() error {
log.Error("failed to create new DataCoord client", zap.Error(err))
panic(err)
}
if err = dataCoordClient.Init(); err != nil {
log.Error("failed to init DataCoord client", zap.Error(err))
panic(err)
}
if err = dataCoordClient.Start(); err != nil {
log.Error("failed to start DataCoord client", zap.Error(err))
panic(err)
}
if err = componentutil.WaitForComponentInitOrHealthy(ctx, dataCoordClient, "DataCoord", 1000000, time.Millisecond*200); err != nil {
log.Error("failed to wait for DataCoord client to be ready", zap.Error(err))
panic(err)
@ -340,12 +326,12 @@ func (s *Server) start() error {
// GetComponentStates gets the component states of Datanode
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.datanode.GetComponentStates(ctx)
return s.datanode.GetComponentStates(ctx, req)
}
// GetStatisticsChannel gets the statistics channel of Datanode.
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.datanode.GetStatisticsChannel(ctx)
return s.datanode.GetStatisticsChannel(ctx, req)
}
// Deprecated

View File

@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -90,19 +91,19 @@ func (m *MockDataNode) GetAddress() string {
return ""
}
func (m *MockDataNode) SetRootCoord(rc types.RootCoord) error {
func (m *MockDataNode) SetRootCoordClient(rc types.RootCoordClient) error {
return m.err
}
func (m *MockDataNode) SetDataCoord(dc types.DataCoord) error {
func (m *MockDataNode) SetDataCoordClient(dc types.DataCoordClient) error {
return m.err
}
func (m *MockDataNode) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *MockDataNode) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return m.states, m.err
}
func (m *MockDataNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *MockDataNode) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return m.strResp, m.err
}
@ -163,18 +164,10 @@ func (m *MockDataNode) CheckChannelOperationProgress(ctx context.Context, req *d
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type mockDataCoord struct {
types.DataCoord
types.DataCoordClient
}
func (m *mockDataCoord) Init() error {
return nil
}
func (m *mockDataCoord) Start() error {
return nil
}
func (m *mockDataCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *mockDataCoord) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Healthy,
@ -194,18 +187,10 @@ func (m *mockDataCoord) Stop() error {
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type mockRootCoord struct {
types.RootCoord
types.RootCoordClient
}
func (m *mockRootCoord) Init() error {
return nil
}
func (m *mockRootCoord) Start() error {
return nil
}
func (m *mockRootCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *mockRootCoord) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Healthy,
@ -231,11 +216,11 @@ func Test_NewServer(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, server)
server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) {
server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoordClient, error) {
return &mockRootCoord{}, nil
}
server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) {
server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoordClient, error) {
return &mockDataCoord{}, nil
}
@ -370,11 +355,11 @@ func Test_Run(t *testing.T) {
regErr: errors.New("error"),
}
server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoord, error) {
server.newRootCoordClient = func(string, *clientv3.Client) (types.RootCoordClient, error) {
return &mockRootCoord{}, nil
}
server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoord, error) {
server.newDataCoordClient = func(string, *clientv3.Client) (types.DataCoordClient, error) {
return &mockDataCoord{}, nil
}

View File

@ -62,26 +62,11 @@ func NewClient(ctx context.Context, addr string, nodeID int64, encryption bool)
return client, nil
}
// Init initializes IndexNode's grpc client.
func (c *Client) Init() error {
return nil
}
// Start starts IndexNode's client service. But it does nothing here.
func (c *Client) Start() error {
return nil
}
// Stop stops IndexNode's grpc client.
func (c *Client) Stop() error {
// Close stops IndexNode's grpc client.
func (c *Client) Close() error {
return c.grpcClient.Close()
}
// Register dummy
func (c *Client) Register() error {
return nil
}
func (c *Client) newGrpcClient(cc *grpc.ClientConn) indexpb.IndexNodeClient {
return indexpb.NewIndexNodeClient(cc)
}
@ -104,48 +89,48 @@ func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(indexClient i
}
// GetComponentStates gets the component states of IndexNode.
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *Client) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*milvuspb.ComponentStates, error) {
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
}
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*milvuspb.StringResponse, error) {
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
}
// CreateJob sends the build index request to IndexNode.
func (c *Client) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest) (*commonpb.Status, error) {
func (c *Client) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*commonpb.Status, error) {
return client.CreateJob(ctx, req)
})
}
// QueryJobs query the task info of the index task.
func (c *Client) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
func (c *Client) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest, opts ...grpc.CallOption) (*indexpb.QueryJobsResponse, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*indexpb.QueryJobsResponse, error) {
return client.QueryJobs(ctx, req)
})
}
// DropJobs query the task info of the index task.
func (c *Client) DropJobs(ctx context.Context, req *indexpb.DropJobsRequest) (*commonpb.Status, error) {
func (c *Client) DropJobs(ctx context.Context, req *indexpb.DropJobsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*commonpb.Status, error) {
return client.DropJobs(ctx, req)
})
}
// GetJobStats query the task info of the index task.
func (c *Client) GetJobStats(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
func (c *Client) GetJobStats(ctx context.Context, req *indexpb.GetJobStatsRequest, opts ...grpc.CallOption) (*indexpb.GetJobStatsResponse, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*indexpb.GetJobStatsResponse, error) {
return client.GetJobStats(ctx, req)
})
}
// ShowConfigurations gets specified configurations para of IndexNode
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -156,7 +141,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
}
// GetMetrics gets the metrics info of IndexNode.
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),

View File

@ -24,14 +24,9 @@ import (
"github.com/stretchr/testify/assert"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
grpcindexnode "github.com/milvus-io/milvus/internal/distributed/indexnode"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/mock"
"github.com/milvus-io/milvus/pkg/util/etcd"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
@ -47,15 +42,6 @@ func Test_NewClient(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, client)
err = client.Init()
assert.NoError(t, err)
err = client.Start()
assert.NoError(t, err)
err = client.Register()
assert.NoError(t, err)
checkFunc := func(retNotNil bool) {
retCheck := func(notNil bool, ret interface{}, err error) {
if notNil {
@ -67,10 +53,10 @@ func Test_NewClient(t *testing.T) {
}
}
r1, err := client.GetComponentStates(ctx)
r1, err := client.GetComponentStates(ctx, nil)
retCheck(retNotNil, r1, err)
r3, err := client.GetStatisticsChannel(ctx)
r3, err := client.GetStatisticsChannel(ctx, nil)
retCheck(retNotNil, r3, err)
r4, err := client.CreateJob(ctx, nil)
@ -117,57 +103,23 @@ func Test_NewClient(t *testing.T) {
client.grpcClient.SetNewGrpcClientFunc(newFunc3)
checkFunc(true)
err = client.Stop()
err = client.Close()
assert.NoError(t, err)
}
func TestIndexNodeClient(t *testing.T) {
paramtable.Init()
ctx := context.Background()
factory := dependency.NewDefaultFactory(true)
ins, err := grpcindexnode.NewServer(ctx, factory)
assert.NoError(t, err)
assert.NotNil(t, ins)
inm := indexnode.NewIndexNodeMock()
etcdCli, err := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
Params.EtcdCfg.Endpoints.GetAsStrings(),
Params.EtcdCfg.EtcdTLSCert.GetValue(),
Params.EtcdCfg.EtcdTLSKey.GetValue(),
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
assert.NoError(t, err)
inm.SetEtcdClient(etcdCli)
err = ins.SetClient(inm)
assert.NoError(t, err)
err = ins.Run()
assert.NoError(t, err)
inc, err := NewClient(ctx, "localhost:21121", paramtable.GetNodeID(), false)
assert.NoError(t, err)
inc := &mock.GrpcIndexNodeClient{Err: nil}
assert.NotNil(t, inc)
err = inc.Init()
assert.NoError(t, err)
err = inc.Start()
assert.NoError(t, err)
ctx := context.TODO()
t.Run("GetComponentStates", func(t *testing.T) {
states, err := inc.GetComponentStates(ctx)
_, err := inc.GetComponentStates(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, commonpb.StateCode_Healthy, states.State.StateCode)
assert.Equal(t, commonpb.ErrorCode_Success, states.GetStatus().GetErrorCode())
})
t.Run("GetStatisticsChannel", func(t *testing.T) {
resp, err := inc.GetStatisticsChannel(ctx)
_, err := inc.GetStatisticsChannel(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
})
t.Run("CreatJob", func(t *testing.T) {
@ -175,52 +127,43 @@ func TestIndexNodeClient(t *testing.T) {
ClusterID: "0",
BuildID: 0,
}
resp, err := inc.CreateJob(ctx, req)
_, err := inc.CreateJob(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.ErrorCode)
})
t.Run("QueryJob", func(t *testing.T) {
req := &indexpb.QueryJobsRequest{}
resp, err := inc.QueryJobs(ctx, req)
_, err := inc.QueryJobs(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
})
t.Run("DropJob", func(t *testing.T) {
req := &indexpb.DropJobsRequest{}
resp, err := inc.DropJobs(ctx, req)
_, err := inc.DropJobs(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.ErrorCode)
})
t.Run("ShowConfigurations", func(t *testing.T) {
req := &internalpb.ShowConfigurationsRequest{
Pattern: "",
}
resp, err := inc.ShowConfigurations(ctx, req)
_, err := inc.ShowConfigurations(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
})
t.Run("GetMetrics", func(t *testing.T) {
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
assert.NoError(t, err)
resp, err := inc.GetMetrics(ctx, req)
_, err = inc.GetMetrics(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
})
t.Run("GetJobStats", func(t *testing.T) {
req := &indexpb.GetJobStatsRequest{}
resp, err := inc.GetJobStats(ctx, req)
_, err := inc.GetJobStats(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
})
err = ins.Stop()
assert.NoError(t, err)
err = inc.Stop()
err := inc.Close()
assert.NoError(t, err)
}

View File

@ -225,9 +225,9 @@ func (s *Server) Stop() error {
return nil
}
// SetClient sets the IndexNode's instance.
func (s *Server) SetClient(indexNodeClient types.IndexNodeComponent) error {
s.indexnode = indexNodeClient
// setServer sets the IndexNode's instance.
func (s *Server) setServer(indexNode types.IndexNodeComponent) error {
s.indexnode = indexNode
return nil
}
@ -238,12 +238,12 @@ func (s *Server) SetEtcdClient(etcdCli *clientv3.Client) {
// GetComponentStates gets the component states of IndexNode.
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.indexnode.GetComponentStates(ctx)
return s.indexnode.GetComponentStates(ctx, req)
}
// GetStatisticsChannel gets the statistics channel of IndexNode.
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.indexnode.GetStatisticsChannel(ctx)
return s.indexnode.GetStatisticsChannel(ctx, req)
}
// CreateJob sends the create index request to IndexNode.

View File

@ -28,13 +28,10 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/pkg/util/etcd"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
var ParamsGlobal paramtable.ComponentParam
func TestIndexNodeServer(t *testing.T) {
paramtable.Init()
ctx := context.Background()
@ -44,18 +41,7 @@ func TestIndexNodeServer(t *testing.T) {
assert.NotNil(t, server)
inm := indexnode.NewIndexNodeMock()
ParamsGlobal.Init(paramtable.NewBaseTable(paramtable.SkipRemote(true)))
etcdCli, err := etcd.GetEtcdClient(
ParamsGlobal.EtcdCfg.UseEmbedEtcd.GetAsBool(),
ParamsGlobal.EtcdCfg.EtcdUseSSL.GetAsBool(),
ParamsGlobal.EtcdCfg.Endpoints.GetAsStrings(),
ParamsGlobal.EtcdCfg.EtcdTLSCert.GetValue(),
ParamsGlobal.EtcdCfg.EtcdTLSKey.GetValue(),
ParamsGlobal.EtcdCfg.EtcdTLSCACert.GetValue(),
ParamsGlobal.EtcdCfg.EtcdTLSMinVersion.GetValue())
assert.NoError(t, err)
inm.SetEtcdClient(etcdCli)
err = server.SetClient(inm)
err = server.setServer(inm)
assert.NoError(t, err)
err = server.Run()

View File

@ -59,11 +59,6 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (*Client, error)
return client, nil
}
// Init initializes Proxy's grpc client.
func (c *Client) Init() error {
return nil
}
func (c *Client) newGrpcClient(cc *grpc.ClientConn) proxypb.ProxyClient {
return proxypb.NewProxyClient(cc)
}
@ -72,21 +67,11 @@ func (c *Client) getAddr() (string, error) {
return c.addr, nil
}
// Start dummy
func (c *Client) Start() error {
return nil
}
// Stop stops the client, closes the connection
func (c *Client) Stop() error {
// Close stops the client, closes the connection
func (c *Client) Close() error {
return c.grpcClient.Close()
}
// Register dummy
func (c *Client) Register() error {
return nil
}
func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(proxyClient proxypb.ProxyClient) (*T, error)) (*T, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client proxypb.ProxyClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
@ -101,21 +86,21 @@ func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(proxyClient p
}
// GetComponentStates get the component state.
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *Client) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return wrapGrpcCall(ctx, c, func(client proxypb.ProxyClient) (*milvuspb.ComponentStates, error) {
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
}
// GetStatisticsChannel return the statistics channel in string
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client proxypb.ProxyClient) (*milvuspb.StringResponse, error) {
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
}
// InvalidateCollectionMetaCache invalidate collection meta cache
func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, req *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, req *proxypb.InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -126,7 +111,7 @@ func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, req *proxypb
})
}
func (c *Client) InvalidateCredentialCache(ctx context.Context, req *proxypb.InvalidateCredCacheRequest) (*commonpb.Status, error) {
func (c *Client) InvalidateCredentialCache(ctx context.Context, req *proxypb.InvalidateCredCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -137,7 +122,7 @@ func (c *Client) InvalidateCredentialCache(ctx context.Context, req *proxypb.Inv
})
}
func (c *Client) UpdateCredentialCache(ctx context.Context, req *proxypb.UpdateCredCacheRequest) (*commonpb.Status, error) {
func (c *Client) UpdateCredentialCache(ctx context.Context, req *proxypb.UpdateCredCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -148,7 +133,7 @@ func (c *Client) UpdateCredentialCache(ctx context.Context, req *proxypb.UpdateC
})
}
func (c *Client) RefreshPolicyInfoCache(ctx context.Context, req *proxypb.RefreshPolicyInfoCacheRequest) (*commonpb.Status, error) {
func (c *Client) RefreshPolicyInfoCache(ctx context.Context, req *proxypb.RefreshPolicyInfoCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -161,7 +146,7 @@ func (c *Client) RefreshPolicyInfoCache(ctx context.Context, req *proxypb.Refres
// GetProxyMetrics gets the metrics of proxy, it's an internal interface which is different from GetMetrics interface,
// because it only obtains the metrics of Proxy, not including the topological metrics of Query cluster and Data cluster.
func (c *Client) GetProxyMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Client) GetProxyMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -173,7 +158,7 @@ func (c *Client) GetProxyMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
}
// SetRates notifies Proxy to limit rates of requests.
func (c *Client) SetRates(ctx context.Context, req *proxypb.SetRatesRequest) (*commonpb.Status, error) {
func (c *Client) SetRates(ctx context.Context, req *proxypb.SetRatesRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -184,7 +169,7 @@ func (c *Client) SetRates(ctx context.Context, req *proxypb.SetRatesRequest) (*c
})
}
func (c *Client) ListClientInfos(ctx context.Context, req *proxypb.ListClientInfosRequest) (*proxypb.ListClientInfosResponse, error) {
func (c *Client) ListClientInfos(ctx context.Context, req *proxypb.ListClientInfosRequest, opts ...grpc.CallOption) (*proxypb.ListClientInfosResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -194,3 +179,9 @@ func (c *Client) ListClientInfos(ctx context.Context, req *proxypb.ListClientInf
return client.ListClientInfos(ctx, req)
})
}
func (c *Client) GetDdChannel(ctx context.Context, req *internalpb.GetDdChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client proxypb.ProxyClient) (*milvuspb.StringResponse, error) {
return client.GetDdChannel(ctx, req)
})
}

View File

@ -42,12 +42,6 @@ func Test_NewClient(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, client)
err = client.Start()
assert.NoError(t, err)
err = client.Register()
assert.NoError(t, err)
checkFunc := func(retNotNil bool) {
retCheck := func(notNil bool, ret interface{}, err error) {
if notNil {
@ -59,10 +53,10 @@ func Test_NewClient(t *testing.T) {
}
}
r1, err := client.GetComponentStates(ctx)
r1, err := client.GetComponentStates(ctx, nil)
retCheck(retNotNil, r1, err)
r2, err := client.GetStatisticsChannel(ctx)
r2, err := client.GetStatisticsChannel(ctx, nil)
retCheck(retNotNil, r2, err)
r3, err := client.InvalidateCollectionMetaCache(ctx, nil)
@ -123,10 +117,10 @@ func Test_NewClient(t *testing.T) {
assert.Error(t, err)
}
r1Timeout, err := client.GetComponentStates(shortCtx)
r1Timeout, err := client.GetComponentStates(shortCtx, nil)
retCheck(r1Timeout, err)
r2Timeout, err := client.GetStatisticsChannel(shortCtx)
r2Timeout, err := client.GetStatisticsChannel(shortCtx, nil)
retCheck(r2Timeout, err)
r3Timeout, err := client.InvalidateCollectionMetaCache(shortCtx, nil)
@ -144,6 +138,6 @@ func Test_NewClient(t *testing.T) {
}
// cleanup
err = client.Stop()
err = client.Close()
assert.NoError(t, err)
}

View File

@ -95,9 +95,9 @@ type Server struct {
serverID atomic.Int64
etcdCli *clientv3.Client
rootCoordClient types.RootCoord
dataCoordClient types.DataCoord
queryCoordClient types.QueryCoord
rootCoordClient types.RootCoordClient
dataCoordClient types.DataCoordClient
queryCoordClient types.QueryCoordClient
}
// NewServer create a Proxy server.
@ -516,13 +516,6 @@ func (s *Server) init() error {
log.Debug("create RootCoord client for Proxy done")
}
log.Debug("init RootCoord client for Proxy")
if err := s.rootCoordClient.Init(); err != nil {
log.Warn("failed to init RootCoord client for Proxy", zap.Error(err))
return err
}
log.Debug("init RootCoord client for Proxy done")
log.Debug("Proxy wait for RootCoord to be healthy")
if err := componentutil.WaitForComponentHealthy(s.ctx, s.rootCoordClient, "RootCoord", 1000000, time.Millisecond*200); err != nil {
log.Warn("Proxy failed to wait for RootCoord to be healthy", zap.Error(err))
@ -545,13 +538,6 @@ func (s *Server) init() error {
log.Debug("create DataCoord client for Proxy done")
}
log.Debug("init DataCoord client for Proxy")
if err := s.dataCoordClient.Init(); err != nil {
log.Warn("failed to init DataCoord client for Proxy", zap.Error(err))
return err
}
log.Debug("init DataCoord client for Proxy done")
log.Debug("Proxy wait for DataCoord to be healthy")
if err := componentutil.WaitForComponentHealthy(s.ctx, s.dataCoordClient, "DataCoord", 1000000, time.Millisecond*200); err != nil {
log.Warn("Proxy failed to wait for DataCoord to be healthy", zap.Error(err))
@ -574,13 +560,6 @@ func (s *Server) init() error {
log.Debug("create QueryCoord client for Proxy done")
}
log.Debug("init QueryCoord client for Proxy")
if err := s.queryCoordClient.Init(); err != nil {
log.Warn("failed to init QueryCoord client for Proxy", zap.Error(err))
return err
}
log.Debug("init QueryCoord client for Proxy done")
log.Debug("Proxy wait for QueryCoord to be healthy")
if err := componentutil.WaitForComponentHealthy(s.ctx, s.queryCoordClient, "QueryCoord", 1000000, time.Millisecond*200); err != nil {
log.Warn("Proxy failed to wait for QueryCoord to be healthy", zap.Error(err))
@ -678,12 +657,12 @@ func (s *Server) Stop() error {
// GetComponentStates get the component states
func (s *Server) GetComponentStates(ctx context.Context, request *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.proxy.GetComponentStates(ctx)
return s.proxy.GetComponentStates(ctx, request)
}
// GetStatisticsChannel get the statistics channel
func (s *Server) GetStatisticsChannel(ctx context.Context, request *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.proxy.GetStatisticsChannel(ctx)
return s.proxy.GetStatisticsChannel(ctx, request)
}
// InvalidateCollectionMetaCache notifies Proxy to clear all the meta cache of specific collection.
@ -951,7 +930,7 @@ func (s *Server) Check(ctx context.Context, req *grpc_health_v1.HealthCheckReque
ret := &grpc_health_v1.HealthCheckResponse{
Status: grpc_health_v1.HealthCheckResponse_NOT_SERVING,
}
state, err := s.proxy.GetComponentStates(ctx)
state, err := s.proxy.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
if err != nil {
return ret, err
}
@ -970,7 +949,7 @@ func (s *Server) Watch(req *grpc_health_v1.HealthCheckRequest, server grpc_healt
ret := &grpc_health_v1.HealthCheckResponse{
Status: grpc_health_v1.HealthCheckResponse_NOT_SERVING,
}
state, err := s.proxy.GetComponentStates(s.ctx)
state, err := s.proxy.GetComponentStates(s.ctx, nil)
if err != nil {
return server.Send(ret)
}

View File

@ -41,11 +41,8 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/federpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/proxy"
"github.com/milvus-io/milvus/internal/types"
milvusmock "github.com/milvus-io/milvus/internal/util/mock"
@ -74,7 +71,7 @@ func (m *MockBase) On(methodName string, arguments ...interface{}) *mock.Call {
return m.Mock.On(methodName, arguments...)
}
func (m *MockBase) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *MockBase) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
if m.isMockGetComponentStatesOn {
ret1 := &milvuspb.ComponentStates{}
var ret2 error
@ -95,412 +92,11 @@ func (m *MockBase) GetComponentStates(ctx context.Context) (*milvuspb.ComponentS
}, nil
}
func (m *MockBase) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *MockBase) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return nil, nil
}
func (m *MockBase) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return nil, nil
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockRootCoord struct {
MockBase
initErr error
startErr error
regErr error
stopErr error
}
func (m *MockRootCoord) Init() error {
return m.initErr
}
func (m *MockRootCoord) Start() error {
return m.startErr
}
func (m *MockRootCoord) Stop() error {
return m.stopErr
}
func (m *MockRootCoord) Register() error {
return m.regErr
}
func (m *MockRootCoord) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) ListDatabases(ctx context.Context, in *milvuspb.ListDatabasesRequest) (*milvuspb.ListDatabasesResponse, error) {
return nil, nil
}
func (m *MockRootCoord) CreateCollection(ctx context.Context, req *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) DropCollection(ctx context.Context, req *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) HasCollection(ctx context.Context, req *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
return nil, nil
}
func (m *MockRootCoord) DescribeCollection(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
return nil, nil
}
func (m *MockRootCoord) DescribeCollectionInternal(ctx context.Context, req *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
return nil, nil
}
func (m *MockRootCoord) ShowCollections(ctx context.Context, req *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
return nil, nil
}
func (m *MockRootCoord) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) CreatePartition(ctx context.Context, req *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) DropPartition(ctx context.Context, req *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) HasPartition(ctx context.Context, req *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
return nil, nil
}
func (m *MockRootCoord) ShowPartitions(ctx context.Context, req *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
return nil, nil
}
func (m *MockRootCoord) ShowPartitionsInternal(ctx context.Context, req *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
return nil, nil
}
func (m *MockRootCoord) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) AlterAlias(ctx context.Context, req *milvuspb.AlterAliasRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) AllocTimestamp(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
return nil, nil
}
func (m *MockRootCoord) AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
return nil, nil
}
func (m *MockRootCoord) UpdateChannelTimeTick(ctx context.Context, req *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) ShowSegments(ctx context.Context, req *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) {
return nil, nil
}
func (m *MockRootCoord) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return nil, nil
}
func (m *MockRootCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return nil, nil
}
func (m *MockRootCoord) Import(ctx context.Context, req *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error) {
return nil, nil
}
func (m *MockRootCoord) GetImportState(ctx context.Context, req *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error) {
return nil, nil
}
func (m *MockRootCoord) ListImportTasks(ctx context.Context, in *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error) {
return nil, nil
}
func (m *MockRootCoord) ReportImport(ctx context.Context, req *rootcoordpb.ImportResult) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) CreateCredential(ctx context.Context, req *internalpb.CredentialInfo) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) UpdateCredential(ctx context.Context, req *internalpb.CredentialInfo) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCredentialRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) {
return nil, nil
}
func (m *MockRootCoord) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest) (*rootcoordpb.GetCredentialResponse, error) {
return nil, nil
}
func (m *MockRootCoord) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) DropRole(ctx context.Context, in *milvuspb.DropRoleRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) OperateUserRole(ctx context.Context, in *milvuspb.OperateUserRoleRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) SelectRole(ctx context.Context, in *milvuspb.SelectRoleRequest) (*milvuspb.SelectRoleResponse, error) {
return nil, nil
}
func (m *MockRootCoord) SelectUser(ctx context.Context, in *milvuspb.SelectUserRequest) (*milvuspb.SelectUserResponse, error) {
return nil, nil
}
func (m *MockRootCoord) OperatePrivilege(ctx context.Context, in *milvuspb.OperatePrivilegeRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockRootCoord) SelectGrant(ctx context.Context, in *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) {
return nil, nil
}
func (m *MockRootCoord) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) {
return nil, nil
}
func (m *MockRootCoord) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
return &milvuspb.CheckHealthResponse{
IsHealthy: true,
}, nil
}
func (m *MockRootCoord) RenameCollection(ctx context.Context, req *milvuspb.RenameCollectionRequest) (*commonpb.Status, error) {
return nil, nil
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockDataCoord struct {
MockBase
err error
initErr error
startErr error
stopErr error
regErr error
}
func (m *MockDataCoord) Init() error {
return m.initErr
}
func (m *MockDataCoord) Start() error {
return m.startErr
}
func (m *MockDataCoord) Stop() error {
return m.stopErr
}
func (m *MockDataCoord) Register() error {
return m.regErr
}
func (m *MockDataCoord) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return nil, nil
}
func (m *MockDataCoord) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
return nil, nil
}
func (m *MockDataCoord) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return nil, nil
}
func (m *MockDataCoord) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryInfoRequestV2) (*datapb.GetRecoveryInfoResponseV2, error) {
return nil, nil
}
func (m *MockDataCoord) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegmentsByStatesRequest) (*datapb.GetSegmentsByStatesResponse, error) {
return nil, nil
}
func (m *MockDataCoord) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error) {
return nil, nil
}
func (m *MockDataCoord) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest) (*datapb.WatchChannelsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error) {
return nil, nil
}
func (m *MockDataCoord) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) {
return &datapb.DropVirtualChannelResponse{}, nil
}
func (m *MockDataCoord) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest) (*datapb.SetSegmentStateResponse, error) {
return &datapb.SetSegmentStateResponse{}, nil
}
func (m *MockDataCoord) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
return nil, nil
}
func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GcConfirm(ctx context.Context, req *datapb.GcConfirmRequest) (*datapb.GcConfirmResponse, error) {
return nil, nil
}
func (m *MockDataCoord) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return nil, nil
}
func (m *MockDataCoord) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
return nil, nil
}
func (m *MockDataCoord) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetIndexStatistics(ctx context.Context, req *indexpb.GetIndexStatisticsRequest) (*indexpb.GetIndexStatisticsResponse, error) {
return nil, nil
}
func (m *MockDataCoord) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
return nil, nil
}
func (m *MockDataCoord) ReportDataNodeTtMsgs(ctx context.Context, req *datapb.ReportDataNodeTtMsgsRequest) (*commonpb.Status, error) {
func (m *MockBase) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return nil, nil
}
@ -515,6 +111,39 @@ type MockProxy struct {
isMockOn bool
}
func (m *MockProxy) DescribeAlias(ctx context.Context, request *milvuspb.DescribeAliasRequest) (*milvuspb.DescribeAliasResponse, error) {
return nil, nil
}
func (m *MockProxy) ListAliases(ctx context.Context, request *milvuspb.ListAliasesRequest) (*milvuspb.ListAliasesResponse, error) {
return nil, nil
}
func (m *MockProxy) GetVersion(ctx context.Context, request *milvuspb.GetVersionRequest) (*milvuspb.GetVersionResponse, error) {
return nil, nil
}
func (m *MockProxy) ListIndexedSegment(ctx context.Context, request *federpb.ListIndexedSegmentRequest) (*federpb.ListIndexedSegmentResponse, error) {
return nil, nil
}
func (m *MockProxy) DescribeSegmentIndexData(ctx context.Context, request *federpb.DescribeSegmentIndexDataRequest) (*federpb.DescribeSegmentIndexDataResponse, error) {
return nil, nil
}
func (m *MockProxy) SetRootCoordClient(rootCoord types.RootCoordClient) {
}
func (m *MockProxy) SetDataCoordClient(dataCoord types.DataCoordClient) {
}
func (m *MockProxy) SetQueryCoordClient(queryCoord types.QueryCoordClient) {
}
func (m *MockProxy) SetQueryNodeCreator(f func(ctx context.Context, addr string, nodeID int64) (types.QueryNodeClient, error)) {
panic("error")
}
func (m *MockProxy) Init() error {
return m.initErr
}
@ -727,18 +356,6 @@ func (m *MockProxy) GetProxyMetrics(ctx context.Context, request *milvuspb.GetMe
return nil, nil
}
func (m *MockProxy) SetRootCoordClient(rootCoord types.RootCoord) {
}
func (m *MockProxy) SetDataCoordClient(dataCoord types.DataCoord) {
}
func (m *MockProxy) SetQueryCoordClient(queryCoord types.QueryCoord) {
}
func (m *MockProxy) SetQueryNodeCreator(func(ctx context.Context, addr string, nodeID int64) (types.QueryNode, error)) {
}
func (m *MockProxy) GetRateLimiter() (types.Limiter, error) {
return nil, nil
}
@ -1532,13 +1149,12 @@ func getServer(t *testing.T) *Server {
assert.NoError(t, err)
server.proxy = &MockProxy{}
server.rootCoordClient = &MockRootCoord{}
server.dataCoordClient = &MockDataCoord{}
server.rootCoordClient = &milvusmock.GrpcRootCoordClient{}
server.dataCoordClient = &milvusmock.GrpcDataCoordClient{}
mockQC := &mocks.MockQueryCoord{}
mockQC := &mocks.MockQueryCoordClient{}
server.queryCoordClient = mockQC
mockQC.EXPECT().Init().Return(nil)
mockQC.EXPECT().GetComponentStates(mock.Anything).Return(&milvuspb.ComponentStates{
mockQC.EXPECT().GetComponentStates(mock.Anything, mock.Anything, mock.Anything).Return(&milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: int64(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
Role: "MockQueryCoord",

View File

@ -66,11 +66,6 @@ func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (
return client, nil
}
// Init initializes QueryCoord's grpc client.
func (c *Client) Init() error {
return nil
}
func (c *Client) getQueryCoordAddr() (string, error) {
key := c.grpcClient.GetRole()
msess, _, err := c.sess.GetSessions(key)
@ -91,21 +86,11 @@ func (c *Client) newGrpcClient(cc *grpc.ClientConn) querypb.QueryCoordClient {
return querypb.NewQueryCoordClient(cc)
}
// Start starts QueryCoordinator's client service. But it does nothing here.
func (c *Client) Start() error {
return nil
}
// Stop stops QueryCoordinator's grpc client server.
func (c *Client) Stop() error {
// Close stops QueryCoordinator's grpc client server.
func (c *Client) Close() error {
return c.grpcClient.Close()
}
// Register dummy
func (c *Client) Register() error {
return nil
}
func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(grpcClient querypb.QueryCoordClient) (*T, error)) (*T, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
@ -120,28 +105,28 @@ func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(grpcClient qu
}
// GetComponentStates gets the component states of QueryCoord.
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *Client) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryCoordClient) (*milvuspb.ComponentStates, error) {
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
}
// GetTimeTickChannel gets the time tick channel of QueryCoord.
func (c *Client) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryCoordClient) (*milvuspb.StringResponse, error) {
return client.GetTimeTickChannel(ctx, &internalpb.GetTimeTickChannelRequest{})
})
}
// GetStatisticsChannel gets the statistics channel of QueryCoord.
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryCoordClient) (*milvuspb.StringResponse, error) {
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
}
// ShowCollections shows the collections in the QueryCoord.
func (c *Client) ShowCollections(ctx context.Context, req *querypb.ShowCollectionsRequest) (*querypb.ShowCollectionsResponse, error) {
func (c *Client) ShowCollections(ctx context.Context, req *querypb.ShowCollectionsRequest, opts ...grpc.CallOption) (*querypb.ShowCollectionsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -153,7 +138,7 @@ func (c *Client) ShowCollections(ctx context.Context, req *querypb.ShowCollectio
}
// LoadCollection loads the data of the specified collections in the QueryCoord.
func (c *Client) LoadCollection(ctx context.Context, req *querypb.LoadCollectionRequest) (*commonpb.Status, error) {
func (c *Client) LoadCollection(ctx context.Context, req *querypb.LoadCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -165,7 +150,7 @@ func (c *Client) LoadCollection(ctx context.Context, req *querypb.LoadCollection
}
// ReleaseCollection release the data of the specified collections in the QueryCoord.
func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -177,7 +162,7 @@ func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseColl
}
// ShowPartitions shows the partitions in the QueryCoord.
func (c *Client) ShowPartitions(ctx context.Context, req *querypb.ShowPartitionsRequest) (*querypb.ShowPartitionsResponse, error) {
func (c *Client) ShowPartitions(ctx context.Context, req *querypb.ShowPartitionsRequest, opts ...grpc.CallOption) (*querypb.ShowPartitionsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -189,7 +174,7 @@ func (c *Client) ShowPartitions(ctx context.Context, req *querypb.ShowPartitions
}
// LoadPartitions loads the data of the specified partitions in the QueryCoord.
func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest) (*commonpb.Status, error) {
func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -201,7 +186,7 @@ func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitions
}
// ReleasePartitions release the data of the specified partitions in the QueryCoord.
func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest) (*commonpb.Status, error) {
func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -213,7 +198,7 @@ func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePart
}
// SyncNewCreatedPartition notifies QueryCoord to sync new created partition if collection is loaded.
func (c *Client) SyncNewCreatedPartition(ctx context.Context, req *querypb.SyncNewCreatedPartitionRequest) (*commonpb.Status, error) {
func (c *Client) SyncNewCreatedPartition(ctx context.Context, req *querypb.SyncNewCreatedPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -225,7 +210,7 @@ func (c *Client) SyncNewCreatedPartition(ctx context.Context, req *querypb.SyncN
}
// GetPartitionStates gets the states of the specified partition.
func (c *Client) GetPartitionStates(ctx context.Context, req *querypb.GetPartitionStatesRequest) (*querypb.GetPartitionStatesResponse, error) {
func (c *Client) GetPartitionStates(ctx context.Context, req *querypb.GetPartitionStatesRequest, opts ...grpc.CallOption) (*querypb.GetPartitionStatesResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -237,7 +222,7 @@ func (c *Client) GetPartitionStates(ctx context.Context, req *querypb.GetPartiti
}
// GetSegmentInfo gets the information of the specified segment from QueryCoord.
func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfoRequest) (*querypb.GetSegmentInfoResponse, error) {
func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfoRequest, opts ...grpc.CallOption) (*querypb.GetSegmentInfoResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -249,7 +234,7 @@ func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfo
}
// LoadBalance migrate the sealed segments on the source node to the dst nodes.
func (c *Client) LoadBalance(ctx context.Context, req *querypb.LoadBalanceRequest) (*commonpb.Status, error) {
func (c *Client) LoadBalance(ctx context.Context, req *querypb.LoadBalanceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -261,7 +246,7 @@ func (c *Client) LoadBalance(ctx context.Context, req *querypb.LoadBalanceReques
}
// ShowConfigurations gets specified configurations para of QueryCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -273,7 +258,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
}
// GetMetrics gets the metrics information of QueryCoord.
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -285,7 +270,7 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
}
// GetReplicas gets the replicas of a certain collection.
func (c *Client) GetReplicas(ctx context.Context, req *milvuspb.GetReplicasRequest) (*milvuspb.GetReplicasResponse, error) {
func (c *Client) GetReplicas(ctx context.Context, req *milvuspb.GetReplicasRequest, opts ...grpc.CallOption) (*milvuspb.GetReplicasResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -297,7 +282,7 @@ func (c *Client) GetReplicas(ctx context.Context, req *milvuspb.GetReplicasReque
}
// GetShardLeaders gets the shard leaders of a certain collection.
func (c *Client) GetShardLeaders(ctx context.Context, req *querypb.GetShardLeadersRequest) (*querypb.GetShardLeadersResponse, error) {
func (c *Client) GetShardLeaders(ctx context.Context, req *querypb.GetShardLeadersRequest, opts ...grpc.CallOption) (*querypb.GetShardLeadersResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -308,13 +293,13 @@ func (c *Client) GetShardLeaders(ctx context.Context, req *querypb.GetShardLeade
})
}
func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryCoordClient) (*milvuspb.CheckHealthResponse, error) {
return client.CheckHealth(ctx, req)
})
}
func (c *Client) CreateResourceGroup(ctx context.Context, req *milvuspb.CreateResourceGroupRequest) (*commonpb.Status, error) {
func (c *Client) CreateResourceGroup(ctx context.Context, req *milvuspb.CreateResourceGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -325,7 +310,7 @@ func (c *Client) CreateResourceGroup(ctx context.Context, req *milvuspb.CreateRe
})
}
func (c *Client) DropResourceGroup(ctx context.Context, req *milvuspb.DropResourceGroupRequest) (*commonpb.Status, error) {
func (c *Client) DropResourceGroup(ctx context.Context, req *milvuspb.DropResourceGroupRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -336,7 +321,7 @@ func (c *Client) DropResourceGroup(ctx context.Context, req *milvuspb.DropResour
})
}
func (c *Client) DescribeResourceGroup(ctx context.Context, req *querypb.DescribeResourceGroupRequest) (*querypb.DescribeResourceGroupResponse, error) {
func (c *Client) DescribeResourceGroup(ctx context.Context, req *querypb.DescribeResourceGroupRequest, opts ...grpc.CallOption) (*querypb.DescribeResourceGroupResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -347,7 +332,7 @@ func (c *Client) DescribeResourceGroup(ctx context.Context, req *querypb.Describ
})
}
func (c *Client) TransferNode(ctx context.Context, req *milvuspb.TransferNodeRequest) (*commonpb.Status, error) {
func (c *Client) TransferNode(ctx context.Context, req *milvuspb.TransferNodeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -358,7 +343,7 @@ func (c *Client) TransferNode(ctx context.Context, req *milvuspb.TransferNodeReq
})
}
func (c *Client) TransferReplica(ctx context.Context, req *querypb.TransferReplicaRequest) (*commonpb.Status, error) {
func (c *Client) TransferReplica(ctx context.Context, req *querypb.TransferReplicaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -369,7 +354,7 @@ func (c *Client) TransferReplica(ctx context.Context, req *querypb.TransferRepli
})
}
func (c *Client) ListResourceGroups(ctx context.Context, req *milvuspb.ListResourceGroupsRequest) (*milvuspb.ListResourceGroupsResponse, error) {
func (c *Client) ListResourceGroups(ctx context.Context, req *milvuspb.ListResourceGroupsRequest, opts ...grpc.CallOption) (*milvuspb.ListResourceGroupsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),

View File

@ -71,15 +71,6 @@ func Test_NewClient(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, client)
err = client.Init()
assert.NoError(t, err)
err = client.Start()
assert.NoError(t, err)
err = client.Register()
assert.NoError(t, err)
checkFunc := func(retNotNil bool) {
retCheck := func(notNil bool, ret any, err error) {
if notNil {
@ -91,13 +82,13 @@ func Test_NewClient(t *testing.T) {
}
}
r1, err := client.GetComponentStates(ctx)
r1, err := client.GetComponentStates(ctx, nil)
retCheck(retNotNil, r1, err)
r2, err := client.GetTimeTickChannel(ctx)
r2, err := client.GetTimeTickChannel(ctx, nil)
retCheck(retNotNil, r2, err)
r3, err := client.GetStatisticsChannel(ctx)
r3, err := client.GetStatisticsChannel(ctx, nil)
retCheck(retNotNil, r3, err)
r4, err := client.ShowCollections(ctx, nil)
@ -204,6 +195,6 @@ func Test_NewClient(t *testing.T) {
checkFunc(true)
err = client.Stop()
err = client.Close()
assert.NoError(t, err)
}

View File

@ -71,8 +71,8 @@ type Server struct {
etcdCli *clientv3.Client
tikvCli *txnkv.Client
dataCoord types.DataCoord
rootCoord types.RootCoord
dataCoord types.DataCoordClient
rootCoord types.RootCoordClient
}
// NewServer create a new QueryCoord grpc server.
@ -159,15 +159,6 @@ func (s *Server) init() error {
}
}
if err = s.rootCoord.Init(); err != nil {
log.Error("QueryCoord RootCoordClient Init failed", zap.Error(err))
panic(err)
}
if err = s.rootCoord.Start(); err != nil {
log.Error("QueryCoord RootCoordClient Start failed", zap.Error(err))
panic(err)
}
// wait for master init or healthy
log.Debug("QueryCoord try to wait for RootCoord ready")
err = componentutil.WaitForComponentHealthy(s.loopCtx, s.rootCoord, "RootCoord", 1000000, time.Millisecond*200)
@ -190,14 +181,6 @@ func (s *Server) init() error {
}
}
if err = s.dataCoord.Init(); err != nil {
log.Error("QueryCoord DataCoordClient Init failed", zap.Error(err))
panic(err)
}
if err = s.dataCoord.Start(); err != nil {
log.Error("QueryCoord DataCoordClient Start failed", zap.Error(err))
panic(err)
}
log.Debug("QueryCoord try to wait for DataCoord ready")
err = componentutil.WaitForComponentHealthy(s.loopCtx, s.dataCoord, "DataCoord", 1000000, time.Millisecond*200)
if err != nil {
@ -309,30 +292,30 @@ func (s *Server) SetTiKVClient(client *txnkv.Client) {
}
// SetRootCoord sets the RootCoord's client for QueryCoord component.
func (s *Server) SetRootCoord(m types.RootCoord) error {
s.queryCoord.SetRootCoord(m)
func (s *Server) SetRootCoord(m types.RootCoordClient) error {
s.queryCoord.SetRootCoordClient(m)
return nil
}
// SetDataCoord sets the DataCoord's client for QueryCoord component.
func (s *Server) SetDataCoord(d types.DataCoord) error {
s.queryCoord.SetDataCoord(d)
func (s *Server) SetDataCoord(d types.DataCoordClient) error {
s.queryCoord.SetDataCoordClient(d)
return nil
}
// GetComponentStates gets the component states of QueryCoord.
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.queryCoord.GetComponentStates(ctx)
return s.queryCoord.GetComponentStates(ctx, req)
}
// GetTimeTickChannel gets the time tick channel of QueryCoord.
func (s *Server) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return s.queryCoord.GetTimeTickChannel(ctx)
return s.queryCoord.GetTimeTickChannel(ctx, req)
}
// GetStatisticsChannel gets the statistics channel of QueryCoord.
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.queryCoord.GetStatisticsChannel(ctx)
return s.queryCoord.GetStatisticsChannel(ctx, req)
}
// ShowCollections shows the collections in the QueryCoord.

View File

@ -25,6 +25,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/tikv/client-go/v2/txnkv"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -39,31 +40,16 @@ import (
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockRootCoord struct {
types.RootCoord
initErr error
startErr error
regErr error
types.RootCoordClient
stopErr error
stateErr commonpb.ErrorCode
}
func (m *MockRootCoord) Init() error {
return m.initErr
}
func (m *MockRootCoord) Start() error {
return m.startErr
}
func (m *MockRootCoord) Stop() error {
func (m *MockRootCoord) Close() error {
return m.stopErr
}
func (m *MockRootCoord) Register() error {
return m.regErr
}
func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *MockRootCoord) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opt ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: &commonpb.Status{ErrorCode: m.stateErr},
@ -72,31 +58,16 @@ func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*milvuspb.Compo
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockDataCoord struct {
types.DataCoord
initErr error
startErr error
types.DataCoordClient
stopErr error
regErr error
stateErr commonpb.ErrorCode
}
func (m *MockDataCoord) Init() error {
return m.initErr
}
func (m *MockDataCoord) Start() error {
return m.startErr
}
func (m *MockDataCoord) Stop() error {
func (m *MockDataCoord) Close() error {
return m.stopErr
}
func (m *MockDataCoord) Register() error {
return m.regErr
}
func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *MockDataCoord) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: &commonpb.Status{ErrorCode: m.stateErr},
@ -146,7 +117,7 @@ func Test_NewServer(t *testing.T) {
})
t.Run("GetComponentStates", func(t *testing.T) {
mqc.EXPECT().GetComponentStates(mock.Anything).Return(&milvuspb.ComponentStates{
mqc.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(&milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: 0,
Role: "MockQueryCoord",
@ -163,7 +134,7 @@ func Test_NewServer(t *testing.T) {
t.Run("GetStatisticsChannel", func(t *testing.T) {
req := &internalpb.GetStatisticsChannelRequest{}
mqc.EXPECT().GetStatisticsChannel(mock.Anything).Return(
mqc.EXPECT().GetStatisticsChannel(mock.Anything, mock.Anything).Return(
&milvuspb.StringResponse{
Status: successStatus,
}, nil,
@ -175,7 +146,7 @@ func Test_NewServer(t *testing.T) {
t.Run("GetTimeTickChannel", func(t *testing.T) {
req := &internalpb.GetTimeTickChannelRequest{}
mqc.EXPECT().GetTimeTickChannel(mock.Anything).Return(
mqc.EXPECT().GetTimeTickChannel(mock.Anything, mock.Anything).Return(
&milvuspb.StringResponse{
Status: successStatus,
}, nil,
@ -239,7 +210,7 @@ func Test_NewServer(t *testing.T) {
})
t.Run("GetTimeTickChannel", func(t *testing.T) {
mqc.EXPECT().GetTimeTickChannel(mock.Anything).Return(&milvuspb.StringResponse{Status: successStatus}, nil)
mqc.EXPECT().GetTimeTickChannel(mock.Anything, mock.Anything).Return(&milvuspb.StringResponse{Status: successStatus}, nil)
resp, err := server.GetTimeTickChannel(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, resp)
@ -355,118 +326,17 @@ func TestServer_Run1(t *testing.T) {
}
}
func TestServer_Run2(t *testing.T) {
parameters := []string{"tikv", "etcd"}
for _, v := range parameters {
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
return tikv.SetupLocalTxn(), nil
}
defer func() {
getTiKVClient = tikv.GetTiKVClient
}()
server, err := NewServer(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, server)
server.queryCoord = getQueryCoord()
server.rootCoord = &MockRootCoord{
initErr: errors.New("error"),
}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.NoError(t, err)
}
}
func getQueryCoord() *mocks.MockQueryCoord {
mqc := &mocks.MockQueryCoord{}
mqc.EXPECT().Init().Return(nil)
mqc.EXPECT().SetEtcdClient(mock.Anything)
mqc.EXPECT().SetTiKVClient(mock.Anything)
mqc.EXPECT().SetAddress(mock.Anything)
mqc.EXPECT().SetRootCoord(mock.Anything).Return(nil)
mqc.EXPECT().SetDataCoord(mock.Anything).Return(nil)
mqc.EXPECT().SetRootCoordClient(mock.Anything).Return(nil)
mqc.EXPECT().SetDataCoordClient(mock.Anything).Return(nil)
mqc.EXPECT().UpdateStateCode(mock.Anything)
mqc.EXPECT().Register().Return(nil)
mqc.EXPECT().Start().Return(nil)
mqc.EXPECT().Stop().Return(nil)
return mqc
}
func TestServer_Run3(t *testing.T) {
parameters := []string{"tikv", "etcd"}
for _, v := range parameters {
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
return tikv.SetupLocalTxn(), nil
}
defer func() {
getTiKVClient = tikv.GetTiKVClient
}()
server, err := NewServer(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, server)
server.queryCoord = getQueryCoord()
server.rootCoord = &MockRootCoord{
startErr: errors.New("error"),
}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.NoError(t, err)
}
}
func TestServer_Run4(t *testing.T) {
parameters := []string{"tikv", "etcd"}
for _, v := range parameters {
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
return tikv.SetupLocalTxn(), nil
}
defer func() {
getTiKVClient = tikv.GetTiKVClient
}()
server, err := NewServer(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, server)
server.queryCoord = getQueryCoord()
server.rootCoord = &MockRootCoord{}
server.dataCoord = &MockDataCoord{
initErr: errors.New("error"),
}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.NoError(t, err)
}
}
func TestServer_Run5(t *testing.T) {
parameters := []string{"tikv", "etcd"}
for _, v := range parameters {
paramtable.Get().Save(paramtable.Get().MetaStoreCfg.MetaStoreType.Key, v)
ctx := context.Background()
getTiKVClient = func(cfg *paramtable.TiKVConfig) (*txnkv.Client, error) {
return tikv.SetupLocalTxn(), nil
}
defer func() {
getTiKVClient = tikv.GetTiKVClient
}()
server, err := NewServer(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, server)
server.queryCoord = getQueryCoord()
server.rootCoord = &MockRootCoord{}
server.dataCoord = &MockDataCoord{
startErr: errors.New("error"),
}
assert.Panics(t, func() { server.Run() })
err = server.Stop()
assert.NoError(t, err)
}
}

View File

@ -27,7 +27,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/grpcclient"
"github.com/milvus-io/milvus/internal/util/streamrpc"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/paramtable"
@ -59,26 +58,11 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (*Client, error)
return client, nil
}
// Init initializes QueryNode's grpc client.
func (c *Client) Init() error {
return nil
}
// Start starts QueryNode's client service. But it does nothing here.
func (c *Client) Start() error {
return nil
}
// Stop stops QueryNode's grpc client server.
func (c *Client) Stop() error {
// Close close QueryNode's grpc client
func (c *Client) Close() error {
return c.grpcClient.Close()
}
// Register dummy
func (c *Client) Register() error {
return nil
}
func (c *Client) newGrpcClient(cc *grpc.ClientConn) querypb.QueryNodeClient {
return querypb.NewQueryNodeClient(cc)
}
@ -101,28 +85,28 @@ func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(grpcClient qu
}
// GetComponentStates gets the component states of QueryNode.
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *Client) GetComponentStates(ctx context.Context, _ *milvuspb.GetComponentStatesRequest, _ ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*milvuspb.ComponentStates, error) {
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
}
// GetTimeTickChannel gets the time tick channel of QueryNode.
func (c *Client) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest, _ ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*milvuspb.StringResponse, error) {
return client.GetTimeTickChannel(ctx, &internalpb.GetTimeTickChannelRequest{})
})
}
// GetStatisticsChannel gets the statistics channel of QueryNode.
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, _ ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*milvuspb.StringResponse, error) {
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
}
// WatchDmChannels watches the channels about data manipulation.
func (c *Client) WatchDmChannels(ctx context.Context, req *querypb.WatchDmChannelsRequest) (*commonpb.Status, error) {
func (c *Client) WatchDmChannels(ctx context.Context, req *querypb.WatchDmChannelsRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -133,7 +117,7 @@ func (c *Client) WatchDmChannels(ctx context.Context, req *querypb.WatchDmChanne
}
// UnsubDmChannel unsubscribes the channels about data manipulation.
func (c *Client) UnsubDmChannel(ctx context.Context, req *querypb.UnsubDmChannelRequest) (*commonpb.Status, error) {
func (c *Client) UnsubDmChannel(ctx context.Context, req *querypb.UnsubDmChannelRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -144,7 +128,7 @@ func (c *Client) UnsubDmChannel(ctx context.Context, req *querypb.UnsubDmChannel
}
// LoadSegments loads the segments to search.
func (c *Client) LoadSegments(ctx context.Context, req *querypb.LoadSegmentsRequest) (*commonpb.Status, error) {
func (c *Client) LoadSegments(ctx context.Context, req *querypb.LoadSegmentsRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -155,7 +139,7 @@ func (c *Client) LoadSegments(ctx context.Context, req *querypb.LoadSegmentsRequ
}
// ReleaseCollection releases the data of the specified collection in QueryNode.
func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -166,7 +150,7 @@ func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseColl
}
// LoadPartitions updates partitions meta info in QueryNode.
func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest) (*commonpb.Status, error) {
func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -177,7 +161,7 @@ func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitions
}
// ReleasePartitions releases the data of the specified partitions in QueryNode.
func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest) (*commonpb.Status, error) {
func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -188,7 +172,7 @@ func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePart
}
// ReleaseSegments releases the data of the specified segments in QueryNode.
func (c *Client) ReleaseSegments(ctx context.Context, req *querypb.ReleaseSegmentsRequest) (*commonpb.Status, error) {
func (c *Client) ReleaseSegments(ctx context.Context, req *querypb.ReleaseSegmentsRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -199,61 +183,61 @@ func (c *Client) ReleaseSegments(ctx context.Context, req *querypb.ReleaseSegmen
}
// Search performs replica search tasks in QueryNode.
func (c *Client) Search(ctx context.Context, req *querypb.SearchRequest) (*internalpb.SearchResults, error) {
func (c *Client) Search(ctx context.Context, req *querypb.SearchRequest, _ ...grpc.CallOption) (*internalpb.SearchResults, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*internalpb.SearchResults, error) {
return client.Search(ctx, req)
})
}
func (c *Client) SearchSegments(ctx context.Context, req *querypb.SearchRequest) (*internalpb.SearchResults, error) {
func (c *Client) SearchSegments(ctx context.Context, req *querypb.SearchRequest, _ ...grpc.CallOption) (*internalpb.SearchResults, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*internalpb.SearchResults, error) {
return client.SearchSegments(ctx, req)
})
}
// Query performs replica query tasks in QueryNode.
func (c *Client) Query(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error) {
func (c *Client) Query(ctx context.Context, req *querypb.QueryRequest, _ ...grpc.CallOption) (*internalpb.RetrieveResults, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*internalpb.RetrieveResults, error) {
return client.Query(ctx, req)
})
}
func (c *Client) QueryStream(ctx context.Context, req *querypb.QueryRequest, streamer streamrpc.QueryStreamer) error {
_, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryNodeClient) (any, error) {
func (c *Client) QueryStream(ctx context.Context, req *querypb.QueryRequest, _ ...grpc.CallOption) (querypb.QueryNode_QueryStreamClient, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryNodeClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
cli, err := client.QueryStream(ctx, req)
if err == nil {
streamer.SetClient(cli)
}
return nil, err
return client.QueryStream(ctx, req)
})
return err
if err != nil || ret == nil {
return nil, err
}
return ret.(querypb.QueryNode_QueryStreamClient), nil
}
func (c *Client) QuerySegments(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error) {
func (c *Client) QuerySegments(ctx context.Context, req *querypb.QueryRequest, _ ...grpc.CallOption) (*internalpb.RetrieveResults, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*internalpb.RetrieveResults, error) {
return client.QuerySegments(ctx, req)
})
}
func (c *Client) QueryStreamSegments(ctx context.Context, req *querypb.QueryRequest, streamer streamrpc.QueryStreamer) error {
_, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryNodeClient) (any, error) {
func (c *Client) QueryStreamSegments(ctx context.Context, req *querypb.QueryRequest, _ ...grpc.CallOption) (querypb.QueryNode_QueryStreamSegmentsClient, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryNodeClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
cli, err := client.QueryStreamSegments(ctx, req)
if err == nil {
streamer.SetClient(cli)
}
return nil, err
return client.QueryStreamSegments(ctx, req)
})
return err
if err != nil || ret == nil {
return nil, err
}
return ret.(querypb.QueryNode_QueryStreamSegmentsClient), nil
}
// GetSegmentInfo gets the information of the specified segments in QueryNode.
func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfoRequest) (*querypb.GetSegmentInfoResponse, error) {
func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfoRequest, _ ...grpc.CallOption) (*querypb.GetSegmentInfoResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -264,7 +248,7 @@ func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfo
}
// SyncReplicaSegments syncs replica node segments information to shard leaders.
func (c *Client) SyncReplicaSegments(ctx context.Context, req *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) {
func (c *Client) SyncReplicaSegments(ctx context.Context, req *querypb.SyncReplicaSegmentsRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -275,7 +259,7 @@ func (c *Client) SyncReplicaSegments(ctx context.Context, req *querypb.SyncRepli
}
// ShowConfigurations gets specified configurations para of QueryNode
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, _ ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -286,7 +270,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
}
// GetMetrics gets the metrics information of QueryNode.
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, _ ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -296,13 +280,13 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
})
}
func (c *Client) GetStatistics(ctx context.Context, request *querypb.GetStatisticsRequest) (*internalpb.GetStatisticsResponse, error) {
func (c *Client) GetStatistics(ctx context.Context, request *querypb.GetStatisticsRequest, _ ...grpc.CallOption) (*internalpb.GetStatisticsResponse, error) {
return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*internalpb.GetStatisticsResponse, error) {
return client.GetStatistics(ctx, request)
})
}
func (c *Client) GetDataDistribution(ctx context.Context, req *querypb.GetDataDistributionRequest) (*querypb.GetDataDistributionResponse, error) {
func (c *Client) GetDataDistribution(ctx context.Context, req *querypb.GetDataDistributionRequest, _ ...grpc.CallOption) (*querypb.GetDataDistributionResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -312,7 +296,7 @@ func (c *Client) GetDataDistribution(ctx context.Context, req *querypb.GetDataDi
})
}
func (c *Client) SyncDistribution(ctx context.Context, req *querypb.SyncDistributionRequest) (*commonpb.Status, error) {
func (c *Client) SyncDistribution(ctx context.Context, req *querypb.SyncDistributionRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -323,7 +307,7 @@ func (c *Client) SyncDistribution(ctx context.Context, req *querypb.SyncDistribu
}
// Delete is used to forward delete message between delegator and workers.
func (c *Client) Delete(ctx context.Context, req *querypb.DeleteRequest) (*commonpb.Status, error) {
func (c *Client) Delete(ctx context.Context, req *querypb.DeleteRequest, _ ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),

View File

@ -26,7 +26,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/mock"
"github.com/milvus-io/milvus/internal/util/streamrpc"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
@ -42,12 +41,6 @@ func Test_NewClient(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, client)
err = client.Start()
assert.NoError(t, err)
err = client.Register()
assert.NoError(t, err)
ctx, cancel := context.WithCancel(ctx)
checkFunc := func(retNotNil bool) {
@ -61,13 +54,13 @@ func Test_NewClient(t *testing.T) {
}
}
r1, err := client.GetComponentStates(ctx)
r1, err := client.GetComponentStates(ctx, nil)
retCheck(retNotNil, r1, err)
r2, err := client.GetTimeTickChannel(ctx)
r2, err := client.GetTimeTickChannel(ctx, nil)
retCheck(retNotNil, r2, err)
r3, err := client.GetStatisticsChannel(ctx)
r3, err := client.GetStatisticsChannel(ctx, nil)
retCheck(retNotNil, r3, err)
r6, err := client.WatchDmChannels(ctx, nil)
@ -116,13 +109,8 @@ func Test_NewClient(t *testing.T) {
retCheck(retNotNil, r20, err)
// stream rpc
streamer1 := streamrpc.NewGrpcQueryStreamer()
err = client.QueryStream(ctx, nil, streamer1)
retCheck(retNotNil, streamer1.AsClient(), err)
streamer2 := streamrpc.NewGrpcQueryStreamer()
err = client.QueryStreamSegments(ctx, nil, streamer2)
retCheck(retNotNil, streamer2.AsClient(), err)
client, err := client.QueryStream(ctx, nil)
retCheck(retNotNil, client, err)
}
client.grpcClient = &mock.GRPCClientBase[querypb.QueryNodeClient]{
@ -167,6 +155,6 @@ func Test_NewClient(t *testing.T) {
cancel() // make context canceled
checkFunc(false)
err = client.Stop()
err = client.Close()
assert.NoError(t, err)
}

View File

@ -39,7 +39,6 @@ import (
qn "github.com/milvus-io/milvus/internal/querynodev2"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/streamrpc"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/tracer"
"github.com/milvus-io/milvus/pkg/util/etcd"
@ -259,18 +258,18 @@ func (s *Server) SetEtcdClient(etcdCli *clientv3.Client) {
// GetTimeTickChannel gets the time tick channel of QueryNode.
func (s *Server) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return s.querynode.GetTimeTickChannel(ctx)
return s.querynode.GetTimeTickChannel(ctx, req)
}
// GetStatisticsChannel gets the statistics channel of QueryNode.
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.querynode.GetStatisticsChannel(ctx)
return s.querynode.GetStatisticsChannel(ctx, req)
}
// GetComponentStates gets the component states of QueryNode.
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
// ignore ctx and in
return s.querynode.GetComponentStates(ctx)
return s.querynode.GetComponentStates(ctx, req)
}
// WatchDmChannels watches the channels about data manipulation.
@ -332,15 +331,11 @@ func (s *Server) Query(ctx context.Context, req *querypb.QueryRequest) (*interna
}
func (s *Server) QueryStream(req *querypb.QueryRequest, srv querypb.QueryNode_QueryStreamServer) error {
streamer := streamrpc.NewGrpcQueryStreamer()
streamer.SetServer(streamrpc.NewConcurrentQueryStreamServer(srv))
return s.querynode.QueryStream(srv.Context(), req, streamer)
return s.querynode.QueryStream(req, srv)
}
func (s *Server) QueryStreamSegments(req *querypb.QueryRequest, srv querypb.QueryNode_QueryStreamSegmentsServer) error {
streamer := streamrpc.NewGrpcQueryStreamer()
streamer.SetServer(streamrpc.NewConcurrentQueryStreamServer(srv))
return s.querynode.QueryStreamSegments(srv.Context(), req, streamer)
return s.querynode.QueryStreamSegments(req, srv)
}
func (s *Server) QuerySegments(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error) {

View File

@ -32,7 +32,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/types"
streamMocks "github.com/milvus-io/milvus/internal/util/streamrpc/mocks"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
@ -64,7 +63,7 @@ func (m *MockRootCoord) Register() error {
func (m *MockRootCoord) SetEtcdClient(client *clientv3.Client) {
}
func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *MockRootCoord) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: &commonpb.Status{ErrorCode: m.stateErr},
@ -100,7 +99,7 @@ func Test_NewServer(t *testing.T) {
})
t.Run("GetComponentStates", func(t *testing.T) {
mockQN.EXPECT().GetComponentStates(mock.Anything).Return(&milvuspb.ComponentStates{
mockQN.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(&milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Healthy,
},
@ -112,7 +111,7 @@ func Test_NewServer(t *testing.T) {
})
t.Run("GetStatisticsChannel", func(t *testing.T) {
mockQN.EXPECT().GetStatisticsChannel(mock.Anything).Return(&milvuspb.StringResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}}, nil)
mockQN.EXPECT().GetStatisticsChannel(mock.Anything, mock.Anything).Return(&milvuspb.StringResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}}, nil)
req := &internalpb.GetStatisticsChannelRequest{}
resp, err := server.GetStatisticsChannel(ctx, req)
assert.NoError(t, err)
@ -120,7 +119,7 @@ func Test_NewServer(t *testing.T) {
})
t.Run("GetTimeTickChannel", func(t *testing.T) {
mockQN.EXPECT().GetTimeTickChannel(mock.Anything).Return(&milvuspb.StringResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}}, nil)
mockQN.EXPECT().GetTimeTickChannel(mock.Anything, mock.Anything).Return(&milvuspb.StringResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}}, nil)
req := &internalpb.GetTimeTickChannelRequest{}
resp, err := server.GetTimeTickChannel(ctx, req)
assert.NoError(t, err)
@ -227,13 +226,9 @@ func Test_NewServer(t *testing.T) {
})
t.Run("QueryStream", func(t *testing.T) {
mockQN.EXPECT().QueryStream(mock.Anything, mock.Anything, mock.Anything).Return(nil)
req := &querypb.QueryRequest{}
streamer := streamMocks.NewMockQueryStreamServer(t)
streamer.EXPECT().Context().Return(ctx)
err := server.QueryStream(req, streamer)
assert.NoError(t, err)
mockQN.EXPECT().QueryStream(mock.Anything, mock.Anything).Return(nil)
ret := server.QueryStream(nil, nil)
assert.Nil(t, ret)
})
t.Run("QuerySegments", func(t *testing.T) {
@ -247,13 +242,9 @@ func Test_NewServer(t *testing.T) {
})
t.Run("QueryStreamSegments", func(t *testing.T) {
mockQN.EXPECT().QueryStreamSegments(mock.Anything, mock.Anything, mock.Anything).Return(nil)
req := &querypb.QueryRequest{}
streamer := streamMocks.NewMockQueryStreamSegmentsServer(t)
streamer.EXPECT().Context().Return(ctx)
err := server.QueryStreamSegments(req, streamer)
assert.NoError(t, err)
mockQN.EXPECT().QueryStreamSegments(mock.Anything, mock.Anything).Return(nil)
ret := server.QueryStreamSegments(nil, nil)
assert.Nil(t, ret)
})
t.Run("SyncReplicaSegments", func(t *testing.T) {

View File

@ -74,10 +74,6 @@ func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (
}
// Init initialize grpc parameters
func (c *Client) Init() error {
return nil
}
func (c *Client) newGrpcClient(cc *grpc.ClientConn) rootcoordpb.RootCoordClient {
return rootcoordpb.NewRootCoordClient(cc)
}
@ -102,21 +98,11 @@ func (c *Client) getRootCoordAddr() (string, error) {
return ms.Address, nil
}
// Start dummy
func (c *Client) Start() error {
return nil
}
// Stop terminate grpc connection
func (c *Client) Stop() error {
// Close terminate grpc connection
func (c *Client) Close() error {
return c.grpcClient.Close()
}
// Register dummy
func (c *Client) Register() error {
return nil
}
func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(grpcClient rootcoordpb.RootCoordClient) (*T, error)) (*T, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
@ -131,28 +117,28 @@ func wrapGrpcCall[T any](ctx context.Context, c *Client, call func(grpcClient ro
}
// GetComponentStates TODO: timeout need to be propagated through ctx
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (c *Client) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*milvuspb.ComponentStates, error) {
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
}
// GetTimeTickChannel get timetick channel name
func (c *Client) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*milvuspb.StringResponse, error) {
return client.GetTimeTickChannel(ctx, &internalpb.GetTimeTickChannelRequest{})
})
}
// GetStatisticsChannel just define a channel, not used currently
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (c *Client) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*milvuspb.StringResponse, error) {
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
}
// CreateCollection create collection
func (c *Client) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
func (c *Client) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -164,7 +150,7 @@ func (c *Client) CreateCollection(ctx context.Context, in *milvuspb.CreateCollec
}
// DropCollection drop collection
func (c *Client) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
func (c *Client) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -176,7 +162,7 @@ func (c *Client) DropCollection(ctx context.Context, in *milvuspb.DropCollection
}
// HasCollection check collection existence
func (c *Client) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
func (c *Client) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -188,7 +174,7 @@ func (c *Client) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRe
}
// DescribeCollection return collection info
func (c *Client) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (c *Client) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -200,7 +186,7 @@ func (c *Client) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCo
}
// describeCollectionInternal return collection info
func (c *Client) describeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (c *Client) describeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -211,7 +197,7 @@ func (c *Client) describeCollectionInternal(ctx context.Context, in *milvuspb.De
})
}
func (c *Client) DescribeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (c *Client) DescribeCollectionInternal(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
resp, err := c.describeCollectionInternal(ctx, in)
status, ok := grpcStatus.FromError(err)
if ok && status.Code() == grpcCodes.Unimplemented {
@ -221,7 +207,7 @@ func (c *Client) DescribeCollectionInternal(ctx context.Context, in *milvuspb.De
}
// ShowCollections list all collection names
func (c *Client) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
func (c *Client) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -232,7 +218,7 @@ func (c *Client) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectio
})
}
func (c *Client) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) {
func (c *Client) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
request = typeutil.Clone(request)
commonpbutil.UpdateMsgBase(
request.GetBase(),
@ -244,7 +230,7 @@ func (c *Client) AlterCollection(ctx context.Context, request *milvuspb.AlterCol
}
// CreatePartition create partition
func (c *Client) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
func (c *Client) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -256,7 +242,7 @@ func (c *Client) CreatePartition(ctx context.Context, in *milvuspb.CreatePartiti
}
// DropPartition drop partition
func (c *Client) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
func (c *Client) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -268,7 +254,7 @@ func (c *Client) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRe
}
// HasPartition check partition existence
func (c *Client) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
func (c *Client) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest, opts ...grpc.CallOption) (*milvuspb.BoolResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -280,7 +266,7 @@ func (c *Client) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequ
}
// ShowPartitions list all partitions in collection
func (c *Client) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
func (c *Client) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -292,7 +278,7 @@ func (c *Client) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitions
}
// showPartitionsInternal list all partitions in collection
func (c *Client) showPartitionsInternal(ctx context.Context, in *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
func (c *Client) showPartitionsInternal(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -303,7 +289,7 @@ func (c *Client) showPartitionsInternal(ctx context.Context, in *milvuspb.ShowPa
})
}
func (c *Client) ShowPartitionsInternal(ctx context.Context, in *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
func (c *Client) ShowPartitionsInternal(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) {
resp, err := c.showPartitionsInternal(ctx, in)
status, ok := grpcStatus.FromError(err)
if ok && status.Code() == grpcCodes.Unimplemented {
@ -313,7 +299,7 @@ func (c *Client) ShowPartitionsInternal(ctx context.Context, in *milvuspb.ShowPa
}
// AllocTimestamp global timestamp allocator
func (c *Client) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
func (c *Client) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimestampRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocTimestampResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -325,7 +311,7 @@ func (c *Client) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimest
}
// AllocID global ID allocator
func (c *Client) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
func (c *Client) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocIDResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -337,7 +323,7 @@ func (c *Client) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest) (*
}
// UpdateChannelTimeTick used to handle ChannelTimeTickMsg
func (c *Client) UpdateChannelTimeTick(ctx context.Context, in *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) {
func (c *Client) UpdateChannelTimeTick(ctx context.Context, in *internalpb.ChannelTimeTickMsg, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -349,7 +335,7 @@ func (c *Client) UpdateChannelTimeTick(ctx context.Context, in *internalpb.Chann
}
// ShowSegments list all segments
func (c *Client) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) {
func (c *Client) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequest, opts ...grpc.CallOption) (*milvuspb.ShowSegmentsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -361,7 +347,7 @@ func (c *Client) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequ
}
// InvalidateCollectionMetaCache notifies RootCoord to release the collection cache in Proxies.
func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -373,7 +359,7 @@ func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.
}
// ShowConfigurations gets specified configurations para of RootCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -385,7 +371,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
}
// GetMetrics get metrics
func (c *Client) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (c *Client) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -397,7 +383,7 @@ func (c *Client) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest)
}
// CreateAlias create collection alias
func (c *Client) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasRequest) (*commonpb.Status, error) {
func (c *Client) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -409,7 +395,7 @@ func (c *Client) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasReque
}
// DropAlias drop collection alias
func (c *Client) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest) (*commonpb.Status, error) {
func (c *Client) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -421,7 +407,7 @@ func (c *Client) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest)
}
// AlterAlias alter collection alias
func (c *Client) AlterAlias(ctx context.Context, req *milvuspb.AlterAliasRequest) (*commonpb.Status, error) {
func (c *Client) AlterAlias(ctx context.Context, req *milvuspb.AlterAliasRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -433,40 +419,40 @@ func (c *Client) AlterAlias(ctx context.Context, req *milvuspb.AlterAliasRequest
}
// 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) (*milvuspb.ImportResponse, error) {
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) (*milvuspb.GetImportStateResponse, error) {
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) (*milvuspb.ListImportTasksResponse, error) {
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) (*commonpb.Status, error) {
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) (*commonpb.Status, error) {
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)
})
}
func (c *Client) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest) (*rootcoordpb.GetCredentialResponse, error) {
func (c *Client) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest, opts ...grpc.CallOption) (*rootcoordpb.GetCredentialResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -477,13 +463,13 @@ func (c *Client) GetCredential(ctx context.Context, req *rootcoordpb.GetCredenti
})
}
func (c *Client) UpdateCredential(ctx context.Context, req *internalpb.CredentialInfo) (*commonpb.Status, error) {
func (c *Client) UpdateCredential(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.UpdateCredential(ctx, req)
})
}
func (c *Client) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCredentialRequest) (*commonpb.Status, error) {
func (c *Client) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCredentialRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -494,7 +480,7 @@ func (c *Client) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCrede
})
}
func (c *Client) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) {
func (c *Client) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest, opts ...grpc.CallOption) (*milvuspb.ListCredUsersResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -505,7 +491,7 @@ func (c *Client) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersR
})
}
func (c *Client) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest) (*commonpb.Status, error) {
func (c *Client) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -516,7 +502,7 @@ func (c *Client) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest
})
}
func (c *Client) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest) (*commonpb.Status, error) {
func (c *Client) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -527,7 +513,7 @@ func (c *Client) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest) (*
})
}
func (c *Client) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserRoleRequest) (*commonpb.Status, error) {
func (c *Client) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserRoleRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -538,7 +524,7 @@ func (c *Client) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserR
})
}
func (c *Client) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest) (*milvuspb.SelectRoleResponse, error) {
func (c *Client) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest, opts ...grpc.CallOption) (*milvuspb.SelectRoleResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -549,7 +535,7 @@ func (c *Client) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest
})
}
func (c *Client) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest) (*milvuspb.SelectUserResponse, error) {
func (c *Client) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest, opts ...grpc.CallOption) (*milvuspb.SelectUserResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -560,7 +546,7 @@ func (c *Client) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest
})
}
func (c *Client) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePrivilegeRequest) (*commonpb.Status, error) {
func (c *Client) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePrivilegeRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -571,7 +557,7 @@ func (c *Client) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePriv
})
}
func (c *Client) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) {
func (c *Client) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantRequest, opts ...grpc.CallOption) (*milvuspb.SelectGrantResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -582,7 +568,7 @@ func (c *Client) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantReque
})
}
func (c *Client) ListPolicy(ctx context.Context, req *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) {
func (c *Client) ListPolicy(ctx context.Context, req *internalpb.ListPolicyRequest, opts ...grpc.CallOption) (*internalpb.ListPolicyResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -593,13 +579,13 @@ func (c *Client) ListPolicy(ctx context.Context, req *internalpb.ListPolicyReque
})
}
func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) {
return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*milvuspb.CheckHealthResponse, error) {
return client.CheckHealth(ctx, req)
})
}
func (c *Client) RenameCollection(ctx context.Context, req *milvuspb.RenameCollectionRequest) (*commonpb.Status, error) {
func (c *Client) RenameCollection(ctx context.Context, req *milvuspb.RenameCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
@ -610,7 +596,7 @@ func (c *Client) RenameCollection(ctx context.Context, req *milvuspb.RenameColle
})
}
func (c *Client) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest) (*commonpb.Status, error) {
func (c *Client) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -629,7 +615,7 @@ func (c *Client) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabase
return ret.(*commonpb.Status), err
}
func (c *Client) DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequest) (*commonpb.Status, error) {
func (c *Client) DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),
@ -648,7 +634,7 @@ func (c *Client) DropDatabase(ctx context.Context, in *milvuspb.DropDatabaseRequ
return ret.(*commonpb.Status), err
}
func (c *Client) ListDatabases(ctx context.Context, in *milvuspb.ListDatabasesRequest) (*milvuspb.ListDatabasesResponse, error) {
func (c *Client) ListDatabases(ctx context.Context, in *milvuspb.ListDatabasesRequest, opts ...grpc.CallOption) (*milvuspb.ListDatabasesResponse, error) {
in = typeutil.Clone(in)
commonpbutil.UpdateMsgBase(
in.GetBase(),

View File

@ -70,15 +70,6 @@ func Test_NewClient(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, client)
err = client.Init()
assert.NoError(t, err)
err = client.Start()
assert.NoError(t, err)
err = client.Register()
assert.NoError(t, err)
checkFunc := func(retNotNil bool) {
retCheck := func(notNil bool, ret interface{}, err error) {
if notNil {
@ -91,15 +82,15 @@ func Test_NewClient(t *testing.T) {
}
{
r, err := client.GetComponentStates(ctx)
r, err := client.GetComponentStates(ctx, nil)
retCheck(retNotNil, r, err)
}
{
r, err := client.GetTimeTickChannel(ctx)
r, err := client.GetTimeTickChannel(ctx, nil)
retCheck(retNotNil, r, err)
}
{
r, err := client.GetStatisticsChannel(ctx)
r, err := client.GetStatisticsChannel(ctx, nil)
retCheck(retNotNil, r, err)
}
{
@ -305,15 +296,15 @@ func Test_NewClient(t *testing.T) {
assert.Error(t, err)
}
{
rTimeout, err := client.GetComponentStates(shortCtx)
rTimeout, err := client.GetComponentStates(shortCtx, nil)
retCheck(rTimeout, err)
}
{
rTimeout, err := client.GetTimeTickChannel(shortCtx)
rTimeout, err := client.GetTimeTickChannel(shortCtx, nil)
retCheck(rTimeout, err)
}
{
rTimeout, err := client.GetStatisticsChannel(shortCtx)
rTimeout, err := client.GetStatisticsChannel(shortCtx, nil)
retCheck(rTimeout, err)
}
{
@ -473,6 +464,6 @@ func Test_NewClient(t *testing.T) {
retCheck(rTimeout, err)
}
// clean up
err = client.Stop()
err = client.Close()
assert.NoError(t, err)
}

View File

@ -68,11 +68,11 @@ type Server struct {
etcdCli *clientv3.Client
tikvCli *txnkv.Client
dataCoord types.DataCoord
queryCoord types.QueryCoord
dataCoord types.DataCoordClient
queryCoord types.QueryCoordClient
newDataCoordClient func(string, *clientv3.Client) types.DataCoord
newQueryCoordClient func(string, *clientv3.Client) types.QueryCoord
newDataCoordClient func(string, *clientv3.Client) types.DataCoordClient
newQueryCoordClient func(string, *clientv3.Client) types.QueryCoordClient
}
func (s *Server) CreateDatabase(ctx context.Context, request *milvuspb.CreateDatabaseRequest) (*commonpb.Status, error) {
@ -124,7 +124,7 @@ func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error)
}
func (s *Server) setClient() {
s.newDataCoordClient = func(etcdMetaRoot string, etcdCli *clientv3.Client) types.DataCoord {
s.newDataCoordClient = func(etcdMetaRoot string, etcdCli *clientv3.Client) types.DataCoordClient {
dsClient, err := dcc.NewClient(s.ctx, etcdMetaRoot, etcdCli)
if err != nil {
panic(err)
@ -132,7 +132,7 @@ func (s *Server) setClient() {
return dsClient
}
s.newQueryCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.QueryCoord {
s.newQueryCoordClient = func(metaRootPath string, etcdCli *clientv3.Client) types.QueryCoordClient {
qsClient, err := qcc.NewClient(s.ctx, metaRootPath, etcdCli)
if err != nil {
panic(err)
@ -201,15 +201,7 @@ func (s *Server) init() error {
log.Debug("RootCoord start to create DataCoord client")
dataCoord := s.newDataCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
s.dataCoord = dataCoord
if err = s.dataCoord.Init(); err != nil {
log.Error("RootCoord DataCoordClient Init failed", zap.Error(err))
panic(err)
}
if err = s.dataCoord.Start(); err != nil {
log.Error("RootCoord DataCoordClient Start failed", zap.Error(err))
panic(err)
}
if err := s.rootCoord.SetDataCoord(dataCoord); err != nil {
if err := s.rootCoord.SetDataCoordClient(dataCoord); err != nil {
panic(err)
}
}
@ -218,15 +210,7 @@ func (s *Server) init() error {
log.Debug("RootCoord start to create QueryCoord client")
queryCoord := s.newQueryCoordClient(rootcoord.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
s.queryCoord = queryCoord
if err := s.queryCoord.Init(); err != nil {
log.Error("RootCoord QueryCoordClient Init failed", zap.Error(err))
panic(err)
}
if err := s.queryCoord.Start(); err != nil {
log.Error("RootCoord QueryCoordClient Start failed", zap.Error(err))
panic(err)
}
if err := s.rootCoord.SetQueryCoord(queryCoord); err != nil {
if err := s.rootCoord.SetQueryCoordClient(queryCoord); err != nil {
panic(err)
}
}
@ -326,12 +310,12 @@ func (s *Server) Stop() error {
defer s.tikvCli.Close()
}
if s.dataCoord != nil {
if err := s.dataCoord.Stop(); err != nil {
if err := s.dataCoord.Close(); err != nil {
log.Error("Failed to close dataCoord client", zap.Error(err))
}
}
if s.queryCoord != nil {
if err := s.queryCoord.Stop(); err != nil {
if err := s.queryCoord.Close(); err != nil {
log.Error("Failed to close queryCoord client", zap.Error(err))
}
}
@ -352,17 +336,17 @@ func (s *Server) Stop() error {
// GetComponentStates gets the component states of RootCoord.
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.rootCoord.GetComponentStates(ctx)
return s.rootCoord.GetComponentStates(ctx, req)
}
// GetTimeTickChannel receiver time tick from proxy service, and put it into this channel
func (s *Server) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
return s.rootCoord.GetTimeTickChannel(ctx)
return s.rootCoord.GetTimeTickChannel(ctx, req)
}
// GetStatisticsChannel just define a channel, not used currently
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.rootCoord.GetStatisticsChannel(ctx)
return s.rootCoord.GetStatisticsChannel(ctx, req)
}
// CreateCollection creates a collection

View File

@ -28,6 +28,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/tikv/client-go/v2/txnkv"
clientv3 "go.etcd.io/etcd/client/v3"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -80,15 +81,15 @@ func (m *mockCore) SetEtcdClient(etcdClient *clientv3.Client) {
func (m *mockCore) SetTiKVClient(client *txnkv.Client) {
}
func (m *mockCore) SetDataCoord(types.DataCoord) error {
func (m *mockCore) SetDataCoordClient(client types.DataCoordClient) error {
return nil
}
func (m *mockCore) SetQueryCoord(types.QueryCoord) error {
func (m *mockCore) SetQueryCoordClient(client types.QueryCoordClient) error {
return nil
}
func (m *mockCore) SetProxyCreator(func(ctx context.Context, addr string, nodeID int64) (types.Proxy, error)) {
func (m *mockCore) SetProxyCreator(func(ctx context.Context, addr string, nodeID int64) (types.ProxyClient, error)) {
}
func (m *mockCore) Register() error {
@ -108,20 +109,14 @@ func (m *mockCore) Stop() error {
}
type mockDataCoord struct {
types.DataCoord
initErr error
startErr error
types.DataCoordClient
}
func (m *mockDataCoord) Init() error {
return m.initErr
func (m *mockDataCoord) Close() error {
return nil
}
func (m *mockDataCoord) Start() error {
return m.startErr
}
func (m *mockDataCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *mockDataCoord) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Healthy,
@ -140,20 +135,12 @@ func (m *mockDataCoord) Stop() error {
}
type mockQueryCoord struct {
types.QueryCoord
types.QueryCoordClient
initErr error
startErr error
}
func (m *mockQueryCoord) Init() error {
return m.initErr
}
func (m *mockQueryCoord) Start() error {
return m.startErr
}
func (m *mockQueryCoord) Stop() error {
func (m *mockQueryCoord) Close() error {
return fmt.Errorf("stop error")
}
@ -181,10 +168,10 @@ func TestRun(t *testing.T) {
assert.Error(t, err)
assert.EqualError(t, err, "listen tcp: address 1000000: invalid port")
svr.newDataCoordClient = func(string, *clientv3.Client) types.DataCoord {
svr.newDataCoordClient = func(string, *clientv3.Client) types.DataCoordClient {
return &mockDataCoord{}
}
svr.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoord {
svr.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoordClient {
return &mockQueryCoord{}
}
@ -260,8 +247,8 @@ func TestServerRun_DataCoordClientInitErr(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, server)
server.newDataCoordClient = func(string, *clientv3.Client) types.DataCoord {
return &mockDataCoord{initErr: errors.New("mock datacoord init error")}
server.newDataCoordClient = func(string, *clientv3.Client) types.DataCoordClient {
return &mockDataCoord{}
}
assert.Panics(t, func() { server.Run() })
@ -286,8 +273,8 @@ func TestServerRun_DataCoordClientStartErr(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, server)
server.newDataCoordClient = func(string, *clientv3.Client) types.DataCoord {
return &mockDataCoord{startErr: errors.New("mock datacoord start error")}
server.newDataCoordClient = func(string, *clientv3.Client) types.DataCoordClient {
return &mockDataCoord{}
}
assert.Panics(t, func() { server.Run() })
@ -312,7 +299,7 @@ func TestServerRun_QueryCoordClientInitErr(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, server)
server.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoord {
server.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoordClient {
return &mockQueryCoord{initErr: errors.New("mock querycoord init error")}
}
assert.Panics(t, func() { server.Run() })
@ -338,7 +325,7 @@ func TestServer_QueryCoordClientStartErr(t *testing.T) {
assert.NoError(t, err)
assert.NotNil(t, server)
server.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoord {
server.newQueryCoordClient = func(string, *clientv3.Client) types.QueryCoordClient {
return &mockQueryCoord{startErr: errors.New("mock querycoord start error")}
}
assert.Panics(t, func() { server.Run() })

View File

@ -273,7 +273,7 @@ func (i *IndexNode) SetEtcdClient(client *clientv3.Client) {
}
// GetComponentStates gets the component states of IndexNode.
func (i *IndexNode) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (i *IndexNode) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
log.RatedInfo(10, "get IndexNode components states ...")
nodeID := common.NotRegisteredID
if i.session != nil && i.session.Registered() {
@ -300,7 +300,7 @@ func (i *IndexNode) GetComponentStates(ctx context.Context) (*milvuspb.Component
}
// GetTimeTickChannel gets the time tick channel of IndexNode.
func (i *IndexNode) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (i *IndexNode) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
log.RatedInfo(10, "get IndexNode time tick channel ...")
return &milvuspb.StringResponse{
@ -309,7 +309,7 @@ func (i *IndexNode) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringRes
}
// GetStatisticsChannel gets the statistics channel of IndexNode.
func (i *IndexNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (i *IndexNode) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
log.RatedInfo(10, "get IndexNode statistics channel ...")
return &milvuspb.StringResponse{
Status: merr.Status(nil),

View File

@ -155,11 +155,11 @@ func (m *Mock) Stop() error {
return m.CallStop()
}
func (m *Mock) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (m *Mock) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return m.CallGetComponentStates(ctx)
}
func (m *Mock) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (m *Mock) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return m.CallGetStatisticsChannel(ctx)
}

View File

@ -465,26 +465,26 @@ func TestComponentState(t *testing.T) {
paramtable.Init()
in := NewIndexNode(ctx, factory)
in.SetEtcdClient(getEtcdClient())
state, err := in.GetComponentStates(ctx)
state, err := in.GetComponentStates(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, state.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
assert.Equal(t, state.State.StateCode, commonpb.StateCode_Abnormal)
assert.Nil(t, in.Init())
state, err = in.GetComponentStates(ctx)
state, err = in.GetComponentStates(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, state.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
assert.Equal(t, state.State.StateCode, commonpb.StateCode_Initializing)
assert.Nil(t, in.Start())
state, err = in.GetComponentStates(ctx)
state, err = in.GetComponentStates(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, state.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
assert.Equal(t, state.State.StateCode, commonpb.StateCode_Healthy)
assert.Nil(t, in.Stop())
assert.Nil(t, in.Stop())
state, err = in.GetComponentStates(ctx)
state, err = in.GetComponentStates(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, state.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
assert.Equal(t, state.State.StateCode, commonpb.StateCode_Abnormal)
@ -499,7 +499,7 @@ func TestGetTimeTickChannel(t *testing.T) {
)
paramtable.Init()
in := NewIndexNode(ctx, factory)
ret, err := in.GetTimeTickChannel(ctx)
ret, err := in.GetTimeTickChannel(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, ret.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
}
@ -514,7 +514,7 @@ func TestGetStatisticChannel(t *testing.T) {
paramtable.Init()
in := NewIndexNode(ctx, factory)
ret, err := in.GetStatisticsChannel(ctx)
ret, err := in.GetStatisticsChannel(ctx, nil)
assert.NoError(t, err)
assert.Equal(t, ret.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
}

View File

@ -460,48 +460,6 @@ func (_c *MetaKv_MultiRemove_Call) RunAndReturn(run func([]string) error) *MetaK
return _c
}
// MultiRemoveWithPrefix provides a mock function with given fields: keys
func (_m *MetaKv) MultiRemoveWithPrefix(keys []string) error {
ret := _m.Called(keys)
var r0 error
if rf, ok := ret.Get(0).(func([]string) error); ok {
r0 = rf(keys)
} else {
r0 = ret.Error(0)
}
return r0
}
// MetaKv_MultiRemoveWithPrefix_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MultiRemoveWithPrefix'
type MetaKv_MultiRemoveWithPrefix_Call struct {
*mock.Call
}
// MultiRemoveWithPrefix is a helper method to define mock.On call
// - keys []string
func (_e *MetaKv_Expecter) MultiRemoveWithPrefix(keys interface{}) *MetaKv_MultiRemoveWithPrefix_Call {
return &MetaKv_MultiRemoveWithPrefix_Call{Call: _e.mock.On("MultiRemoveWithPrefix", keys)}
}
func (_c *MetaKv_MultiRemoveWithPrefix_Call) Run(run func(keys []string)) *MetaKv_MultiRemoveWithPrefix_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].([]string))
})
return _c
}
func (_c *MetaKv_MultiRemoveWithPrefix_Call) Return(_a0 error) *MetaKv_MultiRemoveWithPrefix_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MetaKv_MultiRemoveWithPrefix_Call) RunAndReturn(run func([]string) error) *MetaKv_MultiRemoveWithPrefix_Call {
_c.Call.Return(run)
return _c
}
// MultiSave provides a mock function with given fields: kvs
func (_m *MetaKv) MultiSave(kvs map[string]string) error {
ret := _m.Called(kvs)

View File

@ -364,48 +364,6 @@ func (_c *TxnKV_MultiRemove_Call) RunAndReturn(run func([]string) error) *TxnKV_
return _c
}
// MultiRemoveWithPrefix provides a mock function with given fields: keys
func (_m *TxnKV) MultiRemoveWithPrefix(keys []string) error {
ret := _m.Called(keys)
var r0 error
if rf, ok := ret.Get(0).(func([]string) error); ok {
r0 = rf(keys)
} else {
r0 = ret.Error(0)
}
return r0
}
// TxnKV_MultiRemoveWithPrefix_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MultiRemoveWithPrefix'
type TxnKV_MultiRemoveWithPrefix_Call struct {
*mock.Call
}
// MultiRemoveWithPrefix is a helper method to define mock.On call
// - keys []string
func (_e *TxnKV_Expecter) MultiRemoveWithPrefix(keys interface{}) *TxnKV_MultiRemoveWithPrefix_Call {
return &TxnKV_MultiRemoveWithPrefix_Call{Call: _e.mock.On("MultiRemoveWithPrefix", keys)}
}
func (_c *TxnKV_MultiRemoveWithPrefix_Call) Run(run func(keys []string)) *TxnKV_MultiRemoveWithPrefix_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].([]string))
})
return _c
}
func (_c *TxnKV_MultiRemoveWithPrefix_Call) Return(_a0 error) *TxnKV_MultiRemoveWithPrefix_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *TxnKV_MultiRemoveWithPrefix_Call) RunAndReturn(run func([]string) error) *TxnKV_MultiRemoveWithPrefix_Call {
_c.Call.Return(run)
return _c
}
// MultiSave provides a mock function with given fields: kvs
func (_m *TxnKV) MultiSave(kvs map[string]string) error {
ret := _m.Called(kvs)

View File

@ -464,48 +464,6 @@ func (_c *WatchKV_MultiRemove_Call) RunAndReturn(run func([]string) error) *Watc
return _c
}
// MultiRemoveWithPrefix provides a mock function with given fields: keys
func (_m *WatchKV) MultiRemoveWithPrefix(keys []string) error {
ret := _m.Called(keys)
var r0 error
if rf, ok := ret.Get(0).(func([]string) error); ok {
r0 = rf(keys)
} else {
r0 = ret.Error(0)
}
return r0
}
// WatchKV_MultiRemoveWithPrefix_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MultiRemoveWithPrefix'
type WatchKV_MultiRemoveWithPrefix_Call struct {
*mock.Call
}
// MultiRemoveWithPrefix is a helper method to define mock.On call
// - keys []string
func (_e *WatchKV_Expecter) MultiRemoveWithPrefix(keys interface{}) *WatchKV_MultiRemoveWithPrefix_Call {
return &WatchKV_MultiRemoveWithPrefix_Call{Call: _e.mock.On("MultiRemoveWithPrefix", keys)}
}
func (_c *WatchKV_MultiRemoveWithPrefix_Call) Run(run func(keys []string)) *WatchKV_MultiRemoveWithPrefix_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].([]string))
})
return _c
}
func (_c *WatchKV_MultiRemoveWithPrefix_Call) Return(_a0 error) *WatchKV_MultiRemoveWithPrefix_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *WatchKV_MultiRemoveWithPrefix_Call) RunAndReturn(run func([]string) error) *WatchKV_MultiRemoveWithPrefix_Call {
_c.Call.Return(run)
return _c
}
// MultiSave provides a mock function with given fields: kvs
func (_m *WatchKV) MultiSave(kvs map[string]string) error {
ret := _m.Called(kvs)

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -32,17 +32,17 @@ func (_m *MockDataNode) EXPECT() *MockDataNode_Expecter {
return &MockDataNode_Expecter{mock: &_m.Mock}
}
// AddImportSegment provides a mock function with given fields: ctx, req
func (_m *MockDataNode) AddImportSegment(ctx context.Context, req *datapb.AddImportSegmentRequest) (*datapb.AddImportSegmentResponse, error) {
ret := _m.Called(ctx, req)
// 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(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.AddImportSegmentRequest) *datapb.AddImportSegmentResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.AddImportSegmentResponse)
@ -50,7 +50,7 @@ func (_m *MockDataNode) AddImportSegment(ctx context.Context, req *datapb.AddImp
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.AddImportSegmentRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -64,13 +64,13 @@ type MockDataNode_AddImportSegment_Call struct {
}
// AddImportSegment is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.AddImportSegmentRequest
func (_e *MockDataNode_Expecter) AddImportSegment(ctx interface{}, req interface{}) *MockDataNode_AddImportSegment_Call {
return &MockDataNode_AddImportSegment_Call{Call: _e.mock.On("AddImportSegment", ctx, req)}
// - _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(ctx context.Context, req *datapb.AddImportSegmentRequest)) *MockDataNode_AddImportSegment_Call {
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))
})
@ -87,17 +87,17 @@ func (_c *MockDataNode_AddImportSegment_Call) RunAndReturn(run func(context.Cont
return _c
}
// CheckChannelOperationProgress provides a mock function with given fields: ctx, req
func (_m *MockDataNode) CheckChannelOperationProgress(ctx context.Context, req *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error) {
ret := _m.Called(ctx, req)
// 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)
var r0 *datapb.ChannelOperationProgressResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ChannelWatchInfo) (*datapb.ChannelOperationProgressResponse, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.ChannelOperationProgressResponse)
@ -105,7 +105,7 @@ func (_m *MockDataNode) CheckChannelOperationProgress(ctx context.Context, req *
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ChannelWatchInfo) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -119,13 +119,13 @@ type MockDataNode_CheckChannelOperationProgress_Call struct {
}
// CheckChannelOperationProgress is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ChannelWatchInfo
func (_e *MockDataNode_Expecter) CheckChannelOperationProgress(ctx interface{}, req interface{}) *MockDataNode_CheckChannelOperationProgress_Call {
return &MockDataNode_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_CheckChannelOperationProgress_Call) Run(run func(ctx context.Context, req *datapb.ChannelWatchInfo)) *MockDataNode_CheckChannelOperationProgress_Call {
func (_c *MockDataNode_CheckChannelOperationProgress_Call) Run(run func(_a0 context.Context, _a1 *datapb.ChannelWatchInfo)) *MockDataNode_CheckChannelOperationProgress_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.ChannelWatchInfo))
})
@ -142,17 +142,17 @@ func (_c *MockDataNode_CheckChannelOperationProgress_Call) RunAndReturn(run func
return _c
}
// Compaction provides a mock function with given fields: ctx, req
func (_m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
// Compaction provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) Compaction(_a0 context.Context, _a1 *datapb.CompactionPlan) (*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.CompactionPlan) (*commonpb.Status, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.CompactionPlan) *commonpb.Status); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
@ -160,7 +160,7 @@ func (_m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPl
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.CompactionPlan) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -174,13 +174,13 @@ type MockDataNode_Compaction_Call struct {
}
// Compaction is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.CompactionPlan
func (_e *MockDataNode_Expecter) Compaction(ctx interface{}, req interface{}) *MockDataNode_Compaction_Call {
return &MockDataNode_Compaction_Call{Call: _e.mock.On("Compaction", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_Compaction_Call) Run(run func(ctx context.Context, req *datapb.CompactionPlan)) *MockDataNode_Compaction_Call {
func (_c *MockDataNode_Compaction_Call) Run(run func(_a0 context.Context, _a1 *datapb.CompactionPlan)) *MockDataNode_Compaction_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.CompactionPlan))
})
@ -197,17 +197,17 @@ func (_c *MockDataNode_Compaction_Call) RunAndReturn(run func(context.Context, *
return _c
}
// FlushChannels provides a mock function with given fields: ctx, req
func (_m *MockDataNode) FlushChannels(ctx context.Context, req *datapb.FlushChannelsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
// FlushChannels provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) FlushChannels(_a0 context.Context, _a1 *datapb.FlushChannelsRequest) (*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.FlushChannelsRequest) (*commonpb.Status, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.FlushChannelsRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
@ -215,7 +215,7 @@ func (_m *MockDataNode) FlushChannels(ctx context.Context, req *datapb.FlushChan
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.FlushChannelsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -229,13 +229,13 @@ type MockDataNode_FlushChannels_Call struct {
}
// FlushChannels is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.FlushChannelsRequest
func (_e *MockDataNode_Expecter) FlushChannels(ctx interface{}, req interface{}) *MockDataNode_FlushChannels_Call {
return &MockDataNode_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_FlushChannels_Call) Run(run func(ctx context.Context, req *datapb.FlushChannelsRequest)) *MockDataNode_FlushChannels_Call {
func (_c *MockDataNode_FlushChannels_Call) Run(run func(_a0 context.Context, _a1 *datapb.FlushChannelsRequest)) *MockDataNode_FlushChannels_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.FlushChannelsRequest))
})
@ -252,17 +252,17 @@ func (_c *MockDataNode_FlushChannels_Call) RunAndReturn(run func(context.Context
return _c
}
// FlushSegments provides a mock function with given fields: ctx, req
func (_m *MockDataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
// FlushSegments provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) FlushSegments(_a0 context.Context, _a1 *datapb.FlushSegmentsRequest) (*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.FlushSegmentsRequest) (*commonpb.Status, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.FlushSegmentsRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
@ -270,7 +270,7 @@ func (_m *MockDataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegm
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.FlushSegmentsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -284,13 +284,13 @@ type MockDataNode_FlushSegments_Call struct {
}
// FlushSegments is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.FlushSegmentsRequest
func (_e *MockDataNode_Expecter) FlushSegments(ctx interface{}, req interface{}) *MockDataNode_FlushSegments_Call {
return &MockDataNode_FlushSegments_Call{Call: _e.mock.On("FlushSegments", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_FlushSegments_Call) Run(run func(ctx context.Context, req *datapb.FlushSegmentsRequest)) *MockDataNode_FlushSegments_Call {
func (_c *MockDataNode_FlushSegments_Call) Run(run func(_a0 context.Context, _a1 *datapb.FlushSegmentsRequest)) *MockDataNode_FlushSegments_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.FlushSegmentsRequest))
})
@ -348,17 +348,17 @@ func (_c *MockDataNode_GetAddress_Call) RunAndReturn(run func() string) *MockDat
return _c
}
// GetCompactionState provides a mock function with given fields: ctx, req
func (_m *MockDataNode) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
ret := _m.Called(ctx, req)
// GetCompactionState provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) GetCompactionState(_a0 context.Context, _a1 *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *datapb.CompactionStateResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.CompactionStateRequest) *datapb.CompactionStateResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.CompactionStateResponse)
@ -366,7 +366,7 @@ func (_m *MockDataNode) GetCompactionState(ctx context.Context, req *datapb.Comp
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.CompactionStateRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -380,13 +380,13 @@ type MockDataNode_GetCompactionState_Call struct {
}
// GetCompactionState is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.CompactionStateRequest
func (_e *MockDataNode_Expecter) GetCompactionState(ctx interface{}, req interface{}) *MockDataNode_GetCompactionState_Call {
return &MockDataNode_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_GetCompactionState_Call) Run(run func(ctx context.Context, req *datapb.CompactionStateRequest)) *MockDataNode_GetCompactionState_Call {
func (_c *MockDataNode_GetCompactionState_Call) Run(run func(_a0 context.Context, _a1 *datapb.CompactionStateRequest)) *MockDataNode_GetCompactionState_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.CompactionStateRequest))
})
@ -403,25 +403,25 @@ func (_c *MockDataNode_GetCompactionState_Call) RunAndReturn(run func(context.Co
return _c
}
// GetComponentStates provides a mock function with given fields: ctx
func (_m *MockDataNode) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
ret := _m.Called(ctx)
// GetComponentStates provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) GetComponentStates(_a0 context.Context, _a1 *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
ret := _m.Called(_a0, _a1)
var r0 *milvuspb.ComponentStates
var r1 error
if rf, ok := ret.Get(0).(func(context.Context) (*milvuspb.ComponentStates, error)); ok {
return rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context) *milvuspb.ComponentStates); ok {
r0 = rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest) *milvuspb.ComponentStates); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.ComponentStates)
}
}
if rf, ok := ret.Get(1).(func(context.Context) error); ok {
r1 = rf(ctx)
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetComponentStatesRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -435,14 +435,15 @@ type MockDataNode_GetComponentStates_Call struct {
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
func (_e *MockDataNode_Expecter) GetComponentStates(ctx interface{}) *MockDataNode_GetComponentStates_Call {
return &MockDataNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", ctx)}
// - _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)}
}
func (_c *MockDataNode_GetComponentStates_Call) Run(run func(ctx context.Context)) *MockDataNode_GetComponentStates_Call {
func (_c *MockDataNode_GetComponentStates_Call) Run(run func(_a0 context.Context, _a1 *milvuspb.GetComponentStatesRequest)) *MockDataNode_GetComponentStates_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
run(args[0].(context.Context), args[1].(*milvuspb.GetComponentStatesRequest))
})
return _c
}
@ -452,22 +453,22 @@ func (_c *MockDataNode_GetComponentStates_Call) Return(_a0 *milvuspb.ComponentSt
return _c
}
func (_c *MockDataNode_GetComponentStates_Call) RunAndReturn(run func(context.Context) (*milvuspb.ComponentStates, error)) *MockDataNode_GetComponentStates_Call {
func (_c *MockDataNode_GetComponentStates_Call) RunAndReturn(run func(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error)) *MockDataNode_GetComponentStates_Call {
_c.Call.Return(run)
return _c
}
// GetMetrics provides a mock function with given fields: ctx, req
func (_m *MockDataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
ret := _m.Called(ctx, req)
// GetMetrics provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) GetMetrics(_a0 context.Context, _a1 *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *milvuspb.GetMetricsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest) *milvuspb.GetMetricsResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.GetMetricsResponse)
@ -475,7 +476,7 @@ func (_m *MockDataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetrics
}
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetMetricsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -489,13 +490,13 @@ type MockDataNode_GetMetrics_Call struct {
}
// GetMetrics is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
func (_e *MockDataNode_Expecter) GetMetrics(ctx interface{}, req interface{}) *MockDataNode_GetMetrics_Call {
return &MockDataNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_GetMetrics_Call) Run(run func(ctx context.Context, req *milvuspb.GetMetricsRequest)) *MockDataNode_GetMetrics_Call {
func (_c *MockDataNode_GetMetrics_Call) Run(run func(_a0 context.Context, _a1 *milvuspb.GetMetricsRequest)) *MockDataNode_GetMetrics_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*milvuspb.GetMetricsRequest))
})
@ -553,25 +554,25 @@ func (_c *MockDataNode_GetStateCode_Call) RunAndReturn(run func() commonpb.State
return _c
}
// GetStatisticsChannel provides a mock function with given fields: ctx
func (_m *MockDataNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
ret := _m.Called(ctx)
// GetStatisticsChannel provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) GetStatisticsChannel(_a0 context.Context, _a1 *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *milvuspb.StringResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context) (*milvuspb.StringResponse, error)); ok {
return rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context) *milvuspb.StringResponse); ok {
r0 = rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest) *milvuspb.StringResponse); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.StringResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context) error); ok {
r1 = rf(ctx)
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.GetStatisticsChannelRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -585,14 +586,15 @@ type MockDataNode_GetStatisticsChannel_Call struct {
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
func (_e *MockDataNode_Expecter) GetStatisticsChannel(ctx interface{}) *MockDataNode_GetStatisticsChannel_Call {
return &MockDataNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", ctx)}
// - _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)}
}
func (_c *MockDataNode_GetStatisticsChannel_Call) Run(run func(ctx context.Context)) *MockDataNode_GetStatisticsChannel_Call {
func (_c *MockDataNode_GetStatisticsChannel_Call) Run(run func(_a0 context.Context, _a1 *internalpb.GetStatisticsChannelRequest)) *MockDataNode_GetStatisticsChannel_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
run(args[0].(context.Context), args[1].(*internalpb.GetStatisticsChannelRequest))
})
return _c
}
@ -602,22 +604,22 @@ func (_c *MockDataNode_GetStatisticsChannel_Call) Return(_a0 *milvuspb.StringRes
return _c
}
func (_c *MockDataNode_GetStatisticsChannel_Call) RunAndReturn(run func(context.Context) (*milvuspb.StringResponse, error)) *MockDataNode_GetStatisticsChannel_Call {
func (_c *MockDataNode_GetStatisticsChannel_Call) RunAndReturn(run func(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error)) *MockDataNode_GetStatisticsChannel_Call {
_c.Call.Return(run)
return _c
}
// Import provides a mock function with given fields: ctx, req
func (_m *MockDataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
// 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(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ImportTaskRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
@ -625,7 +627,7 @@ func (_m *MockDataNode) Import(ctx context.Context, req *datapb.ImportTaskReques
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ImportTaskRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -639,13 +641,13 @@ type MockDataNode_Import_Call struct {
}
// Import is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ImportTaskRequest
func (_e *MockDataNode_Expecter) Import(ctx interface{}, req interface{}) *MockDataNode_Import_Call {
return &MockDataNode_Import_Call{Call: _e.mock.On("Import", ctx, req)}
// - _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(ctx context.Context, req *datapb.ImportTaskRequest)) *MockDataNode_Import_Call {
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))
})
@ -703,17 +705,17 @@ func (_c *MockDataNode_Init_Call) RunAndReturn(run func() error) *MockDataNode_I
return _c
}
// NotifyChannelOperation provides a mock function with given fields: ctx, req
func (_m *MockDataNode) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
// NotifyChannelOperation provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) NotifyChannelOperation(_a0 context.Context, _a1 *datapb.ChannelOperationsRequest) (*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.ChannelOperationsRequest) (*commonpb.Status, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ChannelOperationsRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
@ -721,7 +723,7 @@ func (_m *MockDataNode) NotifyChannelOperation(ctx context.Context, req *datapb.
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ChannelOperationsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -735,13 +737,13 @@ type MockDataNode_NotifyChannelOperation_Call struct {
}
// NotifyChannelOperation is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ChannelOperationsRequest
func (_e *MockDataNode_Expecter) NotifyChannelOperation(ctx interface{}, req interface{}) *MockDataNode_NotifyChannelOperation_Call {
return &MockDataNode_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_NotifyChannelOperation_Call) Run(run func(ctx context.Context, req *datapb.ChannelOperationsRequest)) *MockDataNode_NotifyChannelOperation_Call {
func (_c *MockDataNode_NotifyChannelOperation_Call) Run(run func(_a0 context.Context, _a1 *datapb.ChannelOperationsRequest)) *MockDataNode_NotifyChannelOperation_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.ChannelOperationsRequest))
})
@ -799,17 +801,17 @@ func (_c *MockDataNode_Register_Call) RunAndReturn(run func() error) *MockDataNo
return _c
}
// ResendSegmentStats provides a mock function with given fields: ctx, req
func (_m *MockDataNode) ResendSegmentStats(ctx context.Context, req *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
ret := _m.Called(ctx, req)
// ResendSegmentStats provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) ResendSegmentStats(_a0 context.Context, _a1 *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *datapb.ResendSegmentStatsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ResendSegmentStatsRequest) (*datapb.ResendSegmentStatsResponse, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ResendSegmentStatsRequest) *datapb.ResendSegmentStatsResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.ResendSegmentStatsResponse)
@ -817,7 +819,7 @@ func (_m *MockDataNode) ResendSegmentStats(ctx context.Context, req *datapb.Rese
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.ResendSegmentStatsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -831,13 +833,13 @@ type MockDataNode_ResendSegmentStats_Call struct {
}
// ResendSegmentStats is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.ResendSegmentStatsRequest
func (_e *MockDataNode_Expecter) ResendSegmentStats(ctx interface{}, req interface{}) *MockDataNode_ResendSegmentStats_Call {
return &MockDataNode_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_ResendSegmentStats_Call) Run(run func(ctx context.Context, req *datapb.ResendSegmentStatsRequest)) *MockDataNode_ResendSegmentStats_Call {
func (_c *MockDataNode_ResendSegmentStats_Call) Run(run func(_a0 context.Context, _a1 *datapb.ResendSegmentStatsRequest)) *MockDataNode_ResendSegmentStats_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.ResendSegmentStatsRequest))
})
@ -887,12 +889,12 @@ func (_c *MockDataNode_SetAddress_Call) RunAndReturn(run func(string)) *MockData
return _c
}
// SetDataCoord provides a mock function with given fields: dataCoord
func (_m *MockDataNode) SetDataCoord(dataCoord types.DataCoord) error {
// SetDataCoordClient provides a mock function with given fields: dataCoord
func (_m *MockDataNode) SetDataCoordClient(dataCoord types.DataCoordClient) error {
ret := _m.Called(dataCoord)
var r0 error
if rf, ok := ret.Get(0).(func(types.DataCoord) error); ok {
if rf, ok := ret.Get(0).(func(types.DataCoordClient) error); ok {
r0 = rf(dataCoord)
} else {
r0 = ret.Error(0)
@ -901,30 +903,30 @@ func (_m *MockDataNode) SetDataCoord(dataCoord types.DataCoord) error {
return r0
}
// MockDataNode_SetDataCoord_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetDataCoord'
type MockDataNode_SetDataCoord_Call struct {
// MockDataNode_SetDataCoordClient_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetDataCoordClient'
type MockDataNode_SetDataCoordClient_Call struct {
*mock.Call
}
// SetDataCoord is a helper method to define mock.On call
// - dataCoord types.DataCoord
func (_e *MockDataNode_Expecter) SetDataCoord(dataCoord interface{}) *MockDataNode_SetDataCoord_Call {
return &MockDataNode_SetDataCoord_Call{Call: _e.mock.On("SetDataCoord", dataCoord)}
// SetDataCoordClient is a helper method to define mock.On call
// - dataCoord types.DataCoordClient
func (_e *MockDataNode_Expecter) SetDataCoordClient(dataCoord interface{}) *MockDataNode_SetDataCoordClient_Call {
return &MockDataNode_SetDataCoordClient_Call{Call: _e.mock.On("SetDataCoordClient", dataCoord)}
}
func (_c *MockDataNode_SetDataCoord_Call) Run(run func(dataCoord types.DataCoord)) *MockDataNode_SetDataCoord_Call {
func (_c *MockDataNode_SetDataCoordClient_Call) Run(run func(dataCoord types.DataCoordClient)) *MockDataNode_SetDataCoordClient_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(types.DataCoord))
run(args[0].(types.DataCoordClient))
})
return _c
}
func (_c *MockDataNode_SetDataCoord_Call) Return(_a0 error) *MockDataNode_SetDataCoord_Call {
func (_c *MockDataNode_SetDataCoordClient_Call) Return(_a0 error) *MockDataNode_SetDataCoordClient_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockDataNode_SetDataCoord_Call) RunAndReturn(run func(types.DataCoord) error) *MockDataNode_SetDataCoord_Call {
func (_c *MockDataNode_SetDataCoordClient_Call) RunAndReturn(run func(types.DataCoordClient) error) *MockDataNode_SetDataCoordClient_Call {
_c.Call.Return(run)
return _c
}
@ -962,12 +964,12 @@ func (_c *MockDataNode_SetEtcdClient_Call) RunAndReturn(run func(*clientv3.Clien
return _c
}
// SetRootCoord provides a mock function with given fields: rootCoord
func (_m *MockDataNode) SetRootCoord(rootCoord types.RootCoord) error {
// SetRootCoordClient provides a mock function with given fields: rootCoord
func (_m *MockDataNode) SetRootCoordClient(rootCoord types.RootCoordClient) error {
ret := _m.Called(rootCoord)
var r0 error
if rf, ok := ret.Get(0).(func(types.RootCoord) error); ok {
if rf, ok := ret.Get(0).(func(types.RootCoordClient) error); ok {
r0 = rf(rootCoord)
} else {
r0 = ret.Error(0)
@ -976,45 +978,45 @@ func (_m *MockDataNode) SetRootCoord(rootCoord types.RootCoord) error {
return r0
}
// MockDataNode_SetRootCoord_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetRootCoord'
type MockDataNode_SetRootCoord_Call struct {
// MockDataNode_SetRootCoordClient_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetRootCoordClient'
type MockDataNode_SetRootCoordClient_Call struct {
*mock.Call
}
// SetRootCoord is a helper method to define mock.On call
// - rootCoord types.RootCoord
func (_e *MockDataNode_Expecter) SetRootCoord(rootCoord interface{}) *MockDataNode_SetRootCoord_Call {
return &MockDataNode_SetRootCoord_Call{Call: _e.mock.On("SetRootCoord", rootCoord)}
// SetRootCoordClient is a helper method to define mock.On call
// - rootCoord types.RootCoordClient
func (_e *MockDataNode_Expecter) SetRootCoordClient(rootCoord interface{}) *MockDataNode_SetRootCoordClient_Call {
return &MockDataNode_SetRootCoordClient_Call{Call: _e.mock.On("SetRootCoordClient", rootCoord)}
}
func (_c *MockDataNode_SetRootCoord_Call) Run(run func(rootCoord types.RootCoord)) *MockDataNode_SetRootCoord_Call {
func (_c *MockDataNode_SetRootCoordClient_Call) Run(run func(rootCoord types.RootCoordClient)) *MockDataNode_SetRootCoordClient_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(types.RootCoord))
run(args[0].(types.RootCoordClient))
})
return _c
}
func (_c *MockDataNode_SetRootCoord_Call) Return(_a0 error) *MockDataNode_SetRootCoord_Call {
func (_c *MockDataNode_SetRootCoordClient_Call) Return(_a0 error) *MockDataNode_SetRootCoordClient_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockDataNode_SetRootCoord_Call) RunAndReturn(run func(types.RootCoord) error) *MockDataNode_SetRootCoord_Call {
func (_c *MockDataNode_SetRootCoordClient_Call) RunAndReturn(run func(types.RootCoordClient) error) *MockDataNode_SetRootCoordClient_Call {
_c.Call.Return(run)
return _c
}
// ShowConfigurations provides a mock function with given fields: ctx, req
func (_m *MockDataNode) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
ret := _m.Called(ctx, req)
// ShowConfigurations provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) ShowConfigurations(_a0 context.Context, _a1 *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *internalpb.ShowConfigurationsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*internalpb.ShowConfigurationsResponse)
@ -1022,7 +1024,7 @@ func (_m *MockDataNode) ShowConfigurations(ctx context.Context, req *internalpb.
}
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.ShowConfigurationsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -1036,13 +1038,13 @@ type MockDataNode_ShowConfigurations_Call struct {
}
// ShowConfigurations is a helper method to define mock.On call
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
func (_e *MockDataNode_Expecter) ShowConfigurations(ctx interface{}, req interface{}) *MockDataNode_ShowConfigurations_Call {
return &MockDataNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_ShowConfigurations_Call) Run(run func(ctx context.Context, req *internalpb.ShowConfigurationsRequest)) *MockDataNode_ShowConfigurations_Call {
func (_c *MockDataNode_ShowConfigurations_Call) Run(run func(_a0 context.Context, _a1 *internalpb.ShowConfigurationsRequest)) *MockDataNode_ShowConfigurations_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*internalpb.ShowConfigurationsRequest))
})
@ -1141,17 +1143,17 @@ func (_c *MockDataNode_Stop_Call) RunAndReturn(run func() error) *MockDataNode_S
return _c
}
// SyncSegments provides a mock function with given fields: ctx, req
func (_m *MockDataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
// SyncSegments provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) SyncSegments(_a0 context.Context, _a1 *datapb.SyncSegmentsRequest) (*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.SyncSegmentsRequest) (*commonpb.Status, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.SyncSegmentsRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
@ -1159,7 +1161,7 @@ func (_m *MockDataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegmen
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.SyncSegmentsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -1173,13 +1175,13 @@ type MockDataNode_SyncSegments_Call struct {
}
// SyncSegments is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.SyncSegmentsRequest
func (_e *MockDataNode_Expecter) SyncSegments(ctx interface{}, req interface{}) *MockDataNode_SyncSegments_Call {
return &MockDataNode_SyncSegments_Call{Call: _e.mock.On("SyncSegments", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_SyncSegments_Call) Run(run func(ctx context.Context, req *datapb.SyncSegmentsRequest)) *MockDataNode_SyncSegments_Call {
func (_c *MockDataNode_SyncSegments_Call) Run(run func(_a0 context.Context, _a1 *datapb.SyncSegmentsRequest)) *MockDataNode_SyncSegments_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.SyncSegmentsRequest))
})
@ -1229,17 +1231,17 @@ func (_c *MockDataNode_UpdateStateCode_Call) RunAndReturn(run func(commonpb.Stat
return _c
}
// WatchDmChannels provides a mock function with given fields: ctx, req
func (_m *MockDataNode) WatchDmChannels(ctx context.Context, req *datapb.WatchDmChannelsRequest) (*commonpb.Status, error) {
ret := _m.Called(ctx, req)
// WatchDmChannels provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) WatchDmChannels(_a0 context.Context, _a1 *datapb.WatchDmChannelsRequest) (*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.WatchDmChannelsRequest) (*commonpb.Status, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.WatchDmChannelsRequest) *commonpb.Status); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
@ -1247,7 +1249,7 @@ func (_m *MockDataNode) WatchDmChannels(ctx context.Context, req *datapb.WatchDm
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.WatchDmChannelsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -1261,13 +1263,13 @@ type MockDataNode_WatchDmChannels_Call struct {
}
// WatchDmChannels is a helper method to define mock.On call
// - ctx context.Context
// - req *datapb.WatchDmChannelsRequest
func (_e *MockDataNode_Expecter) WatchDmChannels(ctx interface{}, req interface{}) *MockDataNode_WatchDmChannels_Call {
return &MockDataNode_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", ctx, req)}
// - _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)}
}
func (_c *MockDataNode_WatchDmChannels_Call) Run(run func(ctx context.Context, req *datapb.WatchDmChannelsRequest)) *MockDataNode_WatchDmChannels_Call {
func (_c *MockDataNode_WatchDmChannels_Call) Run(run func(_a0 context.Context, _a1 *datapb.WatchDmChannelsRequest)) *MockDataNode_WatchDmChannels_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.WatchDmChannelsRequest))
})
@ -1289,8 +1291,7 @@ func (_c *MockDataNode_WatchDmChannels_Call) RunAndReturn(run func(context.Conte
func NewMockDataNode(t interface {
mock.TestingT
Cleanup(func())
},
) *MockDataNode {
}) *MockDataNode {
mock := &MockDataNode{}
mock.Mock.Test(t)

File diff suppressed because it is too large Load Diff

View File

@ -181,25 +181,25 @@ func (_c *MockIndexNode_GetAddress_Call) RunAndReturn(run func() string) *MockIn
return _c
}
// GetComponentStates provides a mock function with given fields: ctx
func (_m *MockIndexNode) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
ret := _m.Called(ctx)
// GetComponentStates provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) GetComponentStates(_a0 context.Context, _a1 *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
ret := _m.Called(_a0, _a1)
var r0 *milvuspb.ComponentStates
var r1 error
if rf, ok := ret.Get(0).(func(context.Context) (*milvuspb.ComponentStates, error)); ok {
return rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context) *milvuspb.ComponentStates); ok {
r0 = rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest) *milvuspb.ComponentStates); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.ComponentStates)
}
}
if rf, ok := ret.Get(1).(func(context.Context) error); ok {
r1 = rf(ctx)
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetComponentStatesRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -213,14 +213,15 @@ type MockIndexNode_GetComponentStates_Call struct {
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
func (_e *MockIndexNode_Expecter) GetComponentStates(ctx interface{}) *MockIndexNode_GetComponentStates_Call {
return &MockIndexNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", ctx)}
// - _a0 context.Context
// - _a1 *milvuspb.GetComponentStatesRequest
func (_e *MockIndexNode_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *MockIndexNode_GetComponentStates_Call {
return &MockIndexNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)}
}
func (_c *MockIndexNode_GetComponentStates_Call) Run(run func(ctx context.Context)) *MockIndexNode_GetComponentStates_Call {
func (_c *MockIndexNode_GetComponentStates_Call) Run(run func(_a0 context.Context, _a1 *milvuspb.GetComponentStatesRequest)) *MockIndexNode_GetComponentStates_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
run(args[0].(context.Context), args[1].(*milvuspb.GetComponentStatesRequest))
})
return _c
}
@ -230,7 +231,7 @@ func (_c *MockIndexNode_GetComponentStates_Call) Return(_a0 *milvuspb.ComponentS
return _c
}
func (_c *MockIndexNode_GetComponentStates_Call) RunAndReturn(run func(context.Context) (*milvuspb.ComponentStates, error)) *MockIndexNode_GetComponentStates_Call {
func (_c *MockIndexNode_GetComponentStates_Call) RunAndReturn(run func(context.Context, *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error)) *MockIndexNode_GetComponentStates_Call {
_c.Call.Return(run)
return _c
}
@ -290,17 +291,17 @@ func (_c *MockIndexNode_GetJobStats_Call) RunAndReturn(run func(context.Context,
return _c
}
// GetMetrics provides a mock function with given fields: ctx, req
func (_m *MockIndexNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
ret := _m.Called(ctx, req)
// GetMetrics provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) GetMetrics(_a0 context.Context, _a1 *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *milvuspb.GetMetricsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest) *milvuspb.GetMetricsResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.GetMetricsResponse)
@ -308,7 +309,7 @@ func (_m *MockIndexNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetric
}
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetMetricsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -322,13 +323,13 @@ type MockIndexNode_GetMetrics_Call struct {
}
// GetMetrics is a helper method to define mock.On call
// - ctx context.Context
// - req *milvuspb.GetMetricsRequest
func (_e *MockIndexNode_Expecter) GetMetrics(ctx interface{}, req interface{}) *MockIndexNode_GetMetrics_Call {
return &MockIndexNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", ctx, req)}
// - _a0 context.Context
// - _a1 *milvuspb.GetMetricsRequest
func (_e *MockIndexNode_Expecter) GetMetrics(_a0 interface{}, _a1 interface{}) *MockIndexNode_GetMetrics_Call {
return &MockIndexNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", _a0, _a1)}
}
func (_c *MockIndexNode_GetMetrics_Call) Run(run func(ctx context.Context, req *milvuspb.GetMetricsRequest)) *MockIndexNode_GetMetrics_Call {
func (_c *MockIndexNode_GetMetrics_Call) Run(run func(_a0 context.Context, _a1 *milvuspb.GetMetricsRequest)) *MockIndexNode_GetMetrics_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*milvuspb.GetMetricsRequest))
})
@ -345,25 +346,25 @@ func (_c *MockIndexNode_GetMetrics_Call) RunAndReturn(run func(context.Context,
return _c
}
// GetStatisticsChannel provides a mock function with given fields: ctx
func (_m *MockIndexNode) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
ret := _m.Called(ctx)
// GetStatisticsChannel provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) GetStatisticsChannel(_a0 context.Context, _a1 *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *milvuspb.StringResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context) (*milvuspb.StringResponse, error)); ok {
return rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context) *milvuspb.StringResponse); ok {
r0 = rf(ctx)
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest) *milvuspb.StringResponse); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.StringResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context) error); ok {
r1 = rf(ctx)
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.GetStatisticsChannelRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -377,14 +378,15 @@ type MockIndexNode_GetStatisticsChannel_Call struct {
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
func (_e *MockIndexNode_Expecter) GetStatisticsChannel(ctx interface{}) *MockIndexNode_GetStatisticsChannel_Call {
return &MockIndexNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", ctx)}
// - _a0 context.Context
// - _a1 *internalpb.GetStatisticsChannelRequest
func (_e *MockIndexNode_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *MockIndexNode_GetStatisticsChannel_Call {
return &MockIndexNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)}
}
func (_c *MockIndexNode_GetStatisticsChannel_Call) Run(run func(ctx context.Context)) *MockIndexNode_GetStatisticsChannel_Call {
func (_c *MockIndexNode_GetStatisticsChannel_Call) Run(run func(_a0 context.Context, _a1 *internalpb.GetStatisticsChannelRequest)) *MockIndexNode_GetStatisticsChannel_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
run(args[0].(context.Context), args[1].(*internalpb.GetStatisticsChannelRequest))
})
return _c
}
@ -394,7 +396,7 @@ func (_c *MockIndexNode_GetStatisticsChannel_Call) Return(_a0 *milvuspb.StringRe
return _c
}
func (_c *MockIndexNode_GetStatisticsChannel_Call) RunAndReturn(run func(context.Context) (*milvuspb.StringResponse, error)) *MockIndexNode_GetStatisticsChannel_Call {
func (_c *MockIndexNode_GetStatisticsChannel_Call) RunAndReturn(run func(context.Context, *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error)) *MockIndexNode_GetStatisticsChannel_Call {
_c.Call.Return(run)
return _c
}
@ -602,17 +604,17 @@ func (_c *MockIndexNode_SetEtcdClient_Call) RunAndReturn(run func(*clientv3.Clie
return _c
}
// ShowConfigurations provides a mock function with given fields: ctx, req
func (_m *MockIndexNode) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
ret := _m.Called(ctx, req)
// ShowConfigurations provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) ShowConfigurations(_a0 context.Context, _a1 *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *internalpb.ShowConfigurationsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)); ok {
return rf(ctx, req)
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ShowConfigurationsRequest) *internalpb.ShowConfigurationsResponse); ok {
r0 = rf(ctx, req)
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*internalpb.ShowConfigurationsResponse)
@ -620,7 +622,7 @@ func (_m *MockIndexNode) ShowConfigurations(ctx context.Context, req *internalpb
}
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.ShowConfigurationsRequest) error); ok {
r1 = rf(ctx, req)
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
@ -634,13 +636,13 @@ type MockIndexNode_ShowConfigurations_Call struct {
}
// ShowConfigurations is a helper method to define mock.On call
// - ctx context.Context
// - req *internalpb.ShowConfigurationsRequest
func (_e *MockIndexNode_Expecter) ShowConfigurations(ctx interface{}, req interface{}) *MockIndexNode_ShowConfigurations_Call {
return &MockIndexNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", ctx, req)}
// - _a0 context.Context
// - _a1 *internalpb.ShowConfigurationsRequest
func (_e *MockIndexNode_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *MockIndexNode_ShowConfigurations_Call {
return &MockIndexNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)}
}
func (_c *MockIndexNode_ShowConfigurations_Call) Run(run func(ctx context.Context, req *internalpb.ShowConfigurationsRequest)) *MockIndexNode_ShowConfigurations_Call {
func (_c *MockIndexNode_ShowConfigurations_Call) Run(run func(_a0 context.Context, _a1 *internalpb.ShowConfigurationsRequest)) *MockIndexNode_ShowConfigurations_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*internalpb.ShowConfigurationsRequest))
})

View File

@ -0,0 +1,647 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
package mocks
import (
context "context"
commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
grpc "google.golang.org/grpc"
indexpb "github.com/milvus-io/milvus/internal/proto/indexpb"
internalpb "github.com/milvus-io/milvus/internal/proto/internalpb"
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
mock "github.com/stretchr/testify/mock"
)
// MockIndexNodeClient is an autogenerated mock type for the IndexNodeClient type
type MockIndexNodeClient struct {
mock.Mock
}
type MockIndexNodeClient_Expecter struct {
mock *mock.Mock
}
func (_m *MockIndexNodeClient) EXPECT() *MockIndexNodeClient_Expecter {
return &MockIndexNodeClient_Expecter{mock: &_m.Mock}
}
// Close provides a mock function with given fields:
func (_m *MockIndexNodeClient) Close() error {
ret := _m.Called()
var r0 error
if rf, ok := ret.Get(0).(func() error); ok {
r0 = rf()
} else {
r0 = ret.Error(0)
}
return r0
}
// MockIndexNodeClient_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockIndexNodeClient_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockIndexNodeClient_Expecter) Close() *MockIndexNodeClient_Close_Call {
return &MockIndexNodeClient_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockIndexNodeClient_Close_Call) Run(run func()) *MockIndexNodeClient_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockIndexNodeClient_Close_Call) Return(_a0 error) *MockIndexNodeClient_Close_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockIndexNodeClient_Close_Call) RunAndReturn(run func() error) *MockIndexNodeClient_Close_Call {
_c.Call.Return(run)
return _c
}
// CreateJob provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) CreateJob(ctx context.Context, in *indexpb.CreateJobRequest, 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, *indexpb.CreateJobRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.CreateJobRequest, ...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, *indexpb.CreateJobRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_CreateJob_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateJob'
type MockIndexNodeClient_CreateJob_Call struct {
*mock.Call
}
// CreateJob is a helper method to define mock.On call
// - ctx context.Context
// - in *indexpb.CreateJobRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) CreateJob(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_CreateJob_Call {
return &MockIndexNodeClient_CreateJob_Call{Call: _e.mock.On("CreateJob",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_CreateJob_Call) Run(run func(ctx context.Context, in *indexpb.CreateJobRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_CreateJob_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].(*indexpb.CreateJobRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_CreateJob_Call) Return(_a0 *commonpb.Status, _a1 error) *MockIndexNodeClient_CreateJob_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_CreateJob_Call) RunAndReturn(run func(context.Context, *indexpb.CreateJobRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockIndexNodeClient_CreateJob_Call {
_c.Call.Return(run)
return _c
}
// DropJobs provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) DropJobs(ctx context.Context, in *indexpb.DropJobsRequest, 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, *indexpb.DropJobsRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.DropJobsRequest, ...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, *indexpb.DropJobsRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_DropJobs_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropJobs'
type MockIndexNodeClient_DropJobs_Call struct {
*mock.Call
}
// DropJobs is a helper method to define mock.On call
// - ctx context.Context
// - in *indexpb.DropJobsRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) DropJobs(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_DropJobs_Call {
return &MockIndexNodeClient_DropJobs_Call{Call: _e.mock.On("DropJobs",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_DropJobs_Call) Run(run func(ctx context.Context, in *indexpb.DropJobsRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_DropJobs_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].(*indexpb.DropJobsRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_DropJobs_Call) Return(_a0 *commonpb.Status, _a1 error) *MockIndexNodeClient_DropJobs_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_DropJobs_Call) RunAndReturn(run func(context.Context, *indexpb.DropJobsRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockIndexNodeClient_DropJobs_Call {
_c.Call.Return(run)
return _c
}
// GetComponentStates provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, 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.ComponentStates
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) (*milvuspb.ComponentStates, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) *milvuspb.ComponentStates); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.ComponentStates)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_GetComponentStates_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetComponentStates'
type MockIndexNodeClient_GetComponentStates_Call struct {
*mock.Call
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
// - in *milvuspb.GetComponentStatesRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_GetComponentStates_Call {
return &MockIndexNodeClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_GetComponentStates_Call) Run(run func(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_GetComponentStates_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.GetComponentStatesRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_GetComponentStates_Call) Return(_a0 *milvuspb.ComponentStates, _a1 error) *MockIndexNodeClient_GetComponentStates_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_GetComponentStates_Call) RunAndReturn(run func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) (*milvuspb.ComponentStates, error)) *MockIndexNodeClient_GetComponentStates_Call {
_c.Call.Return(run)
return _c
}
// GetJobStats provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) GetJobStats(ctx context.Context, in *indexpb.GetJobStatsRequest, opts ...grpc.CallOption) (*indexpb.GetJobStatsResponse, 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 *indexpb.GetJobStatsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.GetJobStatsRequest, ...grpc.CallOption) (*indexpb.GetJobStatsResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.GetJobStatsRequest, ...grpc.CallOption) *indexpb.GetJobStatsResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.GetJobStatsResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.GetJobStatsRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_GetJobStats_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetJobStats'
type MockIndexNodeClient_GetJobStats_Call struct {
*mock.Call
}
// GetJobStats is a helper method to define mock.On call
// - ctx context.Context
// - in *indexpb.GetJobStatsRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) GetJobStats(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_GetJobStats_Call {
return &MockIndexNodeClient_GetJobStats_Call{Call: _e.mock.On("GetJobStats",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_GetJobStats_Call) Run(run func(ctx context.Context, in *indexpb.GetJobStatsRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_GetJobStats_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].(*indexpb.GetJobStatsRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_GetJobStats_Call) Return(_a0 *indexpb.GetJobStatsResponse, _a1 error) *MockIndexNodeClient_GetJobStats_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_GetJobStats_Call) RunAndReturn(run func(context.Context, *indexpb.GetJobStatsRequest, ...grpc.CallOption) (*indexpb.GetJobStatsResponse, error)) *MockIndexNodeClient_GetJobStats_Call {
_c.Call.Return(run)
return _c
}
// GetMetrics provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, 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.GetMetricsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) *milvuspb.GetMetricsResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.GetMetricsResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_GetMetrics_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetMetrics'
type MockIndexNodeClient_GetMetrics_Call struct {
*mock.Call
}
// GetMetrics is a helper method to define mock.On call
// - ctx context.Context
// - in *milvuspb.GetMetricsRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_GetMetrics_Call {
return &MockIndexNodeClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_GetMetrics_Call) Run(run func(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_GetMetrics_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.GetMetricsRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_GetMetrics_Call) Return(_a0 *milvuspb.GetMetricsResponse, _a1 error) *MockIndexNodeClient_GetMetrics_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_GetMetrics_Call) RunAndReturn(run func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error)) *MockIndexNodeClient_GetMetrics_Call {
_c.Call.Return(run)
return _c
}
// GetStatisticsChannel provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, 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.StringResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) (*milvuspb.StringResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) *milvuspb.StringResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.StringResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_GetStatisticsChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetStatisticsChannel'
type MockIndexNodeClient_GetStatisticsChannel_Call struct {
*mock.Call
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
// - in *internalpb.GetStatisticsChannelRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_GetStatisticsChannel_Call {
return &MockIndexNodeClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_GetStatisticsChannel_Call) Run(run func(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_GetStatisticsChannel_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].(*internalpb.GetStatisticsChannelRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_GetStatisticsChannel_Call) Return(_a0 *milvuspb.StringResponse, _a1 error) *MockIndexNodeClient_GetStatisticsChannel_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_GetStatisticsChannel_Call) RunAndReturn(run func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) (*milvuspb.StringResponse, error)) *MockIndexNodeClient_GetStatisticsChannel_Call {
_c.Call.Return(run)
return _c
}
// QueryJobs provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) QueryJobs(ctx context.Context, in *indexpb.QueryJobsRequest, opts ...grpc.CallOption) (*indexpb.QueryJobsResponse, 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 *indexpb.QueryJobsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.QueryJobsRequest, ...grpc.CallOption) (*indexpb.QueryJobsResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.QueryJobsRequest, ...grpc.CallOption) *indexpb.QueryJobsResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.QueryJobsResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.QueryJobsRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_QueryJobs_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QueryJobs'
type MockIndexNodeClient_QueryJobs_Call struct {
*mock.Call
}
// QueryJobs is a helper method to define mock.On call
// - ctx context.Context
// - in *indexpb.QueryJobsRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) QueryJobs(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_QueryJobs_Call {
return &MockIndexNodeClient_QueryJobs_Call{Call: _e.mock.On("QueryJobs",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_QueryJobs_Call) Run(run func(ctx context.Context, in *indexpb.QueryJobsRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_QueryJobs_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].(*indexpb.QueryJobsRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_QueryJobs_Call) Return(_a0 *indexpb.QueryJobsResponse, _a1 error) *MockIndexNodeClient_QueryJobs_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_QueryJobs_Call) RunAndReturn(run func(context.Context, *indexpb.QueryJobsRequest, ...grpc.CallOption) (*indexpb.QueryJobsResponse, error)) *MockIndexNodeClient_QueryJobs_Call {
_c.Call.Return(run)
return _c
}
// ShowConfigurations provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) ShowConfigurations(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, 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 *internalpb.ShowConfigurationsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ShowConfigurationsRequest, ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ShowConfigurationsRequest, ...grpc.CallOption) *internalpb.ShowConfigurationsResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*internalpb.ShowConfigurationsResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.ShowConfigurationsRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_ShowConfigurations_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ShowConfigurations'
type MockIndexNodeClient_ShowConfigurations_Call struct {
*mock.Call
}
// ShowConfigurations is a helper method to define mock.On call
// - ctx context.Context
// - in *internalpb.ShowConfigurationsRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_ShowConfigurations_Call {
return &MockIndexNodeClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_ShowConfigurations_Call) Run(run func(ctx context.Context, in *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_ShowConfigurations_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].(*internalpb.ShowConfigurationsRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_ShowConfigurations_Call) Return(_a0 *internalpb.ShowConfigurationsResponse, _a1 error) *MockIndexNodeClient_ShowConfigurations_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_ShowConfigurations_Call) RunAndReturn(run func(context.Context, *internalpb.ShowConfigurationsRequest, ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error)) *MockIndexNodeClient_ShowConfigurations_Call {
_c.Call.Return(run)
return _c
}
// NewMockIndexNodeClient creates a new instance of MockIndexNodeClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockIndexNodeClient(t interface {
mock.TestingT
Cleanup(func())
}) *MockIndexNodeClient {
mock := &MockIndexNodeClient{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,787 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
package mocks
import (
context "context"
commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
grpc "google.golang.org/grpc"
internalpb "github.com/milvus-io/milvus/internal/proto/internalpb"
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
mock "github.com/stretchr/testify/mock"
proxypb "github.com/milvus-io/milvus/internal/proto/proxypb"
)
// MockProxyClient is an autogenerated mock type for the ProxyClient type
type MockProxyClient struct {
mock.Mock
}
type MockProxyClient_Expecter struct {
mock *mock.Mock
}
func (_m *MockProxyClient) EXPECT() *MockProxyClient_Expecter {
return &MockProxyClient_Expecter{mock: &_m.Mock}
}
// Close provides a mock function with given fields:
func (_m *MockProxyClient) Close() error {
ret := _m.Called()
var r0 error
if rf, ok := ret.Get(0).(func() error); ok {
r0 = rf()
} else {
r0 = ret.Error(0)
}
return r0
}
// MockProxyClient_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockProxyClient_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockProxyClient_Expecter) Close() *MockProxyClient_Close_Call {
return &MockProxyClient_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockProxyClient_Close_Call) Run(run func()) *MockProxyClient_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockProxyClient_Close_Call) Return(_a0 error) *MockProxyClient_Close_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockProxyClient_Close_Call) RunAndReturn(run func() error) *MockProxyClient_Close_Call {
_c.Call.Return(run)
return _c
}
// GetComponentStates provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) GetComponentStates(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, 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.ComponentStates
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) (*milvuspb.ComponentStates, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) *milvuspb.ComponentStates); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.ComponentStates)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_GetComponentStates_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetComponentStates'
type MockProxyClient_GetComponentStates_Call struct {
*mock.Call
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
// - in *milvuspb.GetComponentStatesRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_GetComponentStates_Call {
return &MockProxyClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_GetComponentStates_Call) Run(run func(ctx context.Context, in *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption)) *MockProxyClient_GetComponentStates_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.GetComponentStatesRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_GetComponentStates_Call) Return(_a0 *milvuspb.ComponentStates, _a1 error) *MockProxyClient_GetComponentStates_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_GetComponentStates_Call) RunAndReturn(run func(context.Context, *milvuspb.GetComponentStatesRequest, ...grpc.CallOption) (*milvuspb.ComponentStates, error)) *MockProxyClient_GetComponentStates_Call {
_c.Call.Return(run)
return _c
}
// GetDdChannel provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) GetDdChannel(ctx context.Context, in *internalpb.GetDdChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, 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.StringResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetDdChannelRequest, ...grpc.CallOption) (*milvuspb.StringResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetDdChannelRequest, ...grpc.CallOption) *milvuspb.StringResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.StringResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.GetDdChannelRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_GetDdChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetDdChannel'
type MockProxyClient_GetDdChannel_Call struct {
*mock.Call
}
// GetDdChannel is a helper method to define mock.On call
// - ctx context.Context
// - in *internalpb.GetDdChannelRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) GetDdChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_GetDdChannel_Call {
return &MockProxyClient_GetDdChannel_Call{Call: _e.mock.On("GetDdChannel",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_GetDdChannel_Call) Run(run func(ctx context.Context, in *internalpb.GetDdChannelRequest, opts ...grpc.CallOption)) *MockProxyClient_GetDdChannel_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].(*internalpb.GetDdChannelRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_GetDdChannel_Call) Return(_a0 *milvuspb.StringResponse, _a1 error) *MockProxyClient_GetDdChannel_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_GetDdChannel_Call) RunAndReturn(run func(context.Context, *internalpb.GetDdChannelRequest, ...grpc.CallOption) (*milvuspb.StringResponse, error)) *MockProxyClient_GetDdChannel_Call {
_c.Call.Return(run)
return _c
}
// GetProxyMetrics provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) GetProxyMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, 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.GetMetricsResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) *milvuspb.GetMetricsResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.GetMetricsResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_GetProxyMetrics_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetProxyMetrics'
type MockProxyClient_GetProxyMetrics_Call struct {
*mock.Call
}
// GetProxyMetrics is a helper method to define mock.On call
// - ctx context.Context
// - in *milvuspb.GetMetricsRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) GetProxyMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_GetProxyMetrics_Call {
return &MockProxyClient_GetProxyMetrics_Call{Call: _e.mock.On("GetProxyMetrics",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_GetProxyMetrics_Call) Run(run func(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption)) *MockProxyClient_GetProxyMetrics_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.GetMetricsRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_GetProxyMetrics_Call) Return(_a0 *milvuspb.GetMetricsResponse, _a1 error) *MockProxyClient_GetProxyMetrics_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_GetProxyMetrics_Call) RunAndReturn(run func(context.Context, *milvuspb.GetMetricsRequest, ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error)) *MockProxyClient_GetProxyMetrics_Call {
_c.Call.Return(run)
return _c
}
// GetStatisticsChannel provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, 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.StringResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) (*milvuspb.StringResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) *milvuspb.StringResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*milvuspb.StringResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_GetStatisticsChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetStatisticsChannel'
type MockProxyClient_GetStatisticsChannel_Call struct {
*mock.Call
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
// - in *internalpb.GetStatisticsChannelRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_GetStatisticsChannel_Call {
return &MockProxyClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_GetStatisticsChannel_Call) Run(run func(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption)) *MockProxyClient_GetStatisticsChannel_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].(*internalpb.GetStatisticsChannelRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_GetStatisticsChannel_Call) Return(_a0 *milvuspb.StringResponse, _a1 error) *MockProxyClient_GetStatisticsChannel_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_GetStatisticsChannel_Call) RunAndReturn(run func(context.Context, *internalpb.GetStatisticsChannelRequest, ...grpc.CallOption) (*milvuspb.StringResponse, error)) *MockProxyClient_GetStatisticsChannel_Call {
_c.Call.Return(run)
return _c
}
// InvalidateCollectionMetaCache provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest, 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, *proxypb.InvalidateCollMetaCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *proxypb.InvalidateCollMetaCacheRequest, ...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, *proxypb.InvalidateCollMetaCacheRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_InvalidateCollectionMetaCache_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'InvalidateCollectionMetaCache'
type MockProxyClient_InvalidateCollectionMetaCache_Call struct {
*mock.Call
}
// InvalidateCollectionMetaCache is a helper method to define mock.On call
// - ctx context.Context
// - in *proxypb.InvalidateCollMetaCacheRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) InvalidateCollectionMetaCache(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_InvalidateCollectionMetaCache_Call {
return &MockProxyClient_InvalidateCollectionMetaCache_Call{Call: _e.mock.On("InvalidateCollectionMetaCache",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_InvalidateCollectionMetaCache_Call) Run(run func(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest, opts ...grpc.CallOption)) *MockProxyClient_InvalidateCollectionMetaCache_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].(*proxypb.InvalidateCollMetaCacheRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_InvalidateCollectionMetaCache_Call) Return(_a0 *commonpb.Status, _a1 error) *MockProxyClient_InvalidateCollectionMetaCache_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_InvalidateCollectionMetaCache_Call) RunAndReturn(run func(context.Context, *proxypb.InvalidateCollMetaCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockProxyClient_InvalidateCollectionMetaCache_Call {
_c.Call.Return(run)
return _c
}
// InvalidateCredentialCache provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) InvalidateCredentialCache(ctx context.Context, in *proxypb.InvalidateCredCacheRequest, 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, *proxypb.InvalidateCredCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *proxypb.InvalidateCredCacheRequest, ...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, *proxypb.InvalidateCredCacheRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_InvalidateCredentialCache_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'InvalidateCredentialCache'
type MockProxyClient_InvalidateCredentialCache_Call struct {
*mock.Call
}
// InvalidateCredentialCache is a helper method to define mock.On call
// - ctx context.Context
// - in *proxypb.InvalidateCredCacheRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) InvalidateCredentialCache(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_InvalidateCredentialCache_Call {
return &MockProxyClient_InvalidateCredentialCache_Call{Call: _e.mock.On("InvalidateCredentialCache",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_InvalidateCredentialCache_Call) Run(run func(ctx context.Context, in *proxypb.InvalidateCredCacheRequest, opts ...grpc.CallOption)) *MockProxyClient_InvalidateCredentialCache_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].(*proxypb.InvalidateCredCacheRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_InvalidateCredentialCache_Call) Return(_a0 *commonpb.Status, _a1 error) *MockProxyClient_InvalidateCredentialCache_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_InvalidateCredentialCache_Call) RunAndReturn(run func(context.Context, *proxypb.InvalidateCredCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockProxyClient_InvalidateCredentialCache_Call {
_c.Call.Return(run)
return _c
}
// ListClientInfos provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) ListClientInfos(ctx context.Context, in *proxypb.ListClientInfosRequest, opts ...grpc.CallOption) (*proxypb.ListClientInfosResponse, 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 *proxypb.ListClientInfosResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *proxypb.ListClientInfosRequest, ...grpc.CallOption) (*proxypb.ListClientInfosResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *proxypb.ListClientInfosRequest, ...grpc.CallOption) *proxypb.ListClientInfosResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*proxypb.ListClientInfosResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *proxypb.ListClientInfosRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_ListClientInfos_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListClientInfos'
type MockProxyClient_ListClientInfos_Call struct {
*mock.Call
}
// ListClientInfos is a helper method to define mock.On call
// - ctx context.Context
// - in *proxypb.ListClientInfosRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) ListClientInfos(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_ListClientInfos_Call {
return &MockProxyClient_ListClientInfos_Call{Call: _e.mock.On("ListClientInfos",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_ListClientInfos_Call) Run(run func(ctx context.Context, in *proxypb.ListClientInfosRequest, opts ...grpc.CallOption)) *MockProxyClient_ListClientInfos_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].(*proxypb.ListClientInfosRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_ListClientInfos_Call) Return(_a0 *proxypb.ListClientInfosResponse, _a1 error) *MockProxyClient_ListClientInfos_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_ListClientInfos_Call) RunAndReturn(run func(context.Context, *proxypb.ListClientInfosRequest, ...grpc.CallOption) (*proxypb.ListClientInfosResponse, error)) *MockProxyClient_ListClientInfos_Call {
_c.Call.Return(run)
return _c
}
// RefreshPolicyInfoCache provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) RefreshPolicyInfoCache(ctx context.Context, in *proxypb.RefreshPolicyInfoCacheRequest, 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, *proxypb.RefreshPolicyInfoCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *proxypb.RefreshPolicyInfoCacheRequest, ...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, *proxypb.RefreshPolicyInfoCacheRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_RefreshPolicyInfoCache_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RefreshPolicyInfoCache'
type MockProxyClient_RefreshPolicyInfoCache_Call struct {
*mock.Call
}
// RefreshPolicyInfoCache is a helper method to define mock.On call
// - ctx context.Context
// - in *proxypb.RefreshPolicyInfoCacheRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) RefreshPolicyInfoCache(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_RefreshPolicyInfoCache_Call {
return &MockProxyClient_RefreshPolicyInfoCache_Call{Call: _e.mock.On("RefreshPolicyInfoCache",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_RefreshPolicyInfoCache_Call) Run(run func(ctx context.Context, in *proxypb.RefreshPolicyInfoCacheRequest, opts ...grpc.CallOption)) *MockProxyClient_RefreshPolicyInfoCache_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].(*proxypb.RefreshPolicyInfoCacheRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_RefreshPolicyInfoCache_Call) Return(_a0 *commonpb.Status, _a1 error) *MockProxyClient_RefreshPolicyInfoCache_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_RefreshPolicyInfoCache_Call) RunAndReturn(run func(context.Context, *proxypb.RefreshPolicyInfoCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockProxyClient_RefreshPolicyInfoCache_Call {
_c.Call.Return(run)
return _c
}
// SetRates provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) SetRates(ctx context.Context, in *proxypb.SetRatesRequest, 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, *proxypb.SetRatesRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *proxypb.SetRatesRequest, ...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, *proxypb.SetRatesRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_SetRates_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetRates'
type MockProxyClient_SetRates_Call struct {
*mock.Call
}
// SetRates is a helper method to define mock.On call
// - ctx context.Context
// - in *proxypb.SetRatesRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) SetRates(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_SetRates_Call {
return &MockProxyClient_SetRates_Call{Call: _e.mock.On("SetRates",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_SetRates_Call) Run(run func(ctx context.Context, in *proxypb.SetRatesRequest, opts ...grpc.CallOption)) *MockProxyClient_SetRates_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].(*proxypb.SetRatesRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_SetRates_Call) Return(_a0 *commonpb.Status, _a1 error) *MockProxyClient_SetRates_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_SetRates_Call) RunAndReturn(run func(context.Context, *proxypb.SetRatesRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockProxyClient_SetRates_Call {
_c.Call.Return(run)
return _c
}
// UpdateCredentialCache provides a mock function with given fields: ctx, in, opts
func (_m *MockProxyClient) UpdateCredentialCache(ctx context.Context, in *proxypb.UpdateCredCacheRequest, 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, *proxypb.UpdateCredCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *proxypb.UpdateCredCacheRequest, ...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, *proxypb.UpdateCredCacheRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockProxyClient_UpdateCredentialCache_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateCredentialCache'
type MockProxyClient_UpdateCredentialCache_Call struct {
*mock.Call
}
// UpdateCredentialCache is a helper method to define mock.On call
// - ctx context.Context
// - in *proxypb.UpdateCredCacheRequest
// - opts ...grpc.CallOption
func (_e *MockProxyClient_Expecter) UpdateCredentialCache(ctx interface{}, in interface{}, opts ...interface{}) *MockProxyClient_UpdateCredentialCache_Call {
return &MockProxyClient_UpdateCredentialCache_Call{Call: _e.mock.On("UpdateCredentialCache",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockProxyClient_UpdateCredentialCache_Call) Run(run func(ctx context.Context, in *proxypb.UpdateCredCacheRequest, opts ...grpc.CallOption)) *MockProxyClient_UpdateCredentialCache_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].(*proxypb.UpdateCredCacheRequest), variadicArgs...)
})
return _c
}
func (_c *MockProxyClient_UpdateCredentialCache_Call) Return(_a0 *commonpb.Status, _a1 error) *MockProxyClient_UpdateCredentialCache_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockProxyClient_UpdateCredentialCache_Call) RunAndReturn(run func(context.Context, *proxypb.UpdateCredCacheRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockProxyClient_UpdateCredentialCache_Call {
_c.Call.Return(run)
return _c
}
// NewMockProxyClient creates a new instance of MockProxyClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockProxyClient(t interface {
mock.TestingT
Cleanup(func())
}) *MockProxyClient {
mock := &MockProxyClient{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -32,7 +32,7 @@ func TestValidAuth(t *testing.T) {
assert.False(t, res)
// normal metadata
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -65,7 +65,7 @@ func TestAuthenticationInterceptor(t *testing.T) {
assert.Error(t, err)
// mock metacache
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err = InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)

View File

@ -87,7 +87,7 @@ type getChannelsFuncType = func(collectionID UniqueID) (channelInfos, error)
type repackFuncType = func(tsMsgs []msgstream.TsMsg, hashKeys [][]int32) (map[int32]*msgstream.MsgPack, error)
// getDmlChannelsFunc returns a function about how to get dml channels of a collection.
func getDmlChannelsFunc(ctx context.Context, rc types.RootCoord) getChannelsFuncType {
func getDmlChannelsFunc(ctx context.Context, rc types.RootCoordClient) getChannelsFuncType {
return func(collectionID UniqueID) (channelInfos, error) {
req := &milvuspb.DescribeCollectionRequest{
Base: commonpbutil.NewMsgBase(commonpbutil.WithMsgType(commonpb.MsgType_DescribeCollection)),

View File

@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -54,7 +55,7 @@ func Test_getDmlChannelsFunc(t *testing.T) {
t.Run("failed to describe collection", func(t *testing.T) {
ctx := context.Background()
rc := newMockRootCoord()
rc.DescribeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
rc.DescribeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
return nil, errors.New("mock")
}
f := getDmlChannelsFunc(ctx, rc)
@ -65,7 +66,7 @@ func Test_getDmlChannelsFunc(t *testing.T) {
t.Run("error code not success", func(t *testing.T) {
ctx := context.Background()
rc := newMockRootCoord()
rc.DescribeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
rc.DescribeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
return &milvuspb.DescribeCollectionResponse{Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}}, nil
}
f := getDmlChannelsFunc(ctx, rc)
@ -76,7 +77,7 @@ func Test_getDmlChannelsFunc(t *testing.T) {
t.Run("normal case", func(t *testing.T) {
ctx := context.Background()
rc := newMockRootCoord()
rc.DescribeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
rc.DescribeCollectionFunc = func(ctx context.Context, request *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
return &milvuspb.DescribeCollectionResponse{
VirtualChannelNames: []string{"111", "222"},
PhysicalChannelNames: []string{"111", "111"},

View File

@ -20,6 +20,7 @@ import (
"context"
"go.uber.org/atomic"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -34,7 +35,7 @@ import (
)
type DataCoordMock struct {
types.DataCoord
types.DataCoordClient
nodeID typeutil.UniqueID
address string
@ -45,9 +46,9 @@ type DataCoordMock struct {
showConfigurationsFunc showConfigurationsFuncType
statisticsChannel string
timeTickChannel string
checkHealthFunc func(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error)
GetIndexStateFunc func(ctx context.Context, request *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error)
DescribeIndexFunc func(ctx context.Context, request *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error)
checkHealthFunc func(ctx context.Context, req *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error)
GetIndexStateFunc func(ctx context.Context, request *indexpb.GetIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStateResponse, error)
DescribeIndexFunc func(ctx context.Context, request *indexpb.DescribeIndexRequest, opts ...grpc.CallOption) (*indexpb.DescribeIndexResponse, error)
}
func (coord *DataCoordMock) updateState(state commonpb.StateCode) {
@ -62,24 +63,7 @@ func (coord *DataCoordMock) healthy() bool {
return coord.getState() == commonpb.StateCode_Healthy
}
func (coord *DataCoordMock) Init() error {
coord.updateState(commonpb.StateCode_Initializing)
return nil
}
func (coord *DataCoordMock) Start() error {
defer coord.updateState(commonpb.StateCode_Healthy)
return nil
}
func (coord *DataCoordMock) Stop() error {
defer coord.updateState(commonpb.StateCode_Abnormal)
return nil
}
func (coord *DataCoordMock) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (coord *DataCoordMock) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest, opts ...grpc.CallOption) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: coord.nodeID,
@ -92,7 +76,7 @@ func (coord *DataCoordMock) GetComponentStates(ctx context.Context) (*milvuspb.C
}, nil
}
func (coord *DataCoordMock) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (coord *DataCoordMock) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
Value: coord.statisticsChannel,
@ -103,85 +87,85 @@ func (coord *DataCoordMock) Register() error {
return nil
}
func (coord *DataCoordMock) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (coord *DataCoordMock) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
Value: coord.timeTickChannel,
}, nil
}
func (coord *DataCoordMock) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
func (coord *DataCoordMock) Flush(ctx context.Context, req *datapb.FlushRequest, opts ...grpc.CallOption) (*datapb.FlushResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (coord *DataCoordMock) UnsetIsImportingState(context.Context, *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
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) (*commonpb.Status, error) {
func (coord *DataCoordMock) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (coord *DataCoordMock) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) BroadcastAlteredCollection(ctx context.Context, req *datapb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (coord *DataCoordMock) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
func (coord *DataCoordMock) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest, opts ...grpc.CallOption) (*milvuspb.CheckHealthResponse, error) {
if coord.checkHealthFunc != nil {
return coord.checkHealthFunc(ctx, req)
}
return &milvuspb.CheckHealthResponse{IsHealthy: true}, nil
}
func (coord *DataCoordMock) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
func (coord *DataCoordMock) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegmentIDResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
func (coord *DataCoordMock) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest, opts ...grpc.CallOption) (*datapb.GetSegmentStatesResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
func (coord *DataCoordMock) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*datapb.GetInsertBinlogPathsResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (coord *DataCoordMock) GetSegmentInfoChannel(ctx context.Context, in *datapb.GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) {
func (coord *DataCoordMock) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetCollectionStatisticsResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error) {
func (coord *DataCoordMock) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetPartitionStatisticsResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
func (coord *DataCoordMock) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest, opts ...grpc.CallOption) (*datapb.GetSegmentInfoResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
func (coord *DataCoordMock) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest, opts ...grpc.CallOption) (*datapb.GetRecoveryInfoResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
func (coord *DataCoordMock) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*datapb.GetFlushedSegmentsResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegmentsByStatesRequest) (*datapb.GetSegmentsByStatesResponse, error) {
func (coord *DataCoordMock) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegmentsByStatesRequest, opts ...grpc.CallOption) (*datapb.GetSegmentsByStatesResponse, error) {
panic("implement me")
}
func (coord *DataCoordMock) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
func (coord *DataCoordMock) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest, opts ...grpc.CallOption) (*internalpb.ShowConfigurationsResponse, error) {
if !coord.healthy() {
return &internalpb.ShowConfigurationsResponse{
Status: &commonpb.Status{
@ -203,7 +187,7 @@ func (coord *DataCoordMock) ShowConfigurations(ctx context.Context, req *interna
}, nil
}
func (coord *DataCoordMock) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
func (coord *DataCoordMock) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) {
if !coord.healthy() {
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
@ -227,63 +211,63 @@ func (coord *DataCoordMock) GetMetrics(ctx context.Context, req *milvuspb.GetMet
}, nil
}
func (coord *DataCoordMock) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error) {
func (coord *DataCoordMock) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, nil
}
func (coord *DataCoordMock) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
func (coord *DataCoordMock) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) {
return &milvuspb.ManualCompactionResponse{}, nil
}
func (coord *DataCoordMock) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) {
func (coord *DataCoordMock) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) {
return &milvuspb.GetCompactionStateResponse{}, nil
}
func (coord *DataCoordMock) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error) {
func (coord *DataCoordMock) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) {
return &milvuspb.GetCompactionPlansResponse{}, nil
}
func (coord *DataCoordMock) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest) (*datapb.WatchChannelsResponse, error) {
func (coord *DataCoordMock) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest, opts ...grpc.CallOption) (*datapb.WatchChannelsResponse, error) {
return &datapb.WatchChannelsResponse{}, nil
}
func (coord *DataCoordMock) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
func (coord *DataCoordMock) GetFlushState(ctx context.Context, req *datapb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) {
return &milvuspb.GetFlushStateResponse{}, nil
}
func (coord *DataCoordMock) GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error) {
func (coord *DataCoordMock) GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushAllStateResponse, error) {
return &milvuspb.GetFlushAllStateResponse{}, nil
}
func (coord *DataCoordMock) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) {
func (coord *DataCoordMock) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest, opts ...grpc.CallOption) (*datapb.DropVirtualChannelResponse, error) {
return &datapb.DropVirtualChannelResponse{}, nil
}
func (coord *DataCoordMock) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest) (*datapb.SetSegmentStateResponse, error) {
func (coord *DataCoordMock) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest, opts ...grpc.CallOption) (*datapb.SetSegmentStateResponse, error) {
return &datapb.SetSegmentStateResponse{}, nil
}
func (coord *DataCoordMock) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
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) (*commonpb.Status, error) {
func (coord *DataCoordMock) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (coord *DataCoordMock) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) UpdateChannelCheckpoint(ctx context.Context, req *datapb.UpdateChannelCheckpointRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (coord *DataCoordMock) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (coord *DataCoordMock) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
func (coord *DataCoordMock) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return merr.Status(nil), nil
}
func (coord *DataCoordMock) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
func (coord *DataCoordMock) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStateResponse, error) {
return &indexpb.GetIndexStateResponse{
Status: merr.Status(nil),
State: commonpb.IndexState_Finished,
@ -292,21 +276,21 @@ func (coord *DataCoordMock) GetIndexState(ctx context.Context, req *indexpb.GetI
}
// GetSegmentIndexState gets the index state of the segments in the request from RootCoord.
func (coord *DataCoordMock) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
func (coord *DataCoordMock) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest, opts ...grpc.CallOption) (*indexpb.GetSegmentIndexStateResponse, error) {
return &indexpb.GetSegmentIndexStateResponse{
Status: merr.Status(nil),
}, nil
}
// GetIndexInfos gets the index files of the IndexBuildIDs in the request from RootCoordinator.
func (coord *DataCoordMock) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
func (coord *DataCoordMock) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest, opts ...grpc.CallOption) (*indexpb.GetIndexInfoResponse, error) {
return &indexpb.GetIndexInfoResponse{
Status: merr.Status(nil),
}, nil
}
// DescribeIndex describe the index info of the collection.
func (coord *DataCoordMock) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
func (coord *DataCoordMock) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest, opts ...grpc.CallOption) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
Status: merr.Status(nil),
IndexInfos: nil,
@ -314,7 +298,7 @@ func (coord *DataCoordMock) DescribeIndex(ctx context.Context, req *indexpb.Desc
}
// GetIndexStatistics get the statistics of the index.
func (coord *DataCoordMock) GetIndexStatistics(ctx context.Context, req *indexpb.GetIndexStatisticsRequest) (*indexpb.GetIndexStatisticsResponse, error) {
func (coord *DataCoordMock) GetIndexStatistics(ctx context.Context, req *indexpb.GetIndexStatisticsRequest, opts ...grpc.CallOption) (*indexpb.GetIndexStatisticsResponse, error) {
return &indexpb.GetIndexStatisticsResponse{
Status: merr.Status(nil),
IndexInfos: nil,
@ -322,14 +306,18 @@ func (coord *DataCoordMock) GetIndexStatistics(ctx context.Context, req *indexpb
}
// GetIndexBuildProgress get the index building progress by num rows.
func (coord *DataCoordMock) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
func (coord *DataCoordMock) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest, opts ...grpc.CallOption) (*indexpb.GetIndexBuildProgressResponse, error) {
return &indexpb.GetIndexBuildProgressResponse{
Status: merr.Status(nil),
}, nil
}
func (coord *DataCoordMock) Close() error {
return nil
}
func NewDataCoordMock() *DataCoordMock {
return &DataCoordMock{
dc := &DataCoordMock{
nodeID: typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
address: funcutil.GenRandomStr(), // random address
state: atomic.Value{},
@ -337,4 +325,6 @@ func NewDataCoordMock() *DataCoordMock {
statisticsChannel: funcutil.GenRandomStr(),
timeTickChannel: funcutil.GenRandomStr(),
}
dc.updateState(commonpb.StateCode_Healthy)
return dc
}

View File

@ -69,7 +69,7 @@ func (node *Proxy) UpdateStateCode(code commonpb.StateCode) {
}
// GetComponentStates gets the state of Proxy.
func (node *Proxy) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
func (node *Proxy) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
stats := &milvuspb.ComponentStates{
Status: merr.Status(nil),
}
@ -97,7 +97,7 @@ func (node *Proxy) GetComponentStates(ctx context.Context) (*milvuspb.ComponentS
}
// GetStatisticsChannel gets statistics channel of Proxy.
func (node *Proxy) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
func (node *Proxy) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: merr.Status(nil),
Value: "",
@ -1576,7 +1576,7 @@ func (node *Proxy) GetLoadState(ctx context.Context, request *milvuspb.GetLoadSt
// TODO(longjiquan): https://github.com/milvus-io/milvus/issues/21485, Remove `GetComponentStates` after error code
// is ready to distinguish case whether the querycoord is not healthy or the collection is not even loaded.
if statesResp, err := node.queryCoord.GetComponentStates(ctx); err != nil {
if statesResp, err := node.queryCoord.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{}); err != nil {
return getErrResponse(err), nil
} else if statesResp.State == nil || statesResp.State.StateCode != commonpb.StateCode_Healthy {
return getErrResponse(fmt.Errorf("the querycoord server isn't healthy, state: %v", statesResp.State)), nil
@ -4985,3 +4985,10 @@ func (node *Proxy) AllocTimestamp(ctx context.Context, req *milvuspb.AllocTimest
Timestamp: ts,
}, nil
}
func (node *Proxy) GetVersion(ctx context.Context, request *milvuspb.GetVersionRequest) (*milvuspb.GetVersionResponse, error) {
// TODO implement me
return &milvuspb.GetVersionResponse{
Status: merr.Status(nil),
}, nil
}

View File

@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"google.golang.org/grpc"
"google.golang.org/grpc/metadata"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
@ -73,7 +74,7 @@ func TestProxy_CheckHealth(t *testing.T) {
})
t.Run("proxy health check is ok", func(t *testing.T) {
qc := &mocks.MockQueryCoord{}
qc := &mocks.MockQueryCoordClient{}
qc.EXPECT().CheckHealth(mock.Anything, mock.Anything).Return(&milvuspb.CheckHealthResponse{IsHealthy: true}, nil)
node := &Proxy{
rootCoord: NewRootCoordMock(),
@ -93,6 +94,7 @@ func TestProxy_CheckHealth(t *testing.T) {
t.Run("proxy health check is fail", func(t *testing.T) {
checkHealthFunc1 := func(ctx context.Context,
req *milvuspb.CheckHealthRequest,
opts ...grpc.CallOption,
) (*milvuspb.CheckHealthResponse, error) {
return &milvuspb.CheckHealthResponse{
IsHealthy: false,
@ -103,7 +105,7 @@ func TestProxy_CheckHealth(t *testing.T) {
dataCoordMock := NewDataCoordMock()
dataCoordMock.checkHealthFunc = checkHealthFunc1
qc := &mocks.MockQueryCoord{}
qc := &mocks.MockQueryCoordClient{}
qc.EXPECT().CheckHealth(mock.Anything, mock.Anything).Return(nil, errors.New("test"))
node := &Proxy{
session: &sessionutil.Session{ServerID: 1},
@ -123,7 +125,7 @@ func TestProxy_CheckHealth(t *testing.T) {
})
t.Run("check quota state", func(t *testing.T) {
qc := &mocks.MockQueryCoord{}
qc := &mocks.MockQueryCoordClient{}
qc.EXPECT().CheckHealth(mock.Anything, mock.Anything).Return(&milvuspb.CheckHealthResponse{IsHealthy: true}, nil)
node := &Proxy{
rootCoord: NewRootCoordMock(),
@ -175,7 +177,7 @@ func TestProxyRenameCollection(t *testing.T) {
})
t.Run("rename fail", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("RenameCollection", mock.Anything, mock.Anything).
Return(nil, errors.New("fail"))
node := &Proxy{
@ -191,7 +193,7 @@ func TestProxyRenameCollection(t *testing.T) {
})
t.Run("rename ok", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("RenameCollection", mock.Anything, mock.Anything).
Return(merr.Status(nil), nil)
node := &Proxy{
@ -216,7 +218,7 @@ func TestProxy_ResourceGroup(t *testing.T) {
node.multiRateLimiter = NewMultiRateLimiter()
node.stateCode.Store(commonpb.StateCode_Healthy)
qc := mocks.NewMockQueryCoord(t)
qc := mocks.NewMockQueryCoordClient(t)
node.SetQueryCoordClient(qc)
tsoAllocatorIns := newMockTsoAllocator()
@ -308,7 +310,7 @@ func TestProxy_InvalidResourceGroupName(t *testing.T) {
node.multiRateLimiter = NewMultiRateLimiter()
node.stateCode.Store(commonpb.StateCode_Healthy)
qc := mocks.NewMockQueryCoord(t)
qc := mocks.NewMockQueryCoordClient(t)
node.SetQueryCoordClient(qc)
qc.EXPECT().DropResourceGroup(mock.Anything, mock.Anything).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
@ -406,14 +408,14 @@ func TestProxy_FlushAll_DbCollection(t *testing.T) {
err = node.sched.Start()
assert.NoError(t, err)
defer node.sched.Close()
node.dataCoord = mocks.NewMockDataCoord(t)
node.rootCoord = mocks.NewRootCoord(t)
node.dataCoord = mocks.NewMockDataCoordClient(t)
node.rootCoord = mocks.NewMockRootCoordClient(t)
successStatus := &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
node.dataCoord.(*mocks.MockDataCoord).EXPECT().Flush(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().Flush(mock.Anything, mock.Anything).
Return(&datapb.FlushResponse{Status: successStatus}, nil).Maybe()
node.rootCoord.(*mocks.RootCoord).EXPECT().ShowCollections(mock.Anything, mock.Anything).
node.rootCoord.(*mocks.MockRootCoordClient).EXPECT().ShowCollections(mock.Anything, mock.Anything).
Return(&milvuspb.ShowCollectionsResponse{Status: successStatus, CollectionNames: []string{"col-0"}}, nil).Maybe()
node.rootCoord.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
node.rootCoord.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{Status: successStatus, DbNames: []string{"default"}}, nil).Maybe()
t.Run(test.testName, func(t *testing.T) {
@ -446,8 +448,8 @@ func TestProxy_FlushAll(t *testing.T) {
err = node.sched.Start()
assert.NoError(t, err)
defer node.sched.Close()
node.dataCoord = mocks.NewMockDataCoord(t)
node.rootCoord = mocks.NewRootCoord(t)
node.dataCoord = mocks.NewMockDataCoordClient(t)
node.rootCoord = mocks.NewMockRootCoordClient(t)
cacheBak := globalMetaCache
defer func() { globalMetaCache = cacheBak }()
@ -467,11 +469,11 @@ func TestProxy_FlushAll(t *testing.T) {
globalMetaCache = cache
successStatus := &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
node.dataCoord.(*mocks.MockDataCoord).EXPECT().Flush(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().Flush(mock.Anything, mock.Anything).
Return(&datapb.FlushResponse{Status: successStatus}, nil).Maybe()
node.rootCoord.(*mocks.RootCoord).EXPECT().ShowCollections(mock.Anything, mock.Anything).
node.rootCoord.(*mocks.MockRootCoordClient).EXPECT().ShowCollections(mock.Anything, mock.Anything).
Return(&milvuspb.ShowCollectionsResponse{Status: successStatus, CollectionNames: []string{"col-0"}}, nil).Maybe()
node.rootCoord.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
node.rootCoord.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{Status: successStatus, DbNames: []string{"default"}}, nil).Maybe()
t.Run("FlushAll", func(t *testing.T) {
@ -505,8 +507,8 @@ func TestProxy_FlushAll(t *testing.T) {
})
t.Run("FlushAll failed, DataCoord flush failed", func(t *testing.T) {
node.dataCoord.(*mocks.MockDataCoord).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoord).EXPECT().Flush(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().Flush(mock.Anything, mock.Anything).
Return(&datapb.FlushResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -519,10 +521,10 @@ func TestProxy_FlushAll(t *testing.T) {
})
t.Run("FlushAll failed, RootCoord showCollections failed", func(t *testing.T) {
node.rootCoord.(*mocks.RootCoord).ExpectedCalls = nil
node.rootCoord.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
node.rootCoord.(*mocks.MockRootCoordClient).ExpectedCalls = nil
node.rootCoord.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{Status: successStatus, DbNames: []string{"default"}}, nil).Maybe()
node.rootCoord.(*mocks.RootCoord).EXPECT().ShowCollections(mock.Anything, mock.Anything).
node.rootCoord.(*mocks.MockRootCoordClient).EXPECT().ShowCollections(mock.Anything, mock.Anything).
Return(&milvuspb.ShowCollectionsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -535,8 +537,8 @@ func TestProxy_FlushAll(t *testing.T) {
})
t.Run("FlushAll failed, RootCoord showCollections failed", func(t *testing.T) {
node.rootCoord.(*mocks.RootCoord).ExpectedCalls = nil
node.rootCoord.(*mocks.RootCoord).EXPECT().ListDatabases(mock.Anything, mock.Anything).
node.rootCoord.(*mocks.MockRootCoordClient).ExpectedCalls = nil
node.rootCoord.(*mocks.MockRootCoordClient).EXPECT().ListDatabases(mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -559,12 +561,12 @@ func TestProxy_GetFlushAllState(t *testing.T) {
node.tsoAllocator = &timestampAllocator{
tso: newMockTimestampAllocatorInterface(),
}
node.dataCoord = mocks.NewMockDataCoord(t)
node.rootCoord = mocks.NewRootCoord(t)
node.dataCoord = mocks.NewMockDataCoordClient(t)
node.rootCoord = mocks.NewMockRootCoordClient(t)
// set expectations
successStatus := &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
node.dataCoord.(*mocks.MockDataCoord).EXPECT().GetFlushAllState(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().GetFlushAllState(mock.Anything, mock.Anything).
Return(&milvuspb.GetFlushAllStateResponse{Status: successStatus}, nil).Maybe()
t.Run("GetFlushAllState success", func(t *testing.T) {
@ -582,8 +584,8 @@ func TestProxy_GetFlushAllState(t *testing.T) {
})
t.Run("DataCoord GetFlushAllState failed", func(t *testing.T) {
node.dataCoord.(*mocks.MockDataCoord).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoord).EXPECT().GetFlushAllState(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().GetFlushAllState(mock.Anything, mock.Anything).
Return(&milvuspb.GetFlushAllStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -606,12 +608,12 @@ func TestProxy_GetFlushState(t *testing.T) {
node.tsoAllocator = &timestampAllocator{
tso: newMockTimestampAllocatorInterface(),
}
node.dataCoord = mocks.NewMockDataCoord(t)
node.rootCoord = mocks.NewRootCoord(t)
node.dataCoord = mocks.NewMockDataCoordClient(t)
node.rootCoord = mocks.NewMockRootCoordClient(t)
// set expectations
successStatus := &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
node.dataCoord.(*mocks.MockDataCoord).EXPECT().GetFlushState(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().GetFlushState(mock.Anything, mock.Anything, mock.Anything).
Return(&milvuspb.GetFlushStateResponse{Status: successStatus}, nil).Maybe()
t.Run("GetFlushState success", func(t *testing.T) {
@ -653,8 +655,8 @@ func TestProxy_GetFlushState(t *testing.T) {
})
t.Run("DataCoord GetFlushState failed", func(t *testing.T) {
node.dataCoord.(*mocks.MockDataCoord).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoord).EXPECT().GetFlushState(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().GetFlushState(mock.Anything, mock.Anything, mock.Anything).
Return(&milvuspb.GetFlushStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
@ -667,8 +669,8 @@ func TestProxy_GetFlushState(t *testing.T) {
})
t.Run("GetFlushState return error", func(t *testing.T) {
node.dataCoord.(*mocks.MockDataCoord).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoord).EXPECT().GetFlushState(mock.Anything, mock.Anything).
node.dataCoord.(*mocks.MockDataCoordClient).ExpectedCalls = nil
node.dataCoord.(*mocks.MockDataCoordClient).EXPECT().GetFlushState(mock.Anything, mock.Anything, mock.Anything).
Return(nil, errors.New("fake error"))
resp, err := node.GetFlushState(ctx, &milvuspb.GetFlushStateRequest{})
assert.NoError(t, err)
@ -686,8 +688,8 @@ func TestProxy_GetReplicas(t *testing.T) {
node.tsoAllocator = &timestampAllocator{
tso: newMockTimestampAllocatorInterface(),
}
mockQC := mocks.NewMockQueryCoord(t)
mockRC := mocks.NewRootCoord(t)
mockQC := mocks.NewMockQueryCoordClient(t)
mockRC := mocks.NewMockRootCoordClient(t)
node.queryCoord = mockQC
node.rootCoord = mockRC
@ -735,7 +737,7 @@ func TestProxy_Connect(t *testing.T) {
})
t.Run("failed to list database", func(t *testing.T) {
r := mocks.NewRootCoord(t)
r := mocks.NewMockRootCoordClient(t)
r.On("ListDatabases",
mock.Anything,
mock.Anything,
@ -750,7 +752,7 @@ func TestProxy_Connect(t *testing.T) {
})
t.Run("list database error", func(t *testing.T) {
r := mocks.NewRootCoord(t)
r := mocks.NewMockRootCoordClient(t)
r.On("ListDatabases",
mock.Anything,
mock.Anything,
@ -772,7 +774,7 @@ func TestProxy_Connect(t *testing.T) {
})
ctx := metadata.NewIncomingContext(context.TODO(), md)
r := mocks.NewRootCoord(t)
r := mocks.NewMockRootCoordClient(t)
r.On("ListDatabases",
mock.Anything,
mock.Anything,
@ -795,7 +797,7 @@ func TestProxy_Connect(t *testing.T) {
})
ctx := metadata.NewIncomingContext(context.TODO(), md)
r := mocks.NewRootCoord(t)
r := mocks.NewMockRootCoordClient(t)
r.On("ListDatabases",
mock.Anything,
mock.Anything,
@ -826,7 +828,7 @@ func TestProxy_Connect(t *testing.T) {
})
ctx := metadata.NewIncomingContext(context.TODO(), md)
r := mocks.NewRootCoord(t)
r := mocks.NewMockRootCoordClient(t)
r.On("ListDatabases",
mock.Anything,
mock.Anything,
@ -908,7 +910,7 @@ func TestProxyCreateDatabase(t *testing.T) {
defer node.sched.Close()
t.Run("create database fail", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("CreateDatabase", mock.Anything, mock.Anything).
Return(nil, errors.New("fail"))
node.rootCoord = rc
@ -919,7 +921,7 @@ func TestProxyCreateDatabase(t *testing.T) {
})
t.Run("create database ok", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("CreateDatabase", mock.Anything, mock.Anything).
Return(merr.Status(nil), nil)
node.rootCoord = rc
@ -962,7 +964,7 @@ func TestProxyDropDatabase(t *testing.T) {
defer node.sched.Close()
t.Run("drop database fail", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("DropDatabase", mock.Anything, mock.Anything).
Return(nil, errors.New("fail"))
node.rootCoord = rc
@ -973,7 +975,7 @@ func TestProxyDropDatabase(t *testing.T) {
})
t.Run("drop database ok", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("DropDatabase", mock.Anything, mock.Anything).
Return(merr.Status(nil), nil)
node.rootCoord = rc
@ -1016,7 +1018,7 @@ func TestProxyListDatabase(t *testing.T) {
defer node.sched.Close()
t.Run("list database fail", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("ListDatabases", mock.Anything, mock.Anything).
Return(nil, errors.New("fail"))
node.rootCoord = rc
@ -1027,7 +1029,7 @@ func TestProxyListDatabase(t *testing.T) {
})
t.Run("list database ok", func(t *testing.T) {
rc := mocks.NewRootCoord(t)
rc := mocks.NewMockRootCoordClient(t)
rc.On("ListDatabases", mock.Anything, mock.Anything).
Return(&milvuspb.ListDatabasesResponse{
Status: merr.Status(nil),

View File

@ -19,6 +19,8 @@ package proxy
import (
"context"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
)
@ -32,5 +34,5 @@ type tsoAllocator interface {
//
//go:generate mockery --name=timestampAllocatorInterface --filename=mock_tso_test.go --outpkg=proxy --output=. --inpackage --structname=mockTimestampAllocator --with-expecter
type timestampAllocatorInterface interface {
AllocTimestamp(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error)
AllocTimestamp(ctx context.Context, req *rootcoordpb.AllocTimestampRequest, opts ...grpc.CallOption) (*rootcoordpb.AllocTimestampResponse, error)
}

View File

@ -32,7 +32,7 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type executeFunc func(context.Context, UniqueID, types.QueryNode, ...string) error
type executeFunc func(context.Context, UniqueID, types.QueryNodeClient, ...string) error
type ChannelWorkload struct {
db string

View File

@ -41,9 +41,9 @@ import (
type LBPolicySuite struct {
suite.Suite
rc types.RootCoord
qc *mocks.MockQueryCoord
qn *mocks.MockQueryNode
rc types.RootCoordClient
qc *mocks.MockQueryCoordClient
qn *mocks.MockQueryNodeClient
mgr *MockShardClientManager
lbBalancer *MockLBBalancer
@ -65,7 +65,7 @@ func (s *LBPolicySuite) SetupTest() {
s.nodes = []int64{1, 2, 3, 4, 5}
s.channels = []string{"channel1", "channel2"}
successStatus := commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
qc := mocks.NewMockQueryCoord(s.T())
qc := mocks.NewMockQueryCoordClient(s.T())
qc.EXPECT().LoadCollection(mock.Anything, mock.Anything).Return(&successStatus, nil)
qc.EXPECT().GetShardLeaders(mock.Anything, mock.Anything).Return(&querypb.GetShardLeadersResponse{
@ -90,11 +90,9 @@ func (s *LBPolicySuite) SetupTest() {
s.qc = qc
s.rc = NewRootCoordMock()
s.rc.Start()
s.qn = mocks.NewMockQueryNode(s.T())
s.qn.EXPECT().GetAddress().Return("localhost").Maybe()
s.qn.EXPECT().GetComponentStates(mock.Anything).Return(nil, nil).Maybe()
s.qn = mocks.NewMockQueryNodeClient(s.T())
s.qn.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
s.mgr = NewMockShardClientManager(s.T())
s.mgr.EXPECT().UpdateShardLeaders(mock.Anything, mock.Anything).Return(nil).Maybe()
@ -250,7 +248,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
channel: s.channels[0],
shardLeaders: s.nodes,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
return nil
},
retryTimes: 1,
@ -267,7 +265,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
channel: s.channels[0],
shardLeaders: s.nodes,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
return nil
},
retryTimes: 1,
@ -287,7 +285,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
channel: s.channels[0],
shardLeaders: s.nodes,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
return nil
},
retryTimes: 1,
@ -305,7 +303,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
channel: s.channels[0],
shardLeaders: s.nodes,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
return nil
},
retryTimes: 2,
@ -326,7 +324,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
channel: s.channels[0],
shardLeaders: s.nodes,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
counter++
if counter == 1 {
return errors.New("fake error")
@ -341,8 +339,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
s.mgr.ExpectedCalls = nil
s.mgr.EXPECT().GetClient(mock.Anything, mock.Anything).Return(s.qn, nil)
s.lbBalancer.EXPECT().CancelWorkload(mock.Anything, mock.Anything)
s.qn.EXPECT().GetAddress().Return("localhost").Maybe()
s.qn.EXPECT().GetComponentStates(mock.Anything).Return(nil, nil).Maybe()
s.qn.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
s.qn.EXPECT().Search(mock.Anything, mock.Anything).Return(nil, context.Canceled).Times(1)
s.qn.EXPECT().Search(mock.Anything, mock.Anything).Return(nil, context.DeadlineExceeded)
err = s.lbPolicy.ExecuteWithRetry(ctx, ChannelWorkload{
@ -352,7 +349,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
channel: s.channels[0],
shardLeaders: s.nodes,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
_, err := qn.Search(ctx, nil)
return err
},
@ -373,7 +370,7 @@ func (s *LBPolicySuite) TestExecute() {
collectionName: s.collectionName,
collectionID: s.collectionID,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
return nil
},
})
@ -386,7 +383,7 @@ func (s *LBPolicySuite) TestExecute() {
collectionName: s.collectionName,
collectionID: s.collectionID,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
// succeed in first execute
if counter.Add(1) == 1 {
return nil
@ -407,7 +404,7 @@ func (s *LBPolicySuite) TestExecute() {
collectionName: s.collectionName,
collectionID: s.collectionID,
nq: 1,
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNode, s ...string) error {
exec: func(ctx context.Context, ui UniqueID, qn types.QueryNodeClient, s ...string) error {
return nil
},
})

View File

@ -28,6 +28,7 @@ import (
"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/internal/proto/internalpb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
@ -226,7 +227,7 @@ func (b *LookAsideBalancer) checkQueryNodeHealthLoop(ctx context.Context) {
return struct{}{}, nil
}
resp, err := qn.GetComponentStates(ctx)
resp, err := qn.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
if err != nil {
if b.trySetQueryNodeUnReachable(node, err) {
log.Warn("get component status failed, set node unreachable", zap.Int64("node", node), zap.Error(err))

View File

@ -46,9 +46,9 @@ func (suite *LookAsideBalancerSuite) SetupTest() {
suite.balancer = NewLookAsideBalancer(suite.clientMgr)
suite.balancer.Start(context.Background())
qn := mocks.NewMockQueryNode(suite.T())
qn := mocks.NewMockQueryNodeClient(suite.T())
suite.clientMgr.EXPECT().GetClient(mock.Anything, int64(1)).Return(qn, nil).Maybe()
qn.EXPECT().GetComponentStates(mock.Anything).Return(nil, errors.New("fake error")).Maybe()
qn.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(nil, errors.New("fake error")).Maybe()
}
func (suite *LookAsideBalancerSuite) TearDownTest() {
@ -306,9 +306,9 @@ func (suite *LookAsideBalancerSuite) TestCancelWorkload() {
}
func (suite *LookAsideBalancerSuite) TestCheckHealthLoop() {
qn2 := mocks.NewMockQueryNode(suite.T())
qn2 := mocks.NewMockQueryNodeClient(suite.T())
suite.clientMgr.EXPECT().GetClient(mock.Anything, int64(2)).Return(qn2, nil)
qn2.EXPECT().GetComponentStates(mock.Anything).Return(&milvuspb.ComponentStates{
qn2.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(&milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Healthy,
},
@ -336,15 +336,15 @@ func (suite *LookAsideBalancerSuite) TestCheckHealthLoop() {
func (suite *LookAsideBalancerSuite) TestNodeRecover() {
// mock qn down for a while and then recover
qn3 := mocks.NewMockQueryNode(suite.T())
qn3 := mocks.NewMockQueryNodeClient(suite.T())
suite.clientMgr.EXPECT().GetClient(mock.Anything, int64(3)).Return(qn3, nil)
qn3.EXPECT().GetComponentStates(mock.Anything).Return(&milvuspb.ComponentStates{
qn3.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(&milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Abnormal,
},
}, nil).Times(3)
qn3.EXPECT().GetComponentStates(mock.Anything).Return(&milvuspb.ComponentStates{
qn3.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(&milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Healthy,
},
@ -364,9 +364,9 @@ func (suite *LookAsideBalancerSuite) TestNodeOffline() {
Params.Save(Params.CommonCfg.SessionTTL.Key, "10")
Params.Save(Params.ProxyCfg.HealthCheckTimeout.Key, "1000")
// mock qn down for a while and then recover
qn3 := mocks.NewMockQueryNode(suite.T())
qn3 := mocks.NewMockQueryNodeClient(suite.T())
suite.clientMgr.EXPECT().GetClient(mock.Anything, int64(3)).Return(qn3, nil)
qn3.EXPECT().GetComponentStates(mock.Anything).Return(&milvuspb.ComponentStates{
qn3.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(&milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
StateCode: commonpb.StateCode_Abnormal,
},

View File

@ -192,8 +192,8 @@ var _ Cache = (*MetaCache)(nil)
// MetaCache implements Cache, provides collection meta cache based on internal RootCoord
type MetaCache struct {
rootCoord types.RootCoord
queryCoord types.QueryCoord
rootCoord types.RootCoordClient
queryCoord types.QueryCoordClient
collInfo map[string]map[string]*collectionInfo // database -> collection -> collection_info
credMap map[string]*internalpb.CredentialInfo // cache for credential, lazy load
@ -209,7 +209,7 @@ type MetaCache struct {
var globalMetaCache Cache
// InitMetaCache initializes globalMetaCache
func InitMetaCache(ctx context.Context, rootCoord types.RootCoord, queryCoord types.QueryCoord, shardMgr shardClientMgr) error {
func InitMetaCache(ctx context.Context, rootCoord types.RootCoordClient, queryCoord types.QueryCoordClient, shardMgr shardClientMgr) error {
var err error
globalMetaCache, err = NewMetaCache(rootCoord, queryCoord, shardMgr)
if err != nil {
@ -229,7 +229,7 @@ func InitMetaCache(ctx context.Context, rootCoord types.RootCoord, queryCoord ty
}
// NewMetaCache creates a MetaCache with provided RootCoord and QueryNode
func NewMetaCache(rootCoord types.RootCoord, queryCoord types.QueryCoord, shardMgr shardClientMgr) (*MetaCache, error) {
func NewMetaCache(rootCoord types.RootCoordClient, queryCoord types.QueryCoordClient, shardMgr shardClientMgr) (*MetaCache, error) {
return &MetaCache{
rootCoord: rootCoord,
queryCoord: queryCoord,

View File

@ -29,6 +29,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
uatomic "go.uber.org/atomic"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
@ -48,7 +49,7 @@ import (
var dbName = GetCurDBNameFromContextOrDefault(context.Background())
type MockRootCoordClientInterface struct {
types.RootCoord
types.RootCoordClient
Error bool
AccessCount int32
@ -64,7 +65,7 @@ func (m *MockRootCoordClientInterface) GetAccessCount() int {
return int(ret)
}
func (m *MockRootCoordClientInterface) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
func (m *MockRootCoordClientInterface) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error) {
if m.Error {
return nil, errors.New("mocked error")
}
@ -106,7 +107,7 @@ func (m *MockRootCoordClientInterface) ShowPartitions(ctx context.Context, in *m
}, nil
}
func (m *MockRootCoordClientInterface) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
func (m *MockRootCoordClientInterface) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
if m.Error {
return nil, errors.New("mocked error")
}
@ -151,7 +152,7 @@ func (m *MockRootCoordClientInterface) DescribeCollection(ctx context.Context, i
}, nil
}
func (m *MockRootCoordClientInterface) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest) (*rootcoordpb.GetCredentialResponse, error) {
func (m *MockRootCoordClientInterface) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest, opts ...grpc.CallOption) (*rootcoordpb.GetCredentialResponse, error) {
if m.Error {
return nil, errors.New("mocked error")
}
@ -169,7 +170,7 @@ func (m *MockRootCoordClientInterface) GetCredential(ctx context.Context, req *r
return nil, err
}
func (m *MockRootCoordClientInterface) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) {
func (m *MockRootCoordClientInterface) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest, opts ...grpc.CallOption) (*milvuspb.ListCredUsersResponse, error) {
if m.Error {
return nil, errors.New("mocked error")
}
@ -180,7 +181,7 @@ func (m *MockRootCoordClientInterface) ListCredUsers(ctx context.Context, req *m
}, nil
}
func (m *MockRootCoordClientInterface) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) {
func (m *MockRootCoordClientInterface) ListPolicy(ctx context.Context, in *internalpb.ListPolicyRequest, opts ...grpc.CallOption) (*internalpb.ListPolicyResponse, error) {
if m.listPolicy != nil {
return m.listPolicy(ctx, in)
}
@ -193,7 +194,7 @@ func (m *MockRootCoordClientInterface) ListPolicy(ctx context.Context, in *inter
func TestMetaCache_GetCollection(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -243,7 +244,7 @@ func TestMetaCache_GetCollection(t *testing.T) {
func TestMetaCache_GetBasicCollectionInfo(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -277,7 +278,7 @@ func TestMetaCache_GetBasicCollectionInfo(t *testing.T) {
func TestMetaCache_GetCollectionName(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -327,7 +328,7 @@ func TestMetaCache_GetCollectionName(t *testing.T) {
func TestMetaCache_GetCollectionFailure(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -360,7 +361,7 @@ func TestMetaCache_GetCollectionFailure(t *testing.T) {
func TestMetaCache_GetNonExistCollection(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -376,7 +377,7 @@ func TestMetaCache_GetNonExistCollection(t *testing.T) {
func TestMetaCache_GetPartitionID(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -398,7 +399,7 @@ func TestMetaCache_GetPartitionID(t *testing.T) {
func TestMetaCache_ConcurrentTest1(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -452,7 +453,7 @@ func TestMetaCache_ConcurrentTest1(t *testing.T) {
func TestMetaCache_GetPartitionError(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, mgr)
assert.NoError(t, err)
@ -485,16 +486,11 @@ func TestMetaCache_GetShards(t *testing.T) {
)
rootCoord := &MockRootCoordClientInterface{}
qc := getQueryCoord()
qc.EXPECT().Init().Return(nil)
qc := getQueryCoordClient()
shardMgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, qc, shardMgr)
require.Nil(t, err)
qc.Init()
qc.Start()
defer qc.Stop()
t.Run("No collection in meta cache", func(t *testing.T) {
shards, err := globalMetaCache.GetShards(ctx, true, dbName, "non-exists", 0)
assert.Error(t, err)
@ -560,16 +556,11 @@ func TestMetaCache_ClearShards(t *testing.T) {
)
rootCoord := &MockRootCoordClientInterface{}
qc := getQueryCoord()
qc.EXPECT().Init().Return(nil)
qc := getQueryCoordClient()
mgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, qc, mgr)
require.Nil(t, err)
qc.Init()
qc.Start()
defer qc.Stop()
t.Run("Clear with no collection info", func(t *testing.T) {
globalMetaCache.DeprecateShardCache(dbName, "collection_not_exist")
})
@ -614,7 +605,7 @@ func TestMetaCache_ClearShards(t *testing.T) {
func TestMetaCache_PolicyInfo(t *testing.T) {
client := &MockRootCoordClientInterface{}
qc := &mocks.MockQueryCoord{}
qc := &mocks.MockQueryCoordClient{}
mgr := newShardClientMgr()
t.Run("InitMetaCache", func(t *testing.T) {
@ -730,7 +721,7 @@ func TestMetaCache_PolicyInfo(t *testing.T) {
func TestMetaCache_RemoveCollection(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
shardMgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, shardMgr)
assert.NoError(t, err)
@ -779,7 +770,7 @@ func TestMetaCache_ExpireShardLeaderCache(t *testing.T) {
ctx := context.Background()
rootCoord := &MockRootCoordClientInterface{}
queryCoord := &mocks.MockQueryCoord{}
queryCoord := &mocks.MockQueryCoordClient{}
shardMgr := newShardClientMgr()
err := InitMetaCache(ctx, rootCoord, queryCoord, shardMgr)
assert.NoError(t, err)

View File

@ -38,16 +38,11 @@ func TestProxy_metrics(t *testing.T) {
ctx := context.Background()
rc := NewRootCoordMock()
rc.Start()
defer rc.Stop()
qc := getQueryCoord()
qc.Start()
defer qc.Stop()
defer rc.Close()
qc := getQueryCoordClient()
dc := NewDataCoordMock()
dc.Start()
defer dc.Stop()
defer dc.Close()
proxy := &Proxy{
rootCoord: rc,

Some files were not shown because too many files have changed in this diff Show More