Skip to content
Snippets Groups Projects
Commit b98b226d authored by Yihao Dai's avatar Yihao Dai Committed by yefu.chen
Browse files

Refactor query node and implement channelRemove, channelAdd


Signed-off-by: default avatarbigsheeper <yihao.dai@zilliz.com>
parent 0971cf65
No related branches found
No related tags found
No related merge requests found
Showing
with 1137 additions and 278 deletions
......@@ -10,16 +10,16 @@ import (
"go.uber.org/zap"
"github.com/zilliztech/milvus-distributed/internal/querynode"
querynodeimp "github.com/zilliztech/milvus-distributed/internal/querynode"
)
func main() {
querynode.Init()
fmt.Println("QueryNodeID is", querynode.Params.QueryNodeID)
querynodeimp.Init()
fmt.Println("QueryNodeID is", querynodeimp.Params.QueryNodeID)
// Creates server.
ctx, cancel := context.WithCancel(context.Background())
svr := querynode.NewQueryNode(ctx, 0)
svr := querynodeimp.NewQueryNode(ctx, 0)
sc := make(chan os.Signal, 1)
signal.Notify(sc,
......
......@@ -17,7 +17,7 @@ import (
"github.com/zilliztech/milvus-distributed/internal/indexnode"
"github.com/zilliztech/milvus-distributed/internal/master"
"github.com/zilliztech/milvus-distributed/internal/proxynode"
"github.com/zilliztech/milvus-distributed/internal/querynode"
querynodeimp "github.com/zilliztech/milvus-distributed/internal/querynode"
"github.com/zilliztech/milvus-distributed/internal/writenode"
)
......@@ -101,11 +101,11 @@ func InitProxy(wg *sync.WaitGroup) {
func InitQueryNode(wg *sync.WaitGroup) {
defer wg.Done()
querynode.Init()
fmt.Println("QueryNodeID is", querynode.Params.QueryNodeID)
querynodeimp.Init()
fmt.Println("QueryNodeID is", querynodeimp.Params.QueryNodeID)
// Creates server.
ctx, cancel := context.WithCancel(context.Background())
svr := querynode.NewQueryNode(ctx, 0)
svr := querynodeimp.NewQueryNode(ctx, 0)
sc := make(chan os.Signal, 1)
signal.Notify(sc,
......
package querynode
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
)
type Client struct {
ctx context.Context
querypb.QueryNodeClient
}
package querynode
import (
"context"
"net"
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
querynodeimp "github.com/zilliztech/milvus-distributed/internal/querynode"
)
type Server struct {
grpcServer *grpc.Server
node querynodeimp.Node
}
func NewServer(ctx context.Context, queryNodeID uint64) *Server {
return &Server{
node: querynodeimp.NewQueryNode(ctx, queryNodeID),
}
}
func (s *Server) StartGrpcServer() {
// TODO: add address
lis, err := net.Listen("tcp", "")
if err != nil {
panic(err)
}
s.grpcServer = grpc.NewServer()
querypb.RegisterQueryNodeServer(s.grpcServer, s)
if err = s.grpcServer.Serve(lis); err != nil {
panic(err)
}
}
func (s *Server) Start() {
go s.StartGrpcServer()
if err := s.node.Start(); err != nil {
panic(err)
}
}
func (s *Server) AddQueryChannel(ctx context.Context, in *querypb.AddQueryChannelsRequest) (*commonpb.Status, error) {
return s.node.AddQueryChannel(ctx, in)
}
func (s *Server) RemoveQueryChannel(ctx context.Context, in *querypb.RemoveQueryChannelsRequest) (*commonpb.Status, error) {
return s.node.RemoveQueryChannel(ctx, in)
}
func (s *Server) WatchDmChannels(ctx context.Context, in *querypb.WatchDmChannelsRequest) (*commonpb.Status, error) {
return s.node.WatchDmChannels(ctx, in)
}
func (s *Server) LoadSegments(ctx context.Context, in *querypb.LoadSegmentRequest) (*commonpb.Status, error) {
return s.node.LoadSegments(ctx, in)
}
func (s *Server) ReleaseSegments(ctx context.Context, in *querypb.ReleaseSegmentRequest) (*commonpb.Status, error) {
return s.node.ReleaseSegments(ctx, in)
}
func (s *Server) GetPartitionState(ctx context.Context, in *querypb.PartitionStatesRequest) (*querypb.PartitionStatesResponse, error) {
return s.node.GetPartitionState(ctx, in)
}
package queryservice
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
)
type Client struct {
ctx context.Context
querypb.QueryServiceClient
}
package queryservice
import (
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
queryServiceImpl "github.com/zilliztech/milvus-distributed/internal/queryservice"
)
type Server struct {
grpcServer *grpc.Server
queryService queryServiceImpl.QueryService
}
func (s *Server) RegisterNode(req querypb.RegisterNodeRequest) (querypb.RegisterNodeResponse, error) {
return s.queryService.RegisterNode(req)
}
func (s *Server) ShowCollections(req querypb.ShowCollectionRequest) (querypb.ShowCollectionResponse, error) {
return s.ShowCollections(req)
}
func (s *Server) LoadCollection(req querypb.LoadCollectionRequest) error {
return s.LoadCollection(req)
}
func (s *Server) ReleaseCollection(req querypb.ReleaseCollectionRequest) error {
return s.ReleaseCollection(req)
}
func (s *Server) ShowPartitions(req querypb.ShowPartitionRequest) (querypb.ShowPartitionResponse, error) {
return s.ShowPartitions(req)
}
func (s *Server) GetPartitionStates(req querypb.PartitionStatesRequest) (querypb.PartitionStatesResponse, error) {
return s.GetPartitionStates(req)
}
func (s *Server) LoadPartitions(req querypb.LoadPartitionRequest) error {
return s.LoadPartitions(req)
}
func (s *Server) ReleasePartitions(req querypb.ReleasePartitionRequest) error {
return s.ReleasePartitions(req)
}
func (s *Server) CreateQueryChannel() (querypb.CreateQueryChannelResponse, error) {
return s.CreateQueryChannel()
}
......@@ -189,9 +189,9 @@ func CreateServer(ctx context.Context) (*Master, error) {
if err != nil {
return nil, err
}
loadIndexClient := client.NewLoadIndexClient(ctx, Params.PulsarAddress, Params.LoadIndexChannelNames)
queryNodeClient := client.NewQueryNodeClient(ctx, Params.PulsarAddress, Params.LoadIndexChannelNames)
m.indexLoadSch = NewIndexLoadScheduler(ctx, loadIndexClient, m.metaTable)
m.indexLoadSch = NewIndexLoadScheduler(ctx, queryNodeClient, m.metaTable)
m.indexBuildSch = NewIndexBuildScheduler(ctx, buildIndexClient, m.metaTable, m.indexLoadSch)
m.flushSch = NewFlushScheduler(ctx, flushClient, m.metaTable, m.indexBuildSch, func() (Timestamp, error) { return m.tsoAllocator.AllocOne() })
......
......@@ -78,7 +78,7 @@ message PartitionStatesResponse {
}
message LoadPartitonRequest {
message LoadPartitionRequest {
internal.MsgBase base = 1;
int64 dbID = 2;
int64 collectionID = 3;
......@@ -98,20 +98,20 @@ message CreateQueryChannelResponse {
}
message AddQueryChannelRequest {
message AddQueryChannelsRequest {
internal.MsgBase base = 1;
string request_channelID = 2;
string result_channelID = 3;
}
message RemoveQueryChannelRequest {
message RemoveQueryChannelsRequest {
internal.MsgBase base = 1;
string request_channelID = 2;
string result_channelID = 3;
}
message WatchDmChannelRequest {
message WatchDmChannelsRequest {
internal.MsgBase base = 1;
repeated string channelIDs = 2;
}
......@@ -135,6 +135,21 @@ message ReleaseSegmentRequest {
repeated int64 fieldIDs = 6;
}
message GetTimeTickChannelResponse {
common.Status status = 1;
string time_tick_channelID = 2;
}
message GetStatsChannelResponse {
common.Status status = 1;
string stats_channelID = 2;
}
message ServiceStatesResponse {
common.Status status = 1;
internal.ServiceStates server_states = 2;
}
service QueryService {
/**
* @brief This method is used to create collection
......@@ -144,17 +159,27 @@ service QueryService {
* @return Status
*/
rpc RegisterNode(RegisterNodeRequest) returns (RegisterNodeResponse) {}
rpc ShowCollections(ShowCollectionRequest) returns (ShowCollectionResponse) {}
rpc LoadCollection(LoadCollectionRequest) returns (common.Status) {}
rpc ReleaseCollection(ReleaseCollectionRequest) returns (common.Status) {}
rpc ShowCollections(ShowCollectionRequest) returns (ShowCollectionResponse) {}
rpc ShowPartitions(ShowPartitionRequest) returns (ShowPartitionResponse) {}
rpc GetPartitionStates(PartitionStatesRequest) returns (PartitionStatesResponse) {}
rpc LoadPartitions(LoadPartitonRequest) returns (common.Status) {}
rpc LoadPartitions(LoadPartitionRequest) returns (common.Status) {}
rpc ReleasePartitions(ReleasePartitionRequest) returns (common.Status) {}
rpc LoadCollection(LoadCollectionRequest) returns (common.Status) {}
rpc ReleaseCollection(ReleaseCollectionRequest) returns (common.Status) {}
rpc CreateQueryChannel(common.Empty ) returns (CreateQueryChannelResponse) {}
rpc GetTimeTickChannel(common.Empty) returns (GetTimeTickChannelResponse) {}
rpc GetStatsChannel(common.Empty) returns (GetStatsChannelResponse) {}
rpc GetPartitionStates(PartitionStatesRequest) returns (PartitionStatesResponse) {}
rpc GetServiceStates(common.Empty) returns (ServiceStatesResponse) {}
}
service QueryNode {
rpc AddQueryChannel(AddQueryChannelsRequest) returns (common.Status) {}
rpc RemoveQueryChannel(RemoveQueryChannelsRequest) returns (common.Status) {}
rpc WatchDmChannels(WatchDmChannelsRequest) returns (common.Status) {}
rpc LoadSegments(LoadSegmentRequest) returns (common.Status) {}
rpc ReleaseSegments(ReleaseSegmentRequest) returns (common.Status) {}
rpc GetPartitionState(PartitionStatesRequest) returns (PartitionStatesResponse) {}
}
This diff is collapsed.
......@@ -12,6 +12,7 @@ import (
"time"
"github.com/opentracing/opentracing-go"
"github.com/uber/jaeger-client-go"
"github.com/uber/jaeger-client-go/config"
"google.golang.org/grpc"
......@@ -71,8 +72,11 @@ func CreateProxy(ctx context.Context) (*Proxy, error) {
Type: "const",
Param: 1,
},
Reporter: &config.ReporterConfig{
LogSpans: true,
},
}
p.tracer, p.closer, err = cfg.NewTracer()
p.tracer, p.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger))
if err != nil {
panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
}
......
package querynodeimp
import (
"context"
"errors"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
queryPb "github.com/zilliztech/milvus-distributed/internal/proto/querypb"
)
func (node *QueryNode) AddQueryChannel(ctx context.Context, in *queryPb.AddQueryChannelsRequest) (*commonpb.Status, error) {
select {
case <-ctx.Done():
errMsg := "context exceeded"
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: errMsg,
}
return status, errors.New(errMsg)
default:
searchStream, ok := node.searchService.searchMsgStream.(*msgstream.PulsarMsgStream)
if !ok {
errMsg := "type assertion failed for search message stream"
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: errMsg,
}
return status, errors.New(errMsg)
}
resultStream, ok := node.searchService.searchResultMsgStream.(*msgstream.PulsarMsgStream)
if !ok {
errMsg := "type assertion failed for search result message stream"
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: errMsg,
}
return status, errors.New(errMsg)
}
// add request channel
pulsarBufSize := Params.SearchPulsarBufSize
consumeChannels := []string{in.RequestChannelID}
consumeSubName := Params.MsgChannelSubName
unmarshalDispatcher := msgstream.NewUnmarshalDispatcher()
searchStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
// add result channel
producerChannels := []string{in.ResultChannelID}
resultStream.CreatePulsarProducers(producerChannels)
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
}
return status, nil
}
}
func (node *QueryNode) RemoveQueryChannel(ctx context.Context, in *queryPb.RemoveQueryChannelsRequest) (*commonpb.Status, error) {
select {
case <-ctx.Done():
errMsg := "context exceeded"
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: errMsg,
}
return status, errors.New(errMsg)
default:
searchStream, ok := node.searchService.searchMsgStream.(*msgstream.PulsarMsgStream)
if !ok {
errMsg := "type assertion failed for search message stream"
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: errMsg,
}
return status, errors.New(errMsg)
}
resultStream, ok := node.searchService.searchResultMsgStream.(*msgstream.PulsarMsgStream)
if !ok {
errMsg := "type assertion failed for search result message stream"
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: errMsg,
}
return status, errors.New(errMsg)
}
// remove request channel
pulsarBufSize := Params.SearchPulsarBufSize
consumeChannels := []string{in.RequestChannelID}
consumeSubName := Params.MsgChannelSubName
unmarshalDispatcher := msgstream.NewUnmarshalDispatcher()
// TODO: searchStream.RemovePulsarConsumers(producerChannels)
searchStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
// remove result channel
producerChannels := []string{in.ResultChannelID}
// TODO: resultStream.RemovePulsarProducer(producerChannels)
resultStream.CreatePulsarProducers(producerChannels)
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
}
return status, nil
}
}
func (node *QueryNode) WatchDmChannels(ctx context.Context, in *queryPb.WatchDmChannelsRequest) (*commonpb.Status, error) {
select {
case <-ctx.Done():
errMsg := "context exceeded"
status := &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: errMsg,
}
return status, errors.New(errMsg)
default:
// TODO: add dmMsgStream reference to dataSyncService
//fgDMMsgStream, ok := node.dataSyncService.dmMsgStream.(*msgstream.PulsarMsgStream)
//if !ok {
// errMsg := "type assertion failed for dm message stream"
// status := &commonpb.Status{
// ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
// Reason: errMsg,
// }
//
// return status, errors.New(errMsg)
//}
//
//// add request channel
//pulsarBufSize := Params.SearchPulsarBufSize
//consumeChannels := in.ChannelIDs
//consumeSubName := Params.MsgChannelSubName
//unmarshalDispatcher := msgstream.NewUnmarshalDispatcher()
//fgDMMsgStream.CreatePulsarConsumers(consumeChannels, consumeSubName, unmarshalDispatcher, pulsarBufSize)
//
//status := &commonpb.Status{
// ErrorCode: commonpb.ErrorCode_SUCCESS,
//}
//return status, nil
return nil, nil
}
}
func (node *QueryNode) LoadSegments(ctx context.Context, in *queryPb.LoadSegmentRequest) (*commonpb.Status, error) {
// TODO: implement
return nil, nil
}
func (node *QueryNode) ReleaseSegments(ctx context.Context, in *queryPb.ReleaseSegmentRequest) (*commonpb.Status, error) {
// TODO: implement
return nil, nil
}
func (node *QueryNode) GetPartitionState(ctx context.Context, in *queryPb.PartitionStatesRequest) (*queryPb.PartitionStatesResponse, error) {
// TODO: implement
return nil, nil
}
......@@ -8,21 +8,29 @@ import (
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
type LoadIndexClient struct {
type Client struct {
inputStream *msgstream.MsgStream
}
func NewLoadIndexClient(ctx context.Context, pulsarAddress string, loadIndexChannels []string) *LoadIndexClient {
func NewQueryNodeClient(ctx context.Context, pulsarAddress string, loadIndexChannels []string) *Client {
loadIndexStream := msgstream.NewPulsarMsgStream(ctx, 0)
loadIndexStream.SetPulsarClient(pulsarAddress)
loadIndexStream.CreatePulsarProducers(loadIndexChannels)
var input msgstream.MsgStream = loadIndexStream
return &LoadIndexClient{
return &Client{
inputStream: &input,
}
}
func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fieldID int64, fieldName string, indexParams map[string]string) error {
func (c *Client) Close() {
(*c.inputStream).Close()
}
func (c *Client) LoadIndex(indexPaths []string,
segmentID int64,
fieldID int64,
fieldName string,
indexParams map[string]string) error {
baseMsg := msgstream.BaseMsg{
BeginTimestamp: 0,
EndTimestamp: 0,
......@@ -53,10 +61,6 @@ func (lic *LoadIndexClient) LoadIndex(indexPaths []string, segmentID int64, fiel
msgPack := msgstream.MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, loadIndexMsg)
err := (*lic.inputStream).Produce(&msgPack)
err := (*c.inputStream).Produce(&msgPack)
return err
}
func (lic *LoadIndexClient) Close() {
(*lic.inputStream).Close()
}
package querynode
package querynodeimp
/*
......
package querynode
package querynodeimp
/*
......
package querynode
package querynodeimp
import (
"testing"
......@@ -8,20 +8,20 @@ import (
//----------------------------------------------------------------------------------------------------- collection
func TestCollectionReplica_getCollectionNum(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
initTestMeta(t, node, "collection0", 0, 0)
assert.Equal(t, node.replica.getCollectionNum(), 1)
node.Close()
}
func TestCollectionReplica_addCollection(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
initTestMeta(t, node, "collection0", 0, 0)
node.Close()
}
func TestCollectionReplica_removeCollection(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
initTestMeta(t, node, "collection0", 0, 0)
assert.Equal(t, node.replica.getCollectionNum(), 1)
......@@ -32,7 +32,7 @@ func TestCollectionReplica_removeCollection(t *testing.T) {
}
func TestCollectionReplica_getCollectionByID(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -45,7 +45,7 @@ func TestCollectionReplica_getCollectionByID(t *testing.T) {
}
func TestCollectionReplica_getCollectionByName(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -60,7 +60,7 @@ func TestCollectionReplica_getCollectionByName(t *testing.T) {
}
func TestCollectionReplica_hasCollection(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -75,7 +75,7 @@ func TestCollectionReplica_hasCollection(t *testing.T) {
//----------------------------------------------------------------------------------------------------- partition
func TestCollectionReplica_getPartitionNum(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -96,7 +96,7 @@ func TestCollectionReplica_getPartitionNum(t *testing.T) {
}
func TestCollectionReplica_addPartition(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -113,7 +113,7 @@ func TestCollectionReplica_addPartition(t *testing.T) {
}
func TestCollectionReplica_removePartition(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -133,7 +133,7 @@ func TestCollectionReplica_removePartition(t *testing.T) {
}
func TestCollectionReplica_addPartitionsByCollectionMeta(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -157,7 +157,7 @@ func TestCollectionReplica_addPartitionsByCollectionMeta(t *testing.T) {
}
func TestCollectionReplica_removePartitionsByCollectionMeta(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -182,7 +182,7 @@ func TestCollectionReplica_removePartitionsByCollectionMeta(t *testing.T) {
}
func TestCollectionReplica_getPartitionByTag(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -201,7 +201,7 @@ func TestCollectionReplica_getPartitionByTag(t *testing.T) {
}
func TestCollectionReplica_hasPartition(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -218,7 +218,7 @@ func TestCollectionReplica_hasPartition(t *testing.T) {
//----------------------------------------------------------------------------------------------------- segment
func TestCollectionReplica_addSegment(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -237,7 +237,7 @@ func TestCollectionReplica_addSegment(t *testing.T) {
}
func TestCollectionReplica_removeSegment(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -259,7 +259,7 @@ func TestCollectionReplica_removeSegment(t *testing.T) {
}
func TestCollectionReplica_getSegmentByID(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -279,7 +279,7 @@ func TestCollectionReplica_getSegmentByID(t *testing.T) {
}
func TestCollectionReplica_hasSegment(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......@@ -303,7 +303,7 @@ func TestCollectionReplica_hasSegment(t *testing.T) {
}
func TestCollectionReplica_freeAll(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......
package querynode
package querynodeimp
import (
"testing"
......@@ -8,7 +8,7 @@ import (
)
func TestCollection_Partitions(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
......
package querynode
package querynodeimp
import (
"context"
......@@ -40,15 +40,15 @@ func (dsService *dataSyncService) initNodes() {
dsService.fg = flowgraph.NewTimeTickedFlowGraph(dsService.ctx)
var dmStreamNode Node = newDmInputNode(dsService.ctx)
var ddStreamNode Node = newDDInputNode(dsService.ctx)
var dmStreamNode node = newDmInputNode(dsService.ctx)
var ddStreamNode node = newDDInputNode(dsService.ctx)
var filterDmNode Node = newFilteredDmNode()
var ddNode Node = newDDNode(dsService.replica)
var filterDmNode node = newFilteredDmNode()
var ddNode node = newDDNode(dsService.replica)
var insertNode Node = newInsertNode(dsService.replica)
var serviceTimeNode Node = newServiceTimeNode(dsService.replica)
var gcNode Node = newGCNode(dsService.replica)
var insertNode node = newInsertNode(dsService.replica)
var serviceTimeNode node = newServiceTimeNode(dsService.replica)
var gcNode node = newGCNode(dsService.replica)
dsService.fg.AddNode(&dmStreamNode)
dsService.fg.AddNode(&ddStreamNode)
......
package querynode
package querynodeimp
import (
"encoding/binary"
......@@ -14,7 +14,7 @@ import (
// NOTE: start pulsar before test
func TestDataSyncService_Start(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
initTestMeta(t, node, "collection0", 0, 0)
// test data generate
const msgLength = 10
......
package querynode
package querynodeimp
import (
"log"
......@@ -12,7 +12,7 @@ import (
)
type ddNode struct {
BaseNode
baseNode
ddMsg *ddMsg
replica collectionReplica
}
......@@ -179,12 +179,12 @@ func newDDNode(replica collectionReplica) *ddNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
baseNode := BaseNode{}
baseNode := baseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &ddNode{
BaseNode: baseNode,
baseNode: baseNode,
replica: replica,
}
}
package querynode
package querynodeimp
type deleteNode struct {
BaseNode
baseNode
deleteMsg deleteMsg
}
......@@ -17,11 +17,11 @@ func newDeleteNode() *deleteNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
baseNode := BaseNode{}
baseNode := baseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &deleteNode{
BaseNode: baseNode,
baseNode: baseNode,
}
}
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment