Refactor query node and implement channelRemove, channelAdd

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/4973/head^2
bigsheeper 2021-01-15 15:28:54 +08:00 committed by yefu.chen
parent 0971cf65b2
commit b98b226d8c
57 changed files with 1342 additions and 393 deletions

View File

@ -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,

View File

@ -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,

View File

@ -0,0 +1,12 @@
package querynode
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
)
type Client struct {
ctx context.Context
querypb.QueryNodeClient
}

View File

@ -0,0 +1,68 @@
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)
}

View File

@ -0,0 +1,12 @@
package queryservice
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
)
type Client struct {
ctx context.Context
querypb.QueryServiceClient
}

View File

@ -0,0 +1,49 @@
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()
}

View File

@ -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() })

View File

@ -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) {}
}

File diff suppressed because it is too large Load Diff

View File

@ -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))
}

167
internal/querynode/api.go Normal file
View File

@ -0,0 +1,167 @@
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
}

View File

@ -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()
}

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*

View File

@ -1,4 +1,4 @@
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)

View File

@ -1,4 +1,4 @@
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)

View File

@ -1,4 +1,4 @@
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)

View File

@ -1,4 +1,4 @@
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

View File

@ -1,4 +1,4 @@
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,
}
}

View File

@ -1,7 +1,7 @@
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,
}
}

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"
@ -12,7 +12,7 @@ import (
)
type filterDmNode struct {
BaseNode
baseNode
ddMsg *ddMsg
}
@ -159,11 +159,11 @@ func newFilteredDmNode() *filterDmNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
baseNode := BaseNode{}
baseNode := baseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &filterDmNode{
BaseNode: baseNode,
baseNode: baseNode,
}
}

View File

@ -1,11 +1,11 @@
package querynode
package querynodeimp
import (
"log"
)
type gcNode struct {
BaseNode
baseNode
replica collectionReplica
}
@ -50,12 +50,12 @@ func newGCNode(replica collectionReplica) *gcNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
baseNode := BaseNode{}
baseNode := baseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &gcNode{
BaseNode: baseNode,
baseNode: baseNode,
replica: replica,
}
}

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"
@ -13,7 +13,7 @@ import (
)
type insertNode struct {
BaseNode
baseNode
replica collectionReplica
}
@ -162,12 +162,12 @@ func newInsertNode(replica collectionReplica) *insertNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
baseNode := BaseNode{}
baseNode := baseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &insertNode{
BaseNode: baseNode,
baseNode: baseNode,
replica: replica,
}
}

View File

@ -1,7 +1,7 @@
package querynode
package querynodeimp
type key2SegNode struct {
BaseNode
baseNode
key2SegMsg key2SegMsg
}
@ -17,12 +17,12 @@ func newKey2SegNode() *key2SegNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
baseNode := BaseNode{}
baseNode := baseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &key2SegNode{
BaseNode: baseNode,
baseNode: baseNode,
}
}

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"github.com/zilliztech/milvus-distributed/internal/msgstream"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"

View File

@ -1,7 +1,7 @@
package querynode
package querynodeimp
import "github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
type BaseNode = flowgraph.BaseNode
type Node = flowgraph.Node
type InputNode = flowgraph.InputNode
type baseNode = flowgraph.BaseNode
type node = flowgraph.Node
type inputNode = flowgraph.InputNode

View File

@ -1,11 +1,11 @@
package querynode
package querynodeimp
import (
"log"
)
type serviceTimeNode struct {
BaseNode
baseNode
replica collectionReplica
}
@ -42,12 +42,12 @@ func newServiceTimeNode(replica collectionReplica) *serviceTimeNode {
maxQueueLength := Params.FlowGraphMaxQueueLength
maxParallelism := Params.FlowGraphMaxParallelism
baseNode := BaseNode{}
baseNode := baseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &serviceTimeNode{
BaseNode: baseNode,
baseNode: baseNode,
replica: replica,
}
}

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*
#cgo CFLAGS: -I${SRCDIR}/../core/output/include

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"testing"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"encoding/binary"
@ -23,7 +23,7 @@ import (
)
func TestLoadIndexService_FloatVector(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionID := rand.Int63n(1000000)
segmentID := rand.Int63n(1000000)
initTestMeta(t, node, "collection0", collectionID, segmentID)
@ -276,7 +276,7 @@ func TestLoadIndexService_FloatVector(t *testing.T) {
// create loadIndexClient
fieldID := UniqueID(100)
loadIndexChannelNames := Params.LoadIndexChannelNames
client := client.NewLoadIndexClient(node.queryNodeLoopCtx, Params.PulsarAddress, loadIndexChannelNames)
client := client.NewQueryNodeClient(node.queryNodeLoopCtx, Params.PulsarAddress, loadIndexChannelNames)
client.LoadIndex(indexPaths, segmentID, fieldID, "vec", indexParams)
// init message stream consumer and do checks
@ -341,7 +341,7 @@ func TestLoadIndexService_FloatVector(t *testing.T) {
}
func TestLoadIndexService_BinaryVector(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionID := rand.Int63n(1000000)
segmentID := rand.Int63n(1000000)
initTestMeta(t, node, "collection0", collectionID, segmentID, true)
@ -584,7 +584,7 @@ func TestLoadIndexService_BinaryVector(t *testing.T) {
// create loadIndexClient
fieldID := UniqueID(100)
loadIndexChannelNames := Params.LoadIndexChannelNames
client := client.NewLoadIndexClient(node.queryNodeLoopCtx, Params.PulsarAddress, loadIndexChannelNames)
client := client.NewQueryNodeClient(node.queryNodeLoopCtx, Params.PulsarAddress, loadIndexChannelNames)
client.LoadIndex(indexPaths, segmentID, fieldID, "vec", indexParams)
// init message stream consumer and do checks

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"math"
@ -10,7 +10,7 @@ import (
)
func TestMetaService_start(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
node.metaService.start()
@ -118,7 +118,7 @@ func TestMetaService_printSegmentStruct(t *testing.T) {
}
func TestMetaService_processCollectionCreate(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
id := "0"
@ -163,7 +163,7 @@ func TestMetaService_processCollectionCreate(t *testing.T) {
}
func TestMetaService_processSegmentCreate(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
@ -185,7 +185,7 @@ func TestMetaService_processSegmentCreate(t *testing.T) {
}
func TestMetaService_processCreate(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
key1 := Params.MetaRootPath + "/collection/0"
@ -241,7 +241,7 @@ func TestMetaService_processCreate(t *testing.T) {
}
func TestMetaService_processSegmentModify(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
segmentID := UniqueID(0)
@ -275,7 +275,7 @@ func TestMetaService_processSegmentModify(t *testing.T) {
}
func TestMetaService_processCollectionModify(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
id := "0"
@ -383,7 +383,7 @@ func TestMetaService_processCollectionModify(t *testing.T) {
}
func TestMetaService_processModify(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
key1 := Params.MetaRootPath + "/collection/0"
@ -516,7 +516,7 @@ func TestMetaService_processModify(t *testing.T) {
}
func TestMetaService_processSegmentDelete(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)
@ -541,7 +541,7 @@ func TestMetaService_processSegmentDelete(t *testing.T) {
}
func TestMetaService_processCollectionDelete(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
id := "0"
@ -589,7 +589,7 @@ func TestMetaService_processCollectionDelete(t *testing.T) {
}
func TestMetaService_processDelete(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
key1 := Params.MetaRootPath + "/collection/0"
@ -652,7 +652,7 @@ func TestMetaService_processDelete(t *testing.T) {
}
func TestMetaService_processResp(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
metaChan := (*node.metaService).kvBase.WatchWithPrefix("")
@ -667,7 +667,7 @@ func TestMetaService_processResp(t *testing.T) {
}
func TestMetaService_loadCollections(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
err2 := (*node.metaService).loadCollections()
@ -676,7 +676,7 @@ func TestMetaService_loadCollections(t *testing.T) {
}
func TestMetaService_loadSegments(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
node.metaService = newMetaService(node.queryNodeLoopCtx, node.replica)
err2 := (*node.metaService).loadSegments()

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"log"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"fmt"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"testing"
@ -7,7 +7,7 @@ import (
)
func TestPartition_Segments(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
collectionName := "collection0"
collectionID := UniqueID(0)
initTestMeta(t, node, collectionName, collectionID, 0)

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*
#cgo CFLAGS: -I${SRCDIR}/../core/output/include

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"encoding/binary"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*
@ -18,18 +18,36 @@ import (
"io"
"github.com/opentracing/opentracing-go"
"github.com/uber/jaeger-client-go"
"github.com/uber/jaeger-client-go/config"
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
queryPb "github.com/zilliztech/milvus-distributed/internal/proto/querypb"
)
type Node interface {
Start() error
Close()
AddQueryChannel(ctx context.Context, in *queryPb.AddQueryChannelsRequest) (*commonpb.Status, error)
RemoveQueryChannel(ctx context.Context, in *queryPb.RemoveQueryChannelsRequest) (*commonpb.Status, error)
WatchDmChannels(ctx context.Context, in *queryPb.WatchDmChannelsRequest) (*commonpb.Status, error)
LoadSegments(ctx context.Context, in *queryPb.LoadSegmentRequest) (*commonpb.Status, error)
ReleaseSegments(ctx context.Context, in *queryPb.ReleaseSegmentRequest) (*commonpb.Status, error)
GetPartitionState(ctx context.Context, in *queryPb.PartitionStatesRequest) (*queryPb.PartitionStatesResponse, error)
}
type QueryNode struct {
queryNodeLoopCtx context.Context
queryNodeLoopCancel context.CancelFunc
QueryNodeID uint64
grpcServer *grpc.Server
replica collectionReplica
// services
// internal services
dataSyncService *dataSyncService
metaService *metaService
searchService *searchService
@ -45,7 +63,12 @@ func Init() {
Params.Init()
}
func NewQueryNode(ctx context.Context, queryNodeID uint64) *QueryNode {
func NewQueryNode(ctx context.Context, queryNodeID uint64) Node {
var node Node = newQueryNode(ctx, queryNodeID)
return node
}
func newQueryNode(ctx context.Context, queryNodeID uint64) *QueryNode {
ctx1, cancel := context.WithCancel(ctx)
q := &QueryNode{
@ -66,8 +89,11 @@ func NewQueryNode(ctx context.Context, queryNodeID uint64) *QueryNode {
Type: "const",
Param: 1,
},
Reporter: &config.ReporterConfig{
LogSpans: true,
},
}
q.tracer, q.closer, err = cfg.NewTracer()
q.tracer, q.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger))
if err != nil {
panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
}

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"
@ -118,7 +118,7 @@ func initTestMeta(t *testing.T, node *QueryNode, collectionName string, collecti
assert.NoError(t, err)
}
func newQueryNode() *QueryNode {
func newQueryNodeMock() *QueryNode {
var ctx context.Context
@ -134,7 +134,7 @@ func newQueryNode() *QueryNode {
ctx = context.Background()
}
svr := NewQueryNode(ctx, 0)
svr := newQueryNode(ctx, 0)
return svr
}
@ -166,7 +166,7 @@ func TestMain(m *testing.M) {
// NOTE: start pulsar and etcd before test
func TestQueryNode_Start(t *testing.T) {
localNode := newQueryNode()
localNode := newQueryNodeMock()
err := localNode.Start()
assert.Nil(t, err)
localNode.Close()

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*
#cgo CFLAGS: -I${SRCDIR}/../core/output/include

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"encoding/binary"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import "C"
import (

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"
@ -18,7 +18,7 @@ import (
)
func TestSearch_Search(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
initTestMeta(t, node, "collection0", 0, 0)
pulsarURL := Params.PulsarAddress
@ -204,7 +204,7 @@ func TestSearch_Search(t *testing.T) {
}
func TestSearch_SearchMultiSegments(t *testing.T) {
node := NewQueryNode(context.Background(), 0)
node := newQueryNode(context.Background(), 0)
initTestMeta(t, node, "collection0", 0, 0)
pulsarURL := Params.PulsarAddress

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
/*

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"encoding/binary"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"context"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"testing"
@ -8,7 +8,7 @@ import (
// NOTE: start pulsar before test
func TestStatsService_start(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
initTestMeta(t, node, "collection0", 0, 0)
node.statsService = newStatsService(node.queryNodeLoopCtx, node.replica, nil)
node.statsService.start()
@ -17,7 +17,7 @@ func TestStatsService_start(t *testing.T) {
//NOTE: start pulsar before test
func TestSegmentManagement_sendSegmentStatistic(t *testing.T) {
node := newQueryNode()
node := newQueryNodeMock()
initTestMeta(t, node, "collection0", 0, 0)
const receiveBufSize = 1024

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"sync"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import (
"testing"

View File

@ -1,4 +1,4 @@
package querynode
package querynodeimp
import "github.com/zilliztech/milvus-distributed/internal/util/typeutil"

View File

@ -0,0 +1,18 @@
package queryserviceimpl
import "github.com/zilliztech/milvus-distributed/internal/proto/querypb"
type Interface interface {
RegisterNode(req querypb.RegisterNodeRequest) (querypb.RegisterNodeResponse, error)
ShowCollections(req querypb.ShowCollectionRequest) (querypb.ShowCollectionResponse, error)
LoadCollection(req querypb.LoadCollectionRequest) error
ReleaseCollection(req querypb.ReleaseCollectionRequest) error
ShowPartitions(req querypb.ShowPartitionRequest) (querypb.ShowPartitionResponse, error)
GetPartitionStates(req querypb.PartitionStatesRequest) (querypb.PartitionStatesResponse, error)
LoadPartitions(req querypb.LoadPartitionRequest) error
ReleasePartitions(req querypb.ReleasePartitionRequest) error
CreateQueryChannel() (querypb.CreateQueryChannelResponse, error)
}

View File

@ -0,0 +1,42 @@
package queryserviceimpl
import "github.com/zilliztech/milvus-distributed/internal/proto/querypb"
type QueryService struct {
}
func (qs *QueryService) RegisterNode(req querypb.RegisterNodeRequest) (querypb.RegisterNodeResponse, error) {
return querypb.RegisterNodeResponse{}, nil
}
func (qs *QueryService) ShowCollections(req querypb.ShowCollectionRequest) (querypb.ShowCollectionResponse, error) {
return querypb.ShowCollectionResponse{}, nil
}
func (qs *QueryService) LoadCollection(req querypb.LoadCollectionRequest) error {
return nil
}
func (qs *QueryService) ReleaseCollection(req querypb.ReleaseCollectionRequest) error {
return nil
}
func (qs *QueryService) ShowPartitions(req querypb.ShowPartitionRequest) (querypb.ShowPartitionResponse, error) {
return querypb.ShowPartitionResponse{}, nil
}
func (qs *QueryService) GetPartitionStates(req querypb.PartitionStatesRequest) (querypb.PartitionStatesResponse, error) {
return querypb.PartitionStatesResponse{}, nil
}
func (qs *QueryService) LoadPartitions(req querypb.LoadPartitionRequest) error {
return nil
}
func (qs *QueryService) ReleasePartitions(req querypb.ReleasePartitionRequest) error {
return nil
}
func (qs *QueryService) CreateQueryChannel() (querypb.CreateQueryChannelResponse, error) {
return querypb.CreateQueryChannelResponse{}, nil
}

View File

@ -6,6 +6,7 @@ import (
"io"
"github.com/opentracing/opentracing-go"
"github.com/uber/jaeger-client-go"
"github.com/uber/jaeger-client-go/config"
)
@ -51,9 +52,12 @@ func (node *WriteNode) Start() error {
Type: "const",
Param: 1,
},
Reporter: &config.ReporterConfig{
LogSpans: true,
},
}
var err error
node.tracer, node.closer, err = cfg.NewTracer()
node.tracer, node.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger))
if err != nil {
panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
}