milvus/internal/querynodev2/pipeline/manager_test.go
yah01 081572d31c
Refactor QueryNode (#21625)
Signed-off-by: yah01 <yang.cen@zilliz.com>
Co-authored-by: Congqi Xia <congqi.xia@zilliz.com>
Co-authored-by: aoiasd <zhicheng.yue@zilliz.com>
2023-03-27 00:42:00 +08:00

117 lines
3.9 KiB
Go

// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package pipeline
import (
"testing"
"github.com/milvus-io/milvus-proto/go-api/msgpb"
"github.com/milvus-io/milvus/internal/mq/msgdispatcher"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
"github.com/milvus-io/milvus/internal/querynodev2/segments"
"github.com/milvus-io/milvus/internal/querynodev2/tsafe"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
)
type PipelineManagerTestSuite struct {
suite.Suite
//data
collectionID int64
channel string
//dependencies
tSafeManager TSafeManager
delegators *typeutil.ConcurrentMap[string, delegator.ShardDelegator]
//mocks
segmentManager *segments.MockSegmentManager
collectionManager *segments.MockCollectionManager
delegator *delegator.MockShardDelegator
msgDispatcher *msgdispatcher.MockClient
msgChan chan *msgstream.MsgPack
}
func (suite *PipelineManagerTestSuite) SetupSuite() {
suite.collectionID = 111
suite.msgChan = make(chan *msgstream.MsgPack, 1)
}
func (suite *PipelineManagerTestSuite) SetupTest() {
paramtable.Init()
//init dependency
// init tsafeManager
suite.tSafeManager = tsafe.NewTSafeReplica()
suite.tSafeManager.Add(suite.channel, 0)
suite.delegators = typeutil.NewConcurrentMap[string, delegator.ShardDelegator]()
//init mock
// init manager
suite.collectionManager = segments.NewMockCollectionManager(suite.T())
suite.segmentManager = segments.NewMockSegmentManager(suite.T())
// init delegator
suite.delegator = delegator.NewMockShardDelegator(suite.T())
suite.delegators.Insert(suite.channel, suite.delegator)
// init mq dispatcher
suite.msgDispatcher = msgdispatcher.NewMockClient(suite.T())
}
func (suite *PipelineManagerTestSuite) TestBasic() {
//init mock
// mock collection manager
suite.collectionManager.EXPECT().Get(suite.collectionID).Return(&segments.Collection{})
// mock mq factory
suite.msgDispatcher.EXPECT().Register(suite.channel, mock.Anything, mqwrapper.SubscriptionPositionUnknown).Return(suite.msgChan, nil)
suite.msgDispatcher.EXPECT().Deregister(suite.channel)
//build manager
manager := &segments.Manager{
Collection: suite.collectionManager,
Segment: suite.segmentManager,
}
pipelineManager := NewManager(manager, suite.tSafeManager, suite.msgDispatcher, suite.delegators)
defer pipelineManager.Close()
//Add pipeline
_, err := pipelineManager.Add(suite.collectionID, suite.channel)
suite.NoError(err)
suite.Equal(1, pipelineManager.Num())
//Get pipeline
pipeline := pipelineManager.Get(suite.channel)
suite.NotNil(pipeline)
//Init Consumer
err = pipeline.ConsumeMsgStream(&msgpb.MsgPosition{})
suite.NoError(err)
//Start pipeline
err = pipelineManager.Start(suite.channel)
suite.NoError(err)
//Remove pipeline
pipelineManager.Remove(suite.channel)
suite.Equal(0, pipelineManager.Num())
}
func TestQueryNodePipelineManager(t *testing.T) {
suite.Run(t, new(PipelineManagerTestSuite))
}