mirror of https://github.com/milvus-io/milvus.git
Fix the logic of node id allocator
Signed-off-by: dragondriver <jiquan.long@zilliz.com>pull/4973/head^2
parent
bea1535425
commit
e384222803
|
@ -4,7 +4,6 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"time"
|
||||
|
||||
ds "github.com/zilliztech/milvus-distributed/internal/dataservice"
|
||||
dsc "github.com/zilliztech/milvus-distributed/internal/distributed/dataservice"
|
||||
|
@ -12,12 +11,10 @@ import (
|
|||
msc "github.com/zilliztech/milvus-distributed/internal/distributed/masterservice"
|
||||
ps "github.com/zilliztech/milvus-distributed/internal/distributed/proxyservice"
|
||||
psc "github.com/zilliztech/milvus-distributed/internal/distributed/proxyservice/client"
|
||||
qsc "github.com/zilliztech/milvus-distributed/internal/distributed/queryservice/client"
|
||||
is "github.com/zilliztech/milvus-distributed/internal/indexservice"
|
||||
ms "github.com/zilliztech/milvus-distributed/internal/masterservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
qs "github.com/zilliztech/milvus-distributed/internal/queryservice"
|
||||
)
|
||||
|
||||
type MasterService struct {
|
||||
|
@ -27,7 +24,6 @@ type MasterService struct {
|
|||
proxyService *psc.Client
|
||||
dataService *dsc.Client
|
||||
indexService *isc.Client
|
||||
queryService *qsc.Client
|
||||
}
|
||||
|
||||
func NewMasterService(ctx context.Context) (*MasterService, error) {
|
||||
|
@ -107,18 +103,6 @@ func NewMasterService(ctx context.Context) (*MasterService, error) {
|
|||
if err = svr.SetIndexService(indexService); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
qs.Params.Init()
|
||||
log.Printf("query service address = %s:%d", qs.Params.Address, qs.Params.Port)
|
||||
queryService, err := qsc.NewClient(fmt.Sprintf("%s:%d", qs.Params.Address, qs.Params.Port), time.Duration(ms.Params.Timeout)*time.Second)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err = svr.SetQueryService(queryService); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &MasterService{
|
||||
ctx: ctx,
|
||||
svr: svr,
|
||||
|
@ -126,7 +110,6 @@ func NewMasterService(ctx context.Context) (*MasterService, error) {
|
|||
proxyService: proxyService,
|
||||
dataService: dataService,
|
||||
indexService: indexService,
|
||||
queryService: queryService,
|
||||
}, nil
|
||||
}
|
||||
|
||||
|
@ -152,9 +135,6 @@ func (m *MasterService) Stop() error {
|
|||
if m.dataService != nil {
|
||||
_ = m.dataService.Stop()
|
||||
}
|
||||
if m.queryService != nil {
|
||||
_ = m.queryService.Stop()
|
||||
}
|
||||
if m.svr != nil {
|
||||
return m.svr.Stop()
|
||||
}
|
||||
|
|
|
@ -111,10 +111,6 @@ func TestGrpcService(t *testing.T) {
|
|||
return nil
|
||||
}
|
||||
|
||||
core.ReleaseCollection = func(ts typeutil.Timestamp, dbID typeutil.UniqueID, collectionID typeutil.UniqueID) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
err = svr.Start()
|
||||
assert.Nil(t, err)
|
||||
|
||||
|
|
|
@ -97,14 +97,6 @@ func (s *GrpcServer) SetIndexService(p cms.IndexServiceInterface) error {
|
|||
return c.SetIndexService(p)
|
||||
}
|
||||
|
||||
func (s *GrpcServer) SetQueryService(q cms.QueryServiceInterface) error {
|
||||
c, ok := s.core.(*cms.Core)
|
||||
if !ok {
|
||||
return errors.Errorf("set query service failed")
|
||||
}
|
||||
return c.SetQueryService(q)
|
||||
}
|
||||
|
||||
func (s *GrpcServer) GetComponentStatesRPC(ctx context.Context, empty *commonpb.Empty) (*internalpb2.ComponentStates, error) {
|
||||
return s.core.GetComponentStates()
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
|
@ -48,10 +47,6 @@ type IndexServiceInterface interface {
|
|||
BuildIndex(req *indexpb.BuildIndexRequest) (*indexpb.BuildIndexResponse, error)
|
||||
}
|
||||
|
||||
type QueryServiceInterface interface {
|
||||
ReleaseCollection(req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error)
|
||||
}
|
||||
|
||||
type Interface interface {
|
||||
//service
|
||||
Init() error
|
||||
|
@ -152,18 +147,15 @@ type Core struct {
|
|||
//setMsgStreams ,if segment flush completed, data node would put segment id into msg stream
|
||||
DataNodeSegmentFlushCompletedChan chan typeutil.UniqueID
|
||||
|
||||
//get binlog file path from data service,
|
||||
//TODO,get binlog file path from data service,
|
||||
GetBinlogFilePathsFromDataServiceReq func(segID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]string, error)
|
||||
|
||||
//call index builder's client to build index, return build id
|
||||
//TODO, call index builder's client to build index, return build id
|
||||
BuildIndexReq func(binlog []string, typeParams []*commonpb.KeyValuePair, indexParams []*commonpb.KeyValuePair, indexID typeutil.UniqueID, indexName string) (typeutil.UniqueID, error)
|
||||
|
||||
//proxy service interface, notify proxy service to drop collection
|
||||
//TODO, proxy service interface, notify proxy service to drop collection
|
||||
InvalidateCollectionMetaCache func(ts typeutil.Timestamp, dbName string, collectionName string) error
|
||||
|
||||
//query service interface, notify query service to release collection
|
||||
ReleaseCollection func(ts typeutil.Timestamp, dbID typeutil.UniqueID, collectionID typeutil.UniqueID) error
|
||||
|
||||
// put create index task into this chan
|
||||
indexTaskQueue chan *CreateIndexTask
|
||||
|
||||
|
@ -253,10 +245,6 @@ func (c *Core) checkInit() error {
|
|||
if c.DataNodeSegmentFlushCompletedChan == nil {
|
||||
return errors.Errorf("DataNodeSegmentFlushCompletedChan is nil")
|
||||
}
|
||||
if c.ReleaseCollection == nil {
|
||||
return errors.Errorf("ReleaseCollection is nil")
|
||||
}
|
||||
|
||||
log.Printf("master node id = %d", Params.NodeID)
|
||||
log.Printf("master dd channel name = %s", Params.DdChannel)
|
||||
log.Printf("master time ticke channel name = %s", Params.TimeTickChannel)
|
||||
|
@ -703,30 +691,6 @@ func (c *Core) SetIndexService(s IndexServiceInterface) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (c *Core) SetQueryService(s QueryServiceInterface) error {
|
||||
c.ReleaseCollection = func(ts typeutil.Timestamp, dbID typeutil.UniqueID, collectionID typeutil.UniqueID) error {
|
||||
req := &querypb.ReleaseCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kReleaseCollection,
|
||||
MsgID: 0, //TODO, msg ID
|
||||
Timestamp: ts,
|
||||
SourceID: int64(Params.NodeID),
|
||||
},
|
||||
DbID: dbID,
|
||||
CollectionID: collectionID,
|
||||
}
|
||||
rsp, err := s.ReleaseCollection(req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if rsp.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||
return errors.Errorf("ReleaseCollection from query service failed, error = %s", rsp.Reason)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Core) Init() error {
|
||||
var initError error = nil
|
||||
c.initOnce.Do(func() {
|
||||
|
|
|
@ -18,7 +18,6 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
)
|
||||
|
@ -80,21 +79,6 @@ func (d *dataMock) GetInsertBinlogPaths(req *datapb.InsertBinlogPathRequest) (*d
|
|||
return rst, nil
|
||||
}
|
||||
|
||||
type queryMock struct {
|
||||
collID []typeutil.UniqueID
|
||||
mutex sync.Mutex
|
||||
}
|
||||
|
||||
func (q *queryMock) ReleaseCollection(req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
|
||||
q.mutex.Lock()
|
||||
defer q.mutex.Unlock()
|
||||
q.collID = append(q.collID, req.CollectionID)
|
||||
return &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (d *dataMock) GetSegmentInfoChannel() (*milvuspb.StringResponse, error) {
|
||||
return &milvuspb.StringResponse{
|
||||
Status: &commonpb.Status{
|
||||
|
@ -179,13 +163,6 @@ func TestMasterService(t *testing.T) {
|
|||
err = core.SetIndexService(im)
|
||||
assert.Nil(t, err)
|
||||
|
||||
qm := &queryMock{
|
||||
collID: nil,
|
||||
mutex: sync.Mutex{},
|
||||
}
|
||||
err = core.SetQueryService(qm)
|
||||
assert.Nil(t, err)
|
||||
|
||||
err = core.Init()
|
||||
assert.Nil(t, err)
|
||||
|
||||
|
@ -791,12 +768,6 @@ func TestMasterService(t *testing.T) {
|
|||
assert.Equal(t, len(collArray), 1)
|
||||
assert.Equal(t, collArray[0], "testColl")
|
||||
|
||||
time.Sleep(time.Millisecond * 100)
|
||||
qm.mutex.Lock()
|
||||
assert.Equal(t, len(qm.collID), 1)
|
||||
assert.Equal(t, qm.collID[0], collMeta.ID)
|
||||
qm.mutex.Unlock()
|
||||
|
||||
req = &milvuspb.DropCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kDropCollection,
|
||||
|
|
|
@ -1,8 +1,6 @@
|
|||
package masterservice
|
||||
|
||||
import (
|
||||
"log"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
|
@ -197,7 +195,6 @@ func (t *DropCollectionReqTask) Execute() error {
|
|||
if err = t.core.InvalidateCollectionMetaCache(t.Req.Base.Timestamp, t.Req.DbName, t.Req.CollectionName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = t.core.MetaTable.DeleteCollection(collMeta.ID)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -217,14 +214,6 @@ func (t *DropCollectionReqTask) Execute() error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
//notify query service to release collection
|
||||
go func() {
|
||||
if err = t.core.ReleaseCollection(t.Req.Base.Timestamp, 0, collMeta.ID); err != nil {
|
||||
log.Printf("%s", err.Error())
|
||||
}
|
||||
}()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -177,8 +177,10 @@ func (m *InsertChannelsMap) closeAllMsgStream() {
|
|||
m.mtx.Lock()
|
||||
defer m.mtx.Unlock()
|
||||
|
||||
for _, stream := range m.insertMsgStreams {
|
||||
stream.Close()
|
||||
for loc, stream := range m.insertMsgStreams {
|
||||
if m.droppedBitMap[loc] == 0 && m.usageHistogram[loc] >= 1 {
|
||||
stream.Close()
|
||||
}
|
||||
}
|
||||
|
||||
m.collectionID2InsertChannels = make(map[UniqueID]int)
|
||||
|
|
|
@ -24,7 +24,7 @@ type NaiveNodeIDAllocatorImpl struct {
|
|||
func (allocator *NaiveNodeIDAllocatorImpl) AllocOne() UniqueID {
|
||||
allocator.mtx.Lock()
|
||||
defer func() {
|
||||
allocator.now++
|
||||
// allocator.now++
|
||||
allocator.mtx.Unlock()
|
||||
}()
|
||||
return allocator.now
|
||||
|
@ -32,6 +32,6 @@ func (allocator *NaiveNodeIDAllocatorImpl) AllocOne() UniqueID {
|
|||
|
||||
func NewNodeIDAllocator() NodeIDAllocator {
|
||||
return &NaiveNodeIDAllocatorImpl{
|
||||
now: 0,
|
||||
now: 1,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -83,8 +83,8 @@ func (ttBarrier *softTimeTickBarrier) Start() error {
|
|||
case ttmsgs := <-ttBarrier.ttStream.Chan():
|
||||
//log.Println("ttmsgs: ", ttmsgs)
|
||||
ttBarrier.peerMtx.RLock()
|
||||
log.Println("peer2LastTt map: ", ttBarrier.peer2LastTt)
|
||||
log.Println("len(ttmsgs.Msgs): ", len(ttmsgs.Msgs))
|
||||
//log.Println("peer2LastTt map: ", ttBarrier.peer2LastTt)
|
||||
//log.Println("len(ttmsgs.Msgs): ", len(ttmsgs.Msgs))
|
||||
if len(ttmsgs.Msgs) > 0 {
|
||||
for _, timetickmsg := range ttmsgs.Msgs {
|
||||
ttmsg := timetickmsg.(*ms.TimeTickMsg)
|
||||
|
|
Loading…
Reference in New Issue