mirror of https://github.com/milvus-io/milvus.git
Add unittests for datanode and queryservice
Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>pull/4973/head^2
parent
f1afd5d311
commit
4acaa97039
|
@ -51,7 +51,17 @@ type (
|
|||
}
|
||||
)
|
||||
|
||||
//----------------------------------------------------------------------------------------------------- collection
|
||||
func newReplica() collectionReplica {
|
||||
collections := make([]*Collection, 0)
|
||||
segments := make([]*Segment, 0)
|
||||
|
||||
var replica collectionReplica = &collectionReplicaImpl{
|
||||
collections: collections,
|
||||
segments: segments,
|
||||
}
|
||||
return replica
|
||||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) getSegmentByID(segmentID UniqueID) (*Segment, error) {
|
||||
colReplica.mu.RLock()
|
||||
defer colReplica.mu.RUnlock()
|
||||
|
@ -61,7 +71,7 @@ func (colReplica *collectionReplicaImpl) getSegmentByID(segmentID UniqueID) (*Se
|
|||
return segment, nil
|
||||
}
|
||||
}
|
||||
return nil, errors.Errorf("cannot find segment, id = %v", segmentID)
|
||||
return nil, errors.Errorf("Cannot find segment, id = %v", segmentID)
|
||||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, collID UniqueID,
|
||||
|
@ -163,7 +173,7 @@ func (colReplica *collectionReplicaImpl) addCollection(collectionID UniqueID, sc
|
|||
|
||||
var newCollection = newCollection(collectionID, schema)
|
||||
colReplica.collections = append(colReplica.collections, newCollection)
|
||||
log.Println("Create collection: ", newCollection.Name())
|
||||
log.Println("Create collection:", newCollection.Name())
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -177,25 +187,25 @@ func (colReplica *collectionReplicaImpl) getCollectionIDByName(collName string)
|
|||
return collection.ID(), nil
|
||||
}
|
||||
}
|
||||
return 0, errors.Errorf("There is no collection name=%v", collName)
|
||||
return 0, errors.Errorf("Cannot get collection ID by name %s: not exist", collName)
|
||||
|
||||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) removeCollection(collectionID UniqueID) error {
|
||||
// GOOSE TODO: optimize
|
||||
colReplica.mu.Lock()
|
||||
defer colReplica.mu.Unlock()
|
||||
|
||||
tmpCollections := make([]*Collection, 0)
|
||||
for _, col := range colReplica.collections {
|
||||
if col.ID() != collectionID {
|
||||
tmpCollections = append(tmpCollections, col)
|
||||
} else {
|
||||
log.Println("Drop collection : ", col.Name())
|
||||
}
|
||||
}
|
||||
colReplica.collections = tmpCollections
|
||||
length := len(colReplica.collections)
|
||||
for index, col := range colReplica.collections {
|
||||
if col.ID() == collectionID {
|
||||
log.Println("Drop collection: ", col.Name())
|
||||
colReplica.collections[index] = colReplica.collections[length-1]
|
||||
colReplica.collections = colReplica.collections[:length-1]
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
return errors.Errorf("Cannot remove collection %d: not exist", collectionID)
|
||||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) getCollectionByID(collectionID UniqueID) (*Collection, error) {
|
||||
|
@ -207,7 +217,7 @@ func (colReplica *collectionReplicaImpl) getCollectionByID(collectionID UniqueID
|
|||
return collection, nil
|
||||
}
|
||||
}
|
||||
return nil, errors.Errorf("cannot find collection, id = %v", collectionID)
|
||||
return nil, errors.Errorf("Cannot get collection %d by ID: not exist", collectionID)
|
||||
}
|
||||
|
||||
func (colReplica *collectionReplicaImpl) getCollectionByName(collectionName string) (*Collection, error) {
|
||||
|
|
|
@ -5,20 +5,11 @@ import (
|
|||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
)
|
||||
|
||||
func newReplica() collectionReplica {
|
||||
collections := make([]*Collection, 0)
|
||||
segments := make([]*Segment, 0)
|
||||
|
||||
var replica collectionReplica = &collectionReplicaImpl{
|
||||
collections: collections,
|
||||
segments: segments,
|
||||
}
|
||||
return replica
|
||||
}
|
||||
|
||||
func initTestReplicaMeta(t *testing.T, replica collectionReplica, collectionName string, collectionID UniqueID, segmentID UniqueID) {
|
||||
// GOOSE TODO remove
|
||||
Factory := &MetaFactory{}
|
||||
collectionMeta := Factory.CollectionMetaFactory(collectionID, collectionName)
|
||||
|
||||
|
@ -33,71 +24,133 @@ func initTestReplicaMeta(t *testing.T, replica collectionReplica, collectionName
|
|||
|
||||
}
|
||||
|
||||
//----------------------------------------------------------------------------------------------------- collection
|
||||
func TestCollectionReplica_getCollectionNum(t *testing.T) {
|
||||
replica := newReplica()
|
||||
initTestReplicaMeta(t, replica, "collection0", 0, 0)
|
||||
assert.Equal(t, replica.getCollectionNum(), 1)
|
||||
}
|
||||
func TestReplica_Collection(t *testing.T) {
|
||||
Factory := &MetaFactory{}
|
||||
collMetaMock := Factory.CollectionMetaFactory(0, "collection0")
|
||||
|
||||
func TestCollectionReplica_addCollection(t *testing.T) {
|
||||
replica := newReplica()
|
||||
initTestReplicaMeta(t, replica, "collection0", 0, 0)
|
||||
}
|
||||
t.Run("Test add collection", func(t *testing.T) {
|
||||
|
||||
func TestCollectionReplica_removeCollection(t *testing.T) {
|
||||
replica := newReplica()
|
||||
initTestReplicaMeta(t, replica, "collection0", 0, 0)
|
||||
assert.Equal(t, replica.getCollectionNum(), 1)
|
||||
assert.False(t, replica.hasCollection(0))
|
||||
num := replica.getCollectionNum()
|
||||
assert.Equal(t, 0, num)
|
||||
|
||||
err := replica.removeCollection(0)
|
||||
err := replica.addCollection(0, collMetaMock.GetSchema())
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, replica.getCollectionNum(), 0)
|
||||
}
|
||||
|
||||
func TestCollectionReplica_getCollectionByID(t *testing.T) {
|
||||
replica := newReplica()
|
||||
collectionName := "collection0"
|
||||
collectionID := UniqueID(0)
|
||||
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
|
||||
targetCollection, err := replica.getCollectionByID(collectionID)
|
||||
assert.True(t, replica.hasCollection(0))
|
||||
num = replica.getCollectionNum()
|
||||
assert.Equal(t, 1, num)
|
||||
|
||||
coll, err := replica.getCollectionByID(0)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, targetCollection)
|
||||
assert.Equal(t, targetCollection.Name(), collectionName)
|
||||
assert.Equal(t, targetCollection.ID(), collectionID)
|
||||
}
|
||||
assert.NotNil(t, coll)
|
||||
assert.Equal(t, UniqueID(0), coll.ID())
|
||||
assert.Equal(t, "collection0", coll.Name())
|
||||
assert.Equal(t, collMetaMock.GetSchema(), coll.Schema())
|
||||
|
||||
func TestCollectionReplica_getCollectionByName(t *testing.T) {
|
||||
replica := newReplica()
|
||||
collectionName := "collection0"
|
||||
collectionID := UniqueID(0)
|
||||
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
|
||||
|
||||
targetCollection, err := replica.getCollectionByName(collectionName)
|
||||
coll, err = replica.getCollectionByName("collection0")
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, targetCollection)
|
||||
assert.Equal(t, targetCollection.Name(), collectionName)
|
||||
assert.Equal(t, targetCollection.ID(), collectionID)
|
||||
assert.NotNil(t, coll)
|
||||
assert.Equal(t, UniqueID(0), coll.ID())
|
||||
assert.Equal(t, "collection0", coll.Name())
|
||||
assert.Equal(t, collMetaMock.GetSchema(), coll.Schema())
|
||||
|
||||
}
|
||||
collID, err := replica.getCollectionIDByName("collection0")
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, UniqueID(0), collID)
|
||||
|
||||
func TestCollectionReplica_hasCollection(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("Test remove collection", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
collectionName := "collection0"
|
||||
collectionID := UniqueID(0)
|
||||
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
|
||||
err := replica.addCollection(0, collMetaMock.GetSchema())
|
||||
require.NoError(t, err)
|
||||
|
||||
hasCollection := replica.hasCollection(collectionID)
|
||||
assert.Equal(t, hasCollection, true)
|
||||
hasCollection = replica.hasCollection(UniqueID(1))
|
||||
assert.Equal(t, hasCollection, false)
|
||||
numsBefore := replica.getCollectionNum()
|
||||
coll, err := replica.getCollectionByID(0)
|
||||
require.NotNil(t, coll)
|
||||
require.NoError(t, err)
|
||||
|
||||
}
|
||||
err = replica.removeCollection(0)
|
||||
assert.NoError(t, err)
|
||||
numsAfter := replica.getCollectionNum()
|
||||
assert.Equal(t, 1, numsBefore-numsAfter)
|
||||
|
||||
func TestCollectionReplica_freeAll(t *testing.T) {
|
||||
coll, err = replica.getCollectionByID(0)
|
||||
assert.Nil(t, coll)
|
||||
assert.Error(t, err)
|
||||
err = replica.removeCollection(999999999)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("Test errors", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
collectionName := "collection0"
|
||||
collectionID := UniqueID(0)
|
||||
initTestReplicaMeta(t, replica, collectionName, collectionID, 0)
|
||||
require.False(t, replica.hasCollection(0))
|
||||
require.Equal(t, 0, replica.getCollectionNum())
|
||||
|
||||
coll, err := replica.getCollectionByName("Name-not-exist")
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, coll)
|
||||
|
||||
coll, err = replica.getCollectionByID(0)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, coll)
|
||||
|
||||
collID, err := replica.getCollectionIDByName("Name-not-exist")
|
||||
assert.Error(t, err)
|
||||
assert.Zero(t, collID)
|
||||
|
||||
err = replica.removeCollection(0)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
func TestReplica_Segment(t *testing.T) {
|
||||
t.Run("Test segment", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
assert.False(t, replica.hasSegment(0))
|
||||
|
||||
err := replica.addSegment(0, 1, 2, make([]*internalpb2.MsgPosition, 0))
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, replica.hasSegment(0))
|
||||
|
||||
seg, err := replica.getSegmentByID(0)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, seg)
|
||||
assert.Equal(t, UniqueID(1), seg.collectionID)
|
||||
assert.Equal(t, UniqueID(2), seg.partitionID)
|
||||
|
||||
assert.Equal(t, int64(0), seg.numRows)
|
||||
|
||||
err = replica.updateStatistics(0, 100)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(100), seg.numRows)
|
||||
|
||||
update, err := replica.getSegmentStatisticsUpdates(0)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, UniqueID(0), update.SegmentID)
|
||||
assert.Equal(t, int64(100), update.NumRows)
|
||||
assert.True(t, update.IsNewSegment)
|
||||
})
|
||||
|
||||
t.Run("Test errors", func(t *testing.T) {
|
||||
replica := newReplica()
|
||||
require.False(t, replica.hasSegment(0))
|
||||
|
||||
seg, err := replica.getSegmentByID(0)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, seg)
|
||||
|
||||
err = replica.removeSegment(0)
|
||||
assert.Error(t, err)
|
||||
|
||||
err = replica.updateStatistics(0, 0)
|
||||
assert.Error(t, err)
|
||||
|
||||
update, err := replica.getSegmentStatisticsUpdates(0)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, update)
|
||||
})
|
||||
}
|
||||
|
|
|
@ -16,6 +16,7 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -38,7 +39,6 @@ type (
|
|||
FlushSegments(in *datapb.FlushSegRequest) (*commonpb.Status, error)
|
||||
|
||||
SetMasterServiceInterface(ms MasterServiceInterface) error
|
||||
|
||||
SetDataServiceInterface(ds DataServiceInterface) error
|
||||
}
|
||||
|
||||
|
@ -55,7 +55,6 @@ type (
|
|||
}
|
||||
|
||||
DataNode struct {
|
||||
// GOOSE TODO: complete interface with component
|
||||
ctx context.Context
|
||||
NodeID UniqueID
|
||||
Role string
|
||||
|
@ -80,8 +79,8 @@ func NewDataNode(ctx context.Context) *DataNode {
|
|||
Params.Init()
|
||||
node := &DataNode{
|
||||
ctx: ctx,
|
||||
NodeID: Params.NodeID, // GOOSE TODO
|
||||
Role: "DataNode", // GOOSE TODO
|
||||
NodeID: Params.NodeID, // GOOSE TODO How to init
|
||||
Role: typeutil.DataNodeRole,
|
||||
State: internalpb2.StateCode_INITIALIZING,
|
||||
dataSyncService: nil,
|
||||
metaService: nil,
|
||||
|
@ -107,7 +106,7 @@ func (node *DataNode) Init() error {
|
|||
|
||||
req := &datapb.RegisterNodeRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kNone, //GOOSE TODO
|
||||
MsgType: commonpb.MsgType_kNone,
|
||||
SourceID: node.NodeID,
|
||||
},
|
||||
Address: &commonpb.Address{
|
||||
|
@ -118,11 +117,10 @@ func (node *DataNode) Init() error {
|
|||
|
||||
resp, err := node.dataService.RegisterNode(req)
|
||||
if err != nil {
|
||||
return errors.Errorf("Init failed: %v", err)
|
||||
return errors.Errorf("Register node failed: %v", err)
|
||||
}
|
||||
|
||||
for _, kv := range resp.InitParams.StartParams {
|
||||
log.Println(kv)
|
||||
switch kv.Key {
|
||||
case "DDChannelName":
|
||||
Params.DDChannelNames = []string{kv.Value}
|
||||
|
@ -150,7 +148,7 @@ func (node *DataNode) Init() error {
|
|||
node.metaService = newMetaService(node.ctx, replica, node.masterService)
|
||||
node.replica = replica
|
||||
|
||||
// Opentracing
|
||||
// --- Opentracing ---
|
||||
cfg := &config.Configuration{
|
||||
ServiceName: "data_node",
|
||||
Sampler: &config.SamplerConfig{
|
||||
|
@ -167,7 +165,6 @@ func (node *DataNode) Init() error {
|
|||
}
|
||||
node.tracer = tracer
|
||||
node.closer = closer
|
||||
|
||||
opentracing.SetGlobalTracer(node.tracer)
|
||||
|
||||
return nil
|
||||
|
@ -183,12 +180,14 @@ func (node *DataNode) Start() error {
|
|||
}
|
||||
|
||||
func (node *DataNode) WatchDmChannels(in *datapb.WatchDmChannelRequest) (*commonpb.Status, error) {
|
||||
log.Println("Init insert channel names:", in.GetChannelNames())
|
||||
Params.InsertChannelNames = append(Params.InsertChannelNames, in.GetChannelNames()...)
|
||||
|
||||
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}, nil
|
||||
}
|
||||
|
||||
func (node *DataNode) GetComponentStates() (*internalpb2.ComponentStates, error) {
|
||||
log.Println("DataNode current state:", node.State)
|
||||
states := &internalpb2.ComponentStates{
|
||||
State: &internalpb2.ComponentInfo{
|
||||
NodeID: Params.NodeID,
|
||||
|
|
|
@ -17,6 +17,7 @@ type (
|
|||
}
|
||||
|
||||
AllocatorFactory struct {
|
||||
ID UniqueID
|
||||
}
|
||||
|
||||
MasterServiceFactory struct {
|
||||
|
@ -161,9 +162,23 @@ func (mf *MetaFactory) CollectionMetaFactory(collectionID UniqueID, collectionNa
|
|||
return &collection
|
||||
}
|
||||
|
||||
func NewAllocatorFactory(id ...UniqueID) *AllocatorFactory {
|
||||
f := &AllocatorFactory{}
|
||||
if len(id) == 1 {
|
||||
f.ID = id[0]
|
||||
}
|
||||
return f
|
||||
}
|
||||
|
||||
func (alloc AllocatorFactory) setID(id UniqueID) {
|
||||
alloc.ID = id
|
||||
}
|
||||
|
||||
func (alloc AllocatorFactory) allocID() (UniqueID, error) {
|
||||
// GOOSE TODO: random ID generate
|
||||
return UniqueID(0), nil
|
||||
if alloc.ID == 0 {
|
||||
return UniqueID(0), nil // GOOSE TODO: random ID generating
|
||||
}
|
||||
return alloc.ID, nil
|
||||
}
|
||||
|
||||
func (m *MasterServiceFactory) setID(id UniqueID) {
|
||||
|
|
|
@ -112,7 +112,7 @@ func (ddNode *ddNode) Operate(in []*Msg) []*Msg {
|
|||
case commonpb.MsgType_kDropPartition:
|
||||
ddNode.dropPartition(msg.(*msgstream.DropPartitionMsg))
|
||||
default:
|
||||
log.Println("Non supporting message type:", msg.Type())
|
||||
log.Println("Not supporting message type:", msg.Type())
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -132,7 +132,7 @@ func (ddNode *ddNode) Operate(in []*Msg) []*Msg {
|
|||
}
|
||||
|
||||
default:
|
||||
log.Println("..........default do nothing")
|
||||
log.Println(". default: do nothing ...")
|
||||
}
|
||||
|
||||
// generate binlog
|
||||
|
@ -303,8 +303,8 @@ func (ddNode *ddNode) createPartition(msg *msgstream.CreatePartitionMsg) {
|
|||
}
|
||||
ddNode.ddRecords.partitionRecords[partitionID] = nil
|
||||
|
||||
partitionTag := msg.PartitionName
|
||||
ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag],
|
||||
partitionName := msg.PartitionName
|
||||
ddNode.ddMsg.partitionRecords[partitionName] = append(ddNode.ddMsg.partitionRecords[partitionName],
|
||||
metaOperateRecord{
|
||||
createOrDrop: true,
|
||||
timestamp: msg.Base.Timestamp,
|
||||
|
@ -341,8 +341,8 @@ func (ddNode *ddNode) dropPartition(msg *msgstream.DropPartitionMsg) {
|
|||
}
|
||||
delete(ddNode.ddRecords.partitionRecords, partitionID)
|
||||
|
||||
partitionTag := msg.PartitionName
|
||||
ddNode.ddMsg.partitionRecords[partitionTag] = append(ddNode.ddMsg.partitionRecords[partitionTag],
|
||||
partitionName := msg.PartitionName
|
||||
ddNode.ddMsg.partitionRecords[partitionName] = append(ddNode.ddMsg.partitionRecords[partitionName],
|
||||
metaOperateRecord{
|
||||
createOrDrop: false,
|
||||
timestamp: msg.Base.Timestamp,
|
||||
|
|
|
@ -2,6 +2,7 @@ package datanode
|
|||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -35,51 +36,57 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
|
|||
require.NoError(t, err)
|
||||
Params.MetaRootPath = testPath
|
||||
|
||||
Params.FlushDdBufferSize = 4
|
||||
// Params.FlushDdBufferSize = 4
|
||||
replica := newReplica()
|
||||
idFactory := AllocatorFactory{}
|
||||
ddNode := newDDNode(ctx, newMetaTable(), inFlushCh, replica, idFactory)
|
||||
allocatorMock := NewAllocatorFactory()
|
||||
ddNode := newDDNode(ctx, newMetaTable(), inFlushCh, replica, allocatorMock)
|
||||
log.Print()
|
||||
|
||||
colID := UniqueID(0)
|
||||
colName := "col-test-0"
|
||||
collID := UniqueID(0)
|
||||
collName := "col-test-0"
|
||||
// create collection
|
||||
createColReq := internalpb2.CreateCollectionRequest{
|
||||
createCollReq := internalpb2.CreateCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kCreateCollection,
|
||||
MsgID: 1,
|
||||
Timestamp: 1,
|
||||
SourceID: 1,
|
||||
},
|
||||
CollectionID: colID,
|
||||
CollectionID: collID,
|
||||
Schema: make([]byte, 0),
|
||||
CollectionName: collName,
|
||||
DbName: "DbName",
|
||||
DbID: UniqueID(0),
|
||||
}
|
||||
createColMsg := msgstream.CreateCollectionMsg{
|
||||
createCollMsg := msgstream.CreateCollectionMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
BeginTimestamp: Timestamp(1),
|
||||
EndTimestamp: Timestamp(1),
|
||||
HashValues: []uint32{uint32(0)},
|
||||
},
|
||||
CreateCollectionRequest: createColReq,
|
||||
CreateCollectionRequest: createCollReq,
|
||||
}
|
||||
|
||||
// drop collection
|
||||
dropColReq := internalpb2.DropCollectionRequest{
|
||||
dropCollReq := internalpb2.DropCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kDropCollection,
|
||||
MsgID: 2,
|
||||
Timestamp: 2,
|
||||
SourceID: 2,
|
||||
},
|
||||
CollectionID: colID,
|
||||
CollectionName: colName,
|
||||
CollectionID: collID,
|
||||
CollectionName: collName,
|
||||
DbName: "DbName",
|
||||
DbID: UniqueID(0),
|
||||
}
|
||||
dropColMsg := msgstream.DropCollectionMsg{
|
||||
dropCollMsg := msgstream.DropCollectionMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
BeginTimestamp: Timestamp(2),
|
||||
EndTimestamp: Timestamp(2),
|
||||
HashValues: []uint32{uint32(0)},
|
||||
},
|
||||
DropCollectionRequest: dropColReq,
|
||||
DropCollectionRequest: dropCollReq,
|
||||
}
|
||||
|
||||
partitionID := UniqueID(100)
|
||||
|
@ -92,10 +99,12 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
|
|||
Timestamp: 3,
|
||||
SourceID: 3,
|
||||
},
|
||||
CollectionID: colID,
|
||||
CollectionID: collID,
|
||||
PartitionID: partitionID,
|
||||
CollectionName: colName,
|
||||
CollectionName: collName,
|
||||
PartitionName: partitionName,
|
||||
DbName: "DbName",
|
||||
DbID: UniqueID(0),
|
||||
}
|
||||
createPartitionMsg := msgstream.CreatePartitionMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
|
@ -114,10 +123,12 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
|
|||
Timestamp: 4,
|
||||
SourceID: 4,
|
||||
},
|
||||
CollectionID: colID,
|
||||
CollectionID: collID,
|
||||
PartitionID: partitionID,
|
||||
CollectionName: colName,
|
||||
CollectionName: collName,
|
||||
PartitionName: partitionName,
|
||||
DbName: "DbName",
|
||||
DbID: UniqueID(0),
|
||||
}
|
||||
dropPartitionMsg := msgstream.DropPartitionMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
|
@ -128,16 +139,17 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
|
|||
DropPartitionRequest: dropPartitionReq,
|
||||
}
|
||||
|
||||
replica.addSegment(1, collID, partitionID, make([]*internalpb2.MsgPosition, 0))
|
||||
inFlushCh <- &flushMsg{
|
||||
msgID: 1,
|
||||
timestamp: 6,
|
||||
msgID: 5,
|
||||
timestamp: 5,
|
||||
segmentIDs: []UniqueID{1},
|
||||
collectionID: UniqueID(1),
|
||||
collectionID: collID,
|
||||
}
|
||||
|
||||
tsMessages := make([]msgstream.TsMsg, 0)
|
||||
tsMessages = append(tsMessages, msgstream.TsMsg(&createColMsg))
|
||||
tsMessages = append(tsMessages, msgstream.TsMsg(&dropColMsg))
|
||||
tsMessages = append(tsMessages, msgstream.TsMsg(&createCollMsg))
|
||||
tsMessages = append(tsMessages, msgstream.TsMsg(&dropCollMsg))
|
||||
tsMessages = append(tsMessages, msgstream.TsMsg(&createPartitionMsg))
|
||||
tsMessages = append(tsMessages, msgstream.TsMsg(&dropPartitionMsg))
|
||||
msgStream := flowgraph.GenerateMsgStreamMsg(tsMessages, Timestamp(0), Timestamp(3), make([]*internalpb2.MsgPosition, 0))
|
||||
|
|
|
@ -27,6 +27,7 @@ func newMetaService(ctx context.Context, replica collectionReplica, m MasterServ
|
|||
}
|
||||
|
||||
func (mService *metaService) init() {
|
||||
log.Println("Initing meta ...")
|
||||
err := mService.loadCollections()
|
||||
if err != nil {
|
||||
log.Fatal("metaService init failed:", err)
|
||||
|
|
|
@ -7,7 +7,6 @@ import (
|
|||
"sync"
|
||||
|
||||
dn "github.com/zilliztech/milvus-distributed/internal/datanode"
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
|
@ -67,11 +66,6 @@ func (s *Server) SetDataServiceInterface(ds dn.DataServiceInterface) error {
|
|||
}
|
||||
|
||||
func (s *Server) Init() error {
|
||||
err := s.core.Init()
|
||||
if err != nil {
|
||||
return errors.Errorf("Init failed: %v", err)
|
||||
}
|
||||
|
||||
return s.core.Init()
|
||||
}
|
||||
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
package proxyservice
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||
|
||||
|
@ -17,22 +17,21 @@ type NodeIDAllocator interface {
|
|||
|
||||
type NaiveNodeIDAllocatorImpl struct {
|
||||
impl *allocator.IDAllocator
|
||||
now UniqueID
|
||||
mtx sync.Mutex
|
||||
}
|
||||
|
||||
func (allocator *NaiveNodeIDAllocatorImpl) AllocOne() UniqueID {
|
||||
id, err := allocator.impl.AllocOne()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return id
|
||||
allocator.mtx.Lock()
|
||||
defer func() {
|
||||
allocator.now++
|
||||
allocator.mtx.Unlock()
|
||||
}()
|
||||
return allocator.now
|
||||
}
|
||||
|
||||
func NewNodeIDAllocator() NodeIDAllocator {
|
||||
impl, err := allocator.NewIDAllocator(context.Background(), Params.MasterAddress())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return &NaiveNodeIDAllocatorImpl{
|
||||
impl: impl,
|
||||
now: 0,
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue