Skip to content

Commit

Permalink
Decoupling client and server API in types interface (milvus-io#27186)
Browse files Browse the repository at this point in the history
Co-authored-by:: aoiasd <zhicheng.yue@zilliz.com>

Signed-off-by: jaime <yun.zhang@zilliz.com>
  • Loading branch information
jaime0815 authored Sep 26, 2023
1 parent 24354b1 commit 7f7c71e
Show file tree
Hide file tree
Showing 170 changed files with 17,853 additions and 7,247 deletions.
9 changes: 9 additions & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
3 changes: 2 additions & 1 deletion internal/allocator/id_allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,15 @@ 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"
)

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),
Expand Down
4 changes: 3 additions & 1 deletion internal/allocator/remote_interface.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
6 changes: 3 additions & 3 deletions internal/datacoord/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
}
}

Expand Down
4 changes: 2 additions & 2 deletions internal/datacoord/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import (

func TestAllocator_Basic(t *testing.T) {
paramtable.Init()
ms := newMockRootCoordService()
ms := newMockRootCoordClient()
allocator := newRootCoordAllocator(ms)
ctx := context.Background()

Expand All @@ -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)
Expand Down
6 changes: 3 additions & 3 deletions internal/datacoord/cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -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))
Expand Down Expand Up @@ -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))
Expand Down
35 changes: 22 additions & 13 deletions internal/datacoord/compaction_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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{
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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)
}
Expand All @@ -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)

Expand Down Expand Up @@ -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)

Expand Down
4 changes: 2 additions & 2 deletions internal/datacoord/coordinator_broker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
}
Expand Down
2 changes: 1 addition & 1 deletion internal/datacoord/index_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Loading

0 comments on commit 7f7c71e

Please sign in to comment.