Refactor query node and services

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/4973/head^2
bigsheeper 2020-11-05 10:52:50 +08:00 committed by yefu.chen
parent 425f57e1c7
commit 88a3fa31fa
37 changed files with 2892 additions and 3310 deletions

View File

@ -1128,7 +1128,7 @@ func NewMetaTable(kv kv.Base) (*metaTable,error)
* Soft Time Tick Barrier
<img src="./raw_figs/Soft_time_tick_barrier.png" width=600>
<img src="./figs/Soft_time_tick_barrier.png" width=500>
```go
type softTimeTickBarrier struct {
@ -1150,7 +1150,7 @@ func newSoftTimeTickBarrier(ctx context.Context, ttStream *MsgStream, peerIds []
* Hard Time Tick Barrier
<img src="./raw_figs/Hard_time_tick_barrier.png" width=500>
<img src="./figs/Hard_time_tick_barrier.png" width=420>
```go
type hardTimeTickBarrier struct {
@ -1170,7 +1170,7 @@ func newHardTimeTickBarrier(ctx context.Context, ttStream *MsgStream, peerIds []
###### 10.5.1 Time Synchronization Message Producer
<img src="./raw_figs/time_sync_msg_producer.png" width=900>
<img src="./figs/time_sync_msg_producer.png" width=700>

View File

@ -212,14 +212,14 @@ func (mc *ReaderMessageClient) InitClient(ctx context.Context, url string) {
proxyIdList := conf.Config.Master.ProxyIdList
searchResultTopicName := "SearchResult-"
searchTopicName := "Search"
searchTopicName := "search"
key2SegTopicName := "Key2Seg"
timeSyncTopicName := "TimeSync"
insertOrDeleteTopicName := "InsertOrDelete-"
if conf.Config.Pulsar.Authentication {
searchResultTopicName = "SearchResult-" + conf.Config.Pulsar.User + "-"
searchTopicName = "Search-" + conf.Config.Pulsar.User
searchTopicName = "search-" + conf.Config.Pulsar.User
key2SegTopicName = "Key2Seg-" + conf.Config.Pulsar.User
// timeSyncTopicName = "TimeSync-" + conf.Config.Pulsar.User
insertOrDeleteTopicName = "InsertOrDelete-" + conf.Config.Pulsar.User + "-"
@ -468,7 +468,7 @@ func (mc *WriterMessageClient) InitClient(url string) {
if conf.Config.Pulsar.Authentication {
key2SegTopicName = "Key2Seg-" + conf.Config.Pulsar.User
searchByIdTopicName = "Search-" + conf.Config.Pulsar.User
searchByIdTopicName = "search-" + conf.Config.Pulsar.User
// timeSyncTopicName = "TimeSync-" + conf.Config.Pulsar.User
insertOrDeleteTopicName = "InsertOrDelete-" + conf.Config.Pulsar.User + "-"
}

View File

@ -3977,7 +3977,7 @@ func (c *milvusServiceClient) GetEntityIDs(ctx context.Context, in *GetEntityIDs
func (c *milvusServiceClient) Search(ctx context.Context, in *SearchParam, opts ...grpc.CallOption) (*QueryResult, error) {
out := new(QueryResult)
err := c.cc.Invoke(ctx, "/milvus.grpc.MilvusService/Search", in, out, opts...)
err := c.cc.Invoke(ctx, "/milvus.grpc.MilvusService/search", in, out, opts...)
if err != nil {
return nil, err
}
@ -4276,7 +4276,7 @@ func (*UnimplementedMilvusServiceServer) GetEntityIDs(ctx context.Context, req *
return nil, status.Errorf(codes.Unimplemented, "method GetEntityIDs not implemented")
}
func (*UnimplementedMilvusServiceServer) Search(ctx context.Context, req *SearchParam) (*QueryResult, error) {
return nil, status.Errorf(codes.Unimplemented, "method Search not implemented")
return nil, status.Errorf(codes.Unimplemented, "method search not implemented")
}
func (*UnimplementedMilvusServiceServer) SearchInSegment(ctx context.Context, req *SearchInSegmentParam) (*QueryResult, error) {
return nil, status.Errorf(codes.Unimplemented, "method SearchInSegment not implemented")
@ -4620,7 +4620,7 @@ func _MilvusService_Search_Handler(srv interface{}, ctx context.Context, dec fun
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.grpc.MilvusService/Search",
FullMethod: "/milvus.grpc.MilvusService/search",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MilvusServiceServer).Search(ctx, req.(*SearchParam))
@ -4827,7 +4827,7 @@ var _MilvusService_serviceDesc = grpc.ServiceDesc{
Handler: _MilvusService_GetEntityIDs_Handler,
},
{
MethodName: "Search",
MethodName: "search",
Handler: _MilvusService_Search_Handler,
},
{

View File

@ -255,7 +255,7 @@ func (c *milvusServiceClient) Insert(ctx context.Context, in *RowBatch, opts ...
func (c *milvusServiceClient) Search(ctx context.Context, in *Query, opts ...grpc.CallOption) (*QueryResult, error) {
out := new(QueryResult)
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/Search", in, out, opts...)
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/search", in, out, opts...)
if err != nil {
return nil, err
}
@ -378,7 +378,7 @@ func (*UnimplementedMilvusServiceServer) Insert(ctx context.Context, req *RowBat
return nil, status.Errorf(codes.Unimplemented, "method Insert not implemented")
}
func (*UnimplementedMilvusServiceServer) Search(ctx context.Context, req *Query) (*QueryResult, error) {
return nil, status.Errorf(codes.Unimplemented, "method Search not implemented")
return nil, status.Errorf(codes.Unimplemented, "method search not implemented")
}
func RegisterMilvusServiceServer(s *grpc.Server, srv MilvusServiceServer) {
@ -593,7 +593,7 @@ func _MilvusService_Search_Handler(srv interface{}, ctx context.Context, dec fun
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.service.MilvusService/Search",
FullMethod: "/milvus.proto.service.MilvusService/search",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MilvusServiceServer).Search(ctx, req.(*Query))
@ -650,7 +650,7 @@ var _MilvusService_serviceDesc = grpc.ServiceDesc{
Handler: _MilvusService_Insert_Handler,
},
{
MethodName: "Search",
MethodName: "search",
Handler: _MilvusService_Search_Handler,
},
},

View File

@ -305,7 +305,7 @@ func (m *PlaceholderValue) GetValue() *commonpb.Blob {
}
//*
// @brief Query for Search call
// @brief Query for search call
type Query struct {
CollectionName string `protobuf:"bytes,1,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
PartitionTags []string `protobuf:"bytes,2,rep,name=partition_tags,json=partitionTags,proto3" json:"partition_tags,omitempty"`

View File

@ -23,10 +23,10 @@ type Collection struct {
Partitions []*Partition
}
func (c *Collection) NewPartition(partitionName string) *Partition {
func (c *Collection) newPartition(partitionName string) *Partition {
/*
CPartition
NewPartition(CCollection collection, const char* partition_name);
newPartition(CCollection collection, const char* partition_name);
*/
cName := C.CString(partitionName)
partitionPtr := C.NewPartition(c.CollectionPtr, cName)
@ -36,10 +36,10 @@ func (c *Collection) NewPartition(partitionName string) *Partition {
return newPartition
}
func (c *Collection) DeletePartition(node *QueryNode, partition *Partition) {
func (c *Collection) deletePartition(node *QueryNode, partition *Partition) {
/*
void
DeletePartition(CPartition partition);
deletePartition(CPartition partition);
*/
cPtr := partition.PartitionPtr
C.DeletePartition(cPtr)
@ -58,3 +58,13 @@ func (c *Collection) DeletePartition(node *QueryNode, partition *Partition) {
c.Partitions = tmpPartitions
}
func (c *Collection) getPartitionByName(partitionName string) (partition *Partition) {
for _, partition := range c.Partitions {
if partition.PartitionName == partitionName {
return partition
}
}
return nil
// TODO: remove from c.Partitions
}

View File

@ -9,30 +9,32 @@ import (
func TestCollection_NewPartition(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, collection.CollectionID, int64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, len(collection.Partitions), 1)
}
func TestCollection_DeletePartition(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, collection.CollectionID, int64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, len(collection.Partitions), 1)
collection.DeletePartition(node, partition)
collection.deletePartition(node, partition)
assert.Equal(t, len(collection.Partitions), 0)
}

View File

@ -14,6 +14,8 @@ func (dmNode *dmNode) Name() string {
}
func (dmNode *dmNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do dmNode operation")
// TODO: add filtered by schema update
// But for now, we think all the messages are valid
@ -28,7 +30,12 @@ func (dmNode *dmNode) Operate(in []*Msg) []*Msg {
// TODO: add error handling
}
var res Msg = dmMsg
var fdmMsg = filteredDmMsg{
insertMessages: dmMsg.insertMessages,
timeRange: dmMsg.timeRange,
}
var res Msg = &fdmMsg
return []*Msg{&res}
}

View File

@ -12,10 +12,12 @@ type filteredDmNode struct {
}
func (fdmNode *filteredDmNode) Name() string {
return "dmNode"
return "fdmNode"
}
func (fdmNode *filteredDmNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do filteredDmNode operation")
if len(in) != 1 {
log.Println("Invalid operate message input in filteredDmNode")
// TODO: add error handling

View File

@ -10,7 +10,7 @@ import (
type insertNode struct {
BaseNode
SegmentsMap *map[int64]*Segment
segmentsMap *map[int64]*Segment
insertMsg *insertMsg
}
@ -19,6 +19,8 @@ func (iNode *insertNode) Name() string {
}
func (iNode *insertNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do insertNode operation")
if len(in) != 1 {
log.Println("Invalid operate message input in insertNode")
// TODO: add error handling
@ -59,7 +61,7 @@ func (iNode *insertNode) preInsert() error {
}
var numOfRecords = len(iNode.insertMsg.insertData.insertRecords[segmentID])
var offset = targetSegment.SegmentPreInsert(numOfRecords)
var offset = targetSegment.segmentPreInsert(numOfRecords)
iNode.insertMsg.insertData.insertOffset[segmentID] = offset
}
@ -67,7 +69,7 @@ func (iNode *insertNode) preInsert() error {
}
func (iNode *insertNode) getSegmentBySegmentID(segmentID int64) (*Segment, error) {
targetSegment, ok := (*iNode.SegmentsMap)[segmentID]
targetSegment, ok := (*iNode.segmentsMap)[segmentID]
if !ok {
return nil, errors.New("cannot found segment with id = " + strconv.FormatInt(segmentID, 10))
@ -89,7 +91,7 @@ func (iNode *insertNode) insert(segmentID int64, wg *sync.WaitGroup) {
records := iNode.insertMsg.insertData.insertRecords[segmentID]
offsets := iNode.insertMsg.insertData.insertOffset[segmentID]
err = targetSegment.SegmentInsert(offsets, &ids, &timestamps, &records)
err = targetSegment.segmentInsert(offsets, &ids, &timestamps, &records)
if err != nil {
log.Println("insert failed")
// TODO: add error handling
@ -100,12 +102,13 @@ func (iNode *insertNode) insert(segmentID int64, wg *sync.WaitGroup) {
wg.Done()
}
func newInsertNode() *insertNode {
func newInsertNode(segmentsMap *map[int64]*Segment) *insertNode {
baseNode := BaseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &insertNode{
BaseNode: baseNode,
BaseNode: baseNode,
segmentsMap: segmentsMap,
}
}

View File

@ -2,22 +2,45 @@ package reader
import (
"context"
"fmt"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"log"
"sync"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
)
type manipulationService struct {
ctx context.Context
pulsarURL string
fg *flowgraph.TimeTickedFlowGraph
msgStream *msgstream.PulsarMsgStream
node *QueryNode
}
func (dmService *manipulationService) Start() {
func newManipulationService(ctx context.Context, node *QueryNode, pulsarURL string) *manipulationService {
return &manipulationService{
ctx: ctx,
pulsarURL: pulsarURL,
msgStream: nil,
node: node,
}
}
func (dmService *manipulationService) start() {
const (
pulsarBufSize = 100
consumerChannelSize = 100
)
consumerChannels := []string{"insert"}
consumerSubName := "subInsert"
outputStream := msgstream.NewOutputStream(dmService.pulsarURL, pulsarBufSize, consumerChannelSize, consumerChannels, consumerSubName, true)
(*outputStream).SetMsgMarshaler(nil, msgstream.GetMarshaler(internalPb.MsgType_kInsert))
go (*outputStream).Start()
dmService.initNodes()
go dmService.fg.Start()
dmService.consumeFromMsgStream()
@ -30,20 +53,20 @@ func (dmService *manipulationService) initNodes() {
var dmNode Node = newDmNode()
// var key2SegNode Node = newKey2SegNode()
var schemaUpdateNode Node = newSchemaUpdateNode()
//var schemaUpdateNode Node = newSchemaUpdateNode()
var filteredDmNode Node = newFilteredDmNode()
var insertNode Node = newInsertNode()
var insertNode Node = newInsertNode(&dmService.node.SegmentsMap)
// var deletePreprocessNode Node = newDeletePreprocessNode()
// var deleteNode Node = newDeleteNode()
var serviceTimeNode Node = newServiceTimeNode()
var serviceTimeNode Node = newServiceTimeNode(dmService.node)
dmService.fg.AddNode(&msgStreamNode)
dmService.fg.AddNode(&dmNode)
// fg.AddNode(&key2SegNode)
dmService.fg.AddNode(&schemaUpdateNode)
//dmService.fg.AddNode(&schemaUpdateNode)
dmService.fg.AddNode(&filteredDmNode)
@ -53,24 +76,24 @@ func (dmService *manipulationService) initNodes() {
dmService.fg.AddNode(&serviceTimeNode)
// TODO: add delete pipeline support
var err = dmService.fg.SetEdges(dmNode.Name(),
var err = dmService.fg.SetEdges(msgStreamNode.Name(),
[]string{},
[]string{dmNode.Name()},
)
if err != nil {
log.Fatal("set edges failed in node:", dmNode.Name())
}
err = dmService.fg.SetEdges(dmNode.Name(),
[]string{msgStreamNode.Name()},
[]string{filteredDmNode.Name()},
)
if err != nil {
log.Fatal("set edges failed in node:", dmNode.Name())
}
err = dmService.fg.SetEdges(schemaUpdateNode.Name(),
[]string{},
[]string{filteredDmNode.Name()},
)
if err != nil {
log.Fatal("set edges failed in node:", schemaUpdateNode.Name())
}
err = dmService.fg.SetEdges(filteredDmNode.Name(),
[]string{dmNode.Name(), schemaUpdateNode.Name()},
[]string{dmNode.Name()},
[]string{insertNode.Name()},
)
if err != nil {
@ -112,241 +135,11 @@ func (dmService *manipulationService) consumeFromMsgStream() {
var msgStreamMsg Msg = &msgStreamMsg{
tsMessages: msgPack.Msgs,
timeRange: TimeRange{
timestampMin: Timestamp(msgPack.BeginTs),
timestampMax: Timestamp(msgPack.EndTs),
timestampMin: msgPack.BeginTs,
timestampMax: msgPack.EndTs,
},
}
dmService.fg.Input(&msgStreamMsg)
}
}
}
func (node *QueryNode) MessagesPreprocess(insertDeleteMessages []*msgPb.InsertOrDeleteMsg, timeRange TimeRange) msgPb.Status {
var tMax = timeRange.timestampMax
// 1. Extract messages before readTimeSync from QueryNodeDataBuffer.
// Set valid bitmap to false.
// If segmentId dose not exist in segments map, creating an new segment.
for i, msg := range node.buffer.InsertDeleteBuffer {
if msg.Timestamp < tMax {
if !node.FoundSegmentBySegmentID(msg.SegmentId) {
collection, _ := node.GetCollectionByCollectionName(msg.CollectionName)
if collection != nil {
partition := collection.GetPartitionByName(msg.PartitionTag)
if partition != nil {
newSegment := partition.NewSegment(msg.SegmentId)
node.SegmentsMap[msg.SegmentId] = newSegment
} else {
log.Fatal("Cannot find partition:", msg.PartitionTag)
}
} else {
log.Fatal("Cannot find collection:", msg.CollectionName)
}
}
if msg.Op == msgPb.OpType_INSERT {
if msg.RowsData == nil {
continue
}
node.insertData.insertIDs[msg.SegmentId] = append(node.insertData.insertIDs[msg.SegmentId], msg.Uid)
node.insertData.insertTimestamps[msg.SegmentId] = append(node.insertData.insertTimestamps[msg.SegmentId], msg.Timestamp)
// node.insertData.insertRecords[msg.SegmentID] = append(node.insertData.insertRecords[msg.SegmentID], msg.RowsData.Blob)
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord{
entityID: msg.Uid,
timestamp: msg.Timestamp,
}
node.deletePreprocessData.deleteRecords = append(node.deletePreprocessData.deleteRecords, &r)
node.deletePreprocessData.count++
}
node.buffer.validInsertDeleteBuffer[i] = false
}
}
// 2. Remove invalid messages from buffer.
tmpInsertOrDeleteBuffer := make([]*msgPb.InsertOrDeleteMsg, 0)
for i, isValid := range node.buffer.validInsertDeleteBuffer {
if isValid {
tmpInsertOrDeleteBuffer = append(tmpInsertOrDeleteBuffer, node.buffer.InsertDeleteBuffer[i])
}
}
node.buffer.InsertDeleteBuffer = tmpInsertOrDeleteBuffer
// 3. Resize the valid bitmap and set all bits to true.
node.buffer.validInsertDeleteBuffer = node.buffer.validInsertDeleteBuffer[:len(node.buffer.InsertDeleteBuffer)]
for i := range node.buffer.validInsertDeleteBuffer {
node.buffer.validInsertDeleteBuffer[i] = true
}
// 4. Extract messages before readTimeSync from current messageClient.
// Move massages after readTimeSync to QueryNodeDataBuffer.
// Set valid bitmap to true.
// If segmentId dose not exist in segments map, creating an new segment.
for _, msg := range insertDeleteMessages {
if msg.Timestamp < tMax {
if !node.FoundSegmentBySegmentID(msg.SegmentId) {
collection, _ := node.GetCollectionByCollectionName(msg.CollectionName)
if collection != nil {
partition := collection.GetPartitionByName(msg.PartitionTag)
if partition != nil {
newSegment := partition.NewSegment(msg.SegmentId)
node.SegmentsMap[msg.SegmentId] = newSegment
} else {
log.Fatal("Cannot find partition:", msg.PartitionTag)
}
} else {
log.Fatal("Cannot find collection:", msg.CollectionName)
}
}
if msg.Op == msgPb.OpType_INSERT {
if msg.RowsData == nil {
continue
}
node.insertData.insertIDs[msg.SegmentId] = append(node.insertData.insertIDs[msg.SegmentId], msg.Uid)
node.insertData.insertTimestamps[msg.SegmentId] = append(node.insertData.insertTimestamps[msg.SegmentId], msg.Timestamp)
// node.insertData.insertRecords[msg.SegmentID] = append(node.insertData.insertRecords[msg.SegmentID], msg.RowsData.Blob)
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord{
entityID: msg.Uid,
timestamp: msg.Timestamp,
}
node.deletePreprocessData.deleteRecords = append(node.deletePreprocessData.deleteRecords, &r)
node.deletePreprocessData.count++
}
} else {
node.buffer.InsertDeleteBuffer = append(node.buffer.InsertDeleteBuffer, msg)
node.buffer.validInsertDeleteBuffer = append(node.buffer.validInsertDeleteBuffer, true)
}
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) WriterDelete() msgPb.Status {
// TODO: set timeout
for {
if node.deletePreprocessData.count == 0 {
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
node.messageClient.PrepareKey2SegmentMsg()
var ids, timestamps, segmentIDs = node.GetKey2Segments()
for i := 0; i < len(*ids); i++ {
id := (*ids)[i]
timestamp := (*timestamps)[i]
segmentID := (*segmentIDs)[i]
for _, r := range node.deletePreprocessData.deleteRecords {
if r.timestamp == timestamp && r.entityID == id {
r.segmentID = segmentID
node.deletePreprocessData.count--
}
}
}
}
}
func (node *QueryNode) PreInsertAndDelete() msgPb.Status {
// 1. Do PreInsert
for segmentID := range node.insertData.insertRecords {
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
var numOfRecords = len(node.insertData.insertRecords[segmentID])
var offset = targetSegment.SegmentPreInsert(numOfRecords)
node.insertData.insertOffset[segmentID] = offset
}
// 2. Sort delete preprocess data by segment id
for _, r := range node.deletePreprocessData.deleteRecords {
node.deleteData.deleteIDs[r.segmentID] = append(node.deleteData.deleteIDs[r.segmentID], r.entityID)
node.deleteData.deleteTimestamps[r.segmentID] = append(node.deleteData.deleteTimestamps[r.segmentID], r.timestamp)
}
// 3. Do PreDelete
for segmentID := range node.deleteData.deleteIDs {
if segmentID < 0 {
continue
}
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
var numOfRecords = len(node.deleteData.deleteIDs[segmentID])
var offset = targetSegment.SegmentPreDelete(numOfRecords)
node.deleteData.deleteOffset[segmentID] = offset
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoInsertAndDelete() msgPb.Status {
var wg sync.WaitGroup
// Do insert
for segmentID := range node.insertData.insertRecords {
wg.Add(1)
go node.DoInsert(segmentID, &wg)
}
// Do delete
for segmentID, deleteIDs := range node.deleteData.deleteIDs {
if segmentID < 0 {
continue
}
wg.Add(1)
var deleteTimestamps = node.deleteData.deleteTimestamps[segmentID]
go node.DoDelete(segmentID, &deleteIDs, &deleteTimestamps, &wg)
fmt.Println("Do delete done")
}
wg.Wait()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoInsert(segmentID int64, wg *sync.WaitGroup) msgPb.Status {
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
ids := node.insertData.insertIDs[segmentID]
timestamps := node.insertData.insertTimestamps[segmentID]
records := node.insertData.insertRecords[segmentID]
offsets := node.insertData.insertOffset[segmentID]
err = targetSegment.SegmentInsert(offsets, &ids, &timestamps, &records)
fmt.Println("Do insert done, len = ", len(node.insertData.insertIDs[segmentID]))
node.QueryLog(len(ids))
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
wg.Done()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoDelete(segmentID int64, deleteIDs *[]int64, deleteTimestamps *[]uint64, wg *sync.WaitGroup) msgPb.Status {
var segment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
offset := node.deleteData.deleteOffset[segmentID]
node.msgCounter.DeleteCounter += int64(len(*deleteIDs))
err = segment.SegmentDelete(offset, deleteIDs, deleteTimestamps)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
wg.Done()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}

File diff suppressed because it is too large Load Diff

View File

@ -1,13 +1,13 @@
package reader
import (
"context"
"fmt"
"log"
"path"
"reflect"
"strconv"
"strings"
"sync"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
@ -23,6 +23,55 @@ const (
SegmentPrefix = "/segment/"
)
type metaService struct {
ctx context.Context
kvBase *kv.EtcdKV
node *QueryNode
}
func newMetaService(ctx context.Context, node *QueryNode) *metaService {
ETCDAddr := "http://"
ETCDAddr += conf.Config.Etcd.Address
ETCDPort := conf.Config.Etcd.Port
ETCDAddr = ETCDAddr + ":" + strconv.FormatInt(int64(ETCDPort), 10)
cli, _ := clientv3.New(clientv3.Config{
Endpoints: []string{ETCDAddr},
DialTimeout: 5 * time.Second,
})
return &metaService{
ctx: ctx,
kvBase: kv.NewEtcdKV(cli, conf.Config.Etcd.Rootpath),
node: node,
}
}
func (mService *metaService) start() {
// init from meta
err := mService.loadCollections()
if err != nil {
log.Fatal("metaService loadCollections failed")
}
err = mService.loadSegments()
if err != nil {
log.Fatal("metaService loadSegments failed")
}
metaChan := mService.kvBase.WatchWithPrefix("")
for {
select {
case <-mService.ctx.Done():
return
case resp := <-metaChan:
err := mService.processResp(resp)
if err != nil {
log.Println(err)
}
}
}
}
func GetCollectionObjID(key string) string {
prefix := path.Join(conf.Config.Etcd.Rootpath, CollectionPrefix) + "/"
return strings.TrimPrefix(key, prefix)
@ -36,9 +85,8 @@ func GetSegmentObjID(key string) string {
func isCollectionObj(key string) bool {
prefix := path.Join(conf.Config.Etcd.Rootpath, CollectionPrefix) + "/"
prefix = strings.TrimSpace(prefix)
// println("prefix is :$", prefix)
index := strings.Index(key, prefix)
// println("index is :", index)
return index == 0
}
@ -46,6 +94,7 @@ func isSegmentObj(key string) bool {
prefix := path.Join(conf.Config.Etcd.Rootpath, SegmentPrefix) + "/"
prefix = strings.TrimSpace(prefix)
index := strings.Index(key, prefix)
return index == 0
}
@ -88,99 +137,104 @@ func printSegmentStruct(obj *segment.Segment) {
}
}
func (node *QueryNode) processCollectionCreate(id string, value string) {
func (mService *metaService) processCollectionCreate(id string, value string) {
println(fmt.Sprintf("Create Collection:$%s$", id))
collection, err := collection.JSON2Collection(value)
col, err := collection.JSON2Collection(value)
if err != nil {
println("error of json 2 collection")
println("error of json 2 col")
println(err.Error())
}
//printCollectionStruct(collection)
newCollection := node.NewCollection(collection.ID, collection.Name, collection.GrpcMarshalString)
for _, partitionTag := range collection.PartitionTags {
newCollection.NewPartition(partitionTag)
if col != nil {
newCollection := mService.node.newCollection(col.ID, col.Name, col.GrpcMarshalString)
for _, partitionTag := range col.PartitionTags {
newCollection.newPartition(partitionTag)
}
}
}
func (node *QueryNode) processSegmentCreate(id string, value string) {
func (mService *metaService) processSegmentCreate(id string, value string) {
println("Create Segment: ", id)
segment, err := segment.JSON2Segment(value)
seg, err := segment.JSON2Segment(value)
if err != nil {
println("error of json 2 segment")
println("error of json 2 seg")
println(err.Error())
}
//printSegmentStruct(segment)
if !isSegmentChannelRangeInQueryNodeChannelRange(segment) {
if !isSegmentChannelRangeInQueryNodeChannelRange(seg) {
return
}
collection := node.GetCollectionByID(segment.CollectionID)
if collection != nil {
partition := collection.GetPartitionByName(segment.PartitionTag)
if partition != nil {
newSegmentID := int64(segment.SegmentID) // todo change all to uint64
// start new segment and add it into partition.OpenedSegments
newSegment := partition.NewSegment(newSegmentID)
// newSegment.SegmentStatus = SegmentOpened
newSegment.SegmentCloseTime = segment.CloseTimeStamp
node.SegmentsMap[newSegmentID] = newSegment
// TODO: what if seg == nil? We need to notify master and return rpc request failed
if seg != nil {
col := mService.node.getCollectionByID(seg.CollectionID)
if col != nil {
partition := col.getPartitionByName(seg.PartitionTag)
if partition != nil {
newSegmentID := seg.SegmentID
newSegment := partition.newSegment(newSegmentID)
newSegment.SegmentCloseTime = seg.CloseTimeStamp
mService.node.SegmentsMap[newSegmentID] = newSegment
}
}
}
// segment.CollectionName
}
func (node *QueryNode) processCreate(key string, msg string) {
func (mService *metaService) processCreate(key string, msg string) {
println("process create", key)
if isCollectionObj(key) {
objID := GetCollectionObjID(key)
node.processCollectionCreate(objID, msg)
mService.processCollectionCreate(objID, msg)
} else if isSegmentObj(key) {
objID := GetSegmentObjID(key)
node.processSegmentCreate(objID, msg)
mService.processSegmentCreate(objID, msg)
} else {
println("can not process create msg:", key)
}
}
func (node *QueryNode) processSegmentModify(id string, value string) {
// println("Modify Segment: ", id)
segment, err := segment.JSON2Segment(value)
func (mService *metaService) processSegmentModify(id string, value string) {
seg, err := segment.JSON2Segment(value)
if err != nil {
println("error of json 2 segment")
println(err.Error())
}
// printSegmentStruct(segment)
if !isSegmentChannelRangeInQueryNodeChannelRange(segment) {
if !isSegmentChannelRangeInQueryNodeChannelRange(seg) {
return
}
seg, err := node.GetSegmentBySegmentID(int64(segment.SegmentID)) // todo change to uint64
if seg != nil {
seg.SegmentCloseTime = segment.CloseTimeStamp
queryNodeSeg, err := mService.node.getSegmentBySegmentID(seg.SegmentID)
if err != nil {
println(err)
}
if queryNodeSeg != nil {
queryNodeSeg.SegmentCloseTime = seg.CloseTimeStamp
}
}
}
func (node *QueryNode) processCollectionModify(id string, value string) {
func (mService *metaService) processCollectionModify(id string, value string) {
println("Modify Collection: ", id)
}
func (node *QueryNode) processModify(key string, msg string) {
// println("process modify")
func (mService *metaService) processModify(key string, msg string) {
if isCollectionObj(key) {
objID := GetCollectionObjID(key)
node.processCollectionModify(objID, msg)
mService.processCollectionModify(objID, msg)
} else if isSegmentObj(key) {
objID := GetSegmentObjID(key)
node.processSegmentModify(objID, msg)
mService.processSegmentModify(objID, msg)
} else {
println("can not process modify msg:", key)
}
}
func (node *QueryNode) processSegmentDelete(id string) {
func (mService *metaService) processSegmentDelete(id string) {
println("Delete segment: ", id)
segmentID, err := strconv.ParseInt(id, 10, 64)
@ -188,18 +242,18 @@ func (node *QueryNode) processSegmentDelete(id string) {
log.Println("Cannot parse segment id:" + id)
}
for _, col := range node.Collections {
for _, col := range mService.node.Collections {
for _, p := range col.Partitions {
for _, s := range p.Segments {
if s.SegmentID == segmentID {
p.DeleteSegment(node, s)
p.deleteSegment(mService.node, s)
}
}
}
}
}
func (node *QueryNode) processCollectionDelete(id string) {
func (mService *metaService) processCollectionDelete(id string) {
println("Delete collection: ", id)
collectionID, err := strconv.ParseInt(id, 10, 64)
@ -207,42 +261,42 @@ func (node *QueryNode) processCollectionDelete(id string) {
log.Println("Cannot parse collection id:" + id)
}
targetCollection := node.GetCollectionByID(collectionID)
node.DeleteCollection(targetCollection)
targetCollection := mService.node.getCollectionByID(collectionID)
mService.node.deleteCollection(targetCollection)
}
func (node *QueryNode) processDelete(key string) {
func (mService *metaService) processDelete(key string) {
println("process delete")
if isCollectionObj(key) {
objID := GetCollectionObjID(key)
node.processCollectionDelete(objID)
mService.processCollectionDelete(objID)
} else if isSegmentObj(key) {
objID := GetSegmentObjID(key)
node.processSegmentDelete(objID)
mService.processSegmentDelete(objID)
} else {
println("can not process delete msg:", key)
}
}
func (node *QueryNode) processResp(resp clientv3.WatchResponse) error {
func (mService *metaService) processResp(resp clientv3.WatchResponse) error {
err := resp.Err()
if err != nil {
return err
}
// println("processResp!!!!!\n")
for _, ev := range resp.Events {
if ev.IsCreate() {
key := string(ev.Kv.Key)
msg := string(ev.Kv.Value)
node.processCreate(key, msg)
mService.processCreate(key, msg)
} else if ev.IsModify() {
key := string(ev.Kv.Key)
msg := string(ev.Kv.Value)
node.processModify(key, msg)
mService.processModify(key, msg)
} else if ev.Type == mvccpb.DELETE {
key := string(ev.Kv.Key)
node.processDelete(key)
mService.processDelete(key)
} else {
println("Unrecognized etcd msg!")
}
@ -250,57 +304,30 @@ func (node *QueryNode) processResp(resp clientv3.WatchResponse) error {
return nil
}
func (node *QueryNode) loadCollections() error {
keys, values, err := node.kvBase.LoadWithPrefix(CollectionPrefix)
func (mService *metaService) loadCollections() error {
keys, values, err := mService.kvBase.LoadWithPrefix(CollectionPrefix)
if err != nil {
return err
}
for i := range keys {
objID := GetCollectionObjID(keys[i])
node.processCollectionCreate(objID, values[i])
mService.processCollectionCreate(objID, values[i])
}
return nil
}
func (node *QueryNode) loadSegments() error {
keys, values, err := node.kvBase.LoadWithPrefix(SegmentPrefix)
func (mService *metaService) loadSegments() error {
keys, values, err := mService.kvBase.LoadWithPrefix(SegmentPrefix)
if err != nil {
return err
}
for i := range keys {
objID := GetSegmentObjID(keys[i])
node.processSegmentCreate(objID, values[i])
mService.processSegmentCreate(objID, values[i])
}
return nil
}
func (node *QueryNode) InitFromMeta() error {
//pass
etcdAddr := "http://"
etcdAddr += conf.Config.Etcd.Address
etcdPort := conf.Config.Etcd.Port
etcdAddr = etcdAddr + ":" + strconv.FormatInt(int64(etcdPort), 10)
cli, _ := clientv3.New(clientv3.Config{
Endpoints: []string{etcdAddr},
DialTimeout: 5 * time.Second,
})
//defer cli.Close()
node.kvBase = kv.NewEtcdKV(cli, conf.Config.Etcd.Rootpath)
node.loadCollections()
node.loadSegments()
return nil
}
func (node *QueryNode) RunMetaService(wg *sync.WaitGroup) {
//node.InitFromMeta()
metaChan := node.kvBase.WatchWithPrefix("")
for {
select {
case <-node.ctx.Done():
wg.Done()
println("DONE!!!!!!")
return
case resp := <-metaChan:
node.processResp(resp)
}
}
}

View File

@ -0,0 +1,616 @@
package reader
//import (
// "context"
// "log"
// "math"
// "sync"
// "testing"
// "time"
//
// "github.com/stretchr/testify/assert"
// "github.com/zilliztech/milvus-distributed/internal/conf"
// "github.com/zilliztech/milvus-distributed/internal/master/collection"
// "github.com/zilliztech/milvus-distributed/internal/master/segment"
// "github.com/zilliztech/milvus-distributed/internal/msgclient"
// "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
//)
//
//func TestMeta_GetCollectionObjId(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// var key = "/collection/collection0"
// var collectionObjID1 = GetCollectionObjID(key)
//
// assert.Equal(t, collectionObjID1, "/collection/collection0")
//
// key = "fakeKey"
// var collectionObjID2 = GetCollectionObjID(key)
//
// assert.Equal(t, collectionObjID2, "fakeKey")
//}
//
//func TestMeta_GetSegmentObjId(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// var key = "/segment/segment0"
// var segmentObjID1 = GetSegmentObjID(key)
//
// assert.Equal(t, segmentObjID1, "/segment/segment0")
//
// key = "fakeKey"
// var segmentObjID2 = GetSegmentObjID(key)
//
// assert.Equal(t, segmentObjID2, "fakeKey")
//}
//
//func TestMeta_isCollectionObj(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// var key = "by-dev/collection/collection0"
// var b1 = isCollectionObj(key)
//
// assert.Equal(t, b1, true)
//
// key = "by-dev/segment/segment0"
// var b2 = isCollectionObj(key)
//
// assert.Equal(t, b2, false)
//}
//
//func TestMeta_isSegmentObj(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// var key = "by-dev/segment/segment0"
// var b1 = isSegmentObj(key)
//
// assert.Equal(t, b1, true)
//
// key = "by-dev/collection/collection0"
// var b2 = isSegmentObj(key)
//
// assert.Equal(t, b2, false)
//}
//
//func TestMeta_isSegmentChannelRangeInQueryNodeChannelRange(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// var s = segment.Segment{
// SegmentID: int64(0),
// CollectionID: int64(0),
// PartitionTag: "partition0",
// ChannelStart: 0,
// ChannelEnd: 128,
// OpenTimeStamp: uint64(0),
// CloseTimeStamp: uint64(math.MaxUint64),
// CollectionName: "collection0",
// Rows: int64(0),
// }
//
// var b = isSegmentChannelRangeInQueryNodeChannelRange(&s)
// assert.Equal(t, b, true)
//
// s = segment.Segment{
// SegmentID: int64(0),
// CollectionID: int64(0),
// PartitionTag: "partition0",
// ChannelStart: 128,
// ChannelEnd: 256,
// OpenTimeStamp: uint64(0),
// CloseTimeStamp: uint64(math.MaxUint64),
// CollectionName: "collection0",
// Rows: int64(0),
// }
//
// b = isSegmentChannelRangeInQueryNodeChannelRange(&s)
// assert.Equal(t, b, false)
//}
//
//func TestMeta_PrintCollectionStruct(t *testing.T) {
// var age = collection.FieldMeta{
// FieldName: "age",
// Type: schemapb.DataType_INT32,
// DIM: int64(1),
// }
//
// var vec = collection.FieldMeta{
// FieldName: "vec",
// Type: schemapb.DataType_VECTOR_FLOAT,
// DIM: int64(16),
// }
//
// var fieldMetas = []collection.FieldMeta{age, vec}
//
// var c = collection.Collection{
// ID: int64(0),
// Name: "collection0",
// CreateTime: uint64(0),
// Schema: fieldMetas,
// SegmentIDs: []int64{
// 0, 1, 2,
// },
// PartitionTags: []string{
// "partition0",
// },
// GrpcMarshalString: "",
// }
//
// printCollectionStruct(&c)
//}
//
//func TestMeta_PrintSegmentStruct(t *testing.T) {
// var s = segment.Segment{
// SegmentID: int64(0),
// CollectionID: int64(0),
// PartitionTag: "partition0",
// ChannelStart: 128,
// ChannelEnd: 256,
// OpenTimeStamp: uint64(0),
// CloseTimeStamp: uint64(math.MaxUint64),
// CollectionName: "collection0",
// Rows: int64(0),
// }
//
// printSegmentStruct(&s)
//}
//
//func TestMeta_ProcessCollectionCreate(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// id := "0"
// value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processCollectionCreate(id, value)
// c := node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//}
//
//func TestMeta_ProcessSegmentCreate(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// id := "0"
// value := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// c := node.newCollection(int64(0), "test", "")
// c.newPartition("default")
//
// node.processSegmentCreate(id, value)
// s := node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
// assert.Equal(t, s.SegmentStatus, 0)
//}
//
//func TestMeta_ProcessCreate(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// key1 := "by-dev/collection/0"
// msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processCreate(key1, msg1)
// c := node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//
// key2 := "by-dev/segment/0"
// msg2 := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// node.processCreate(key2, msg2)
// s := node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
// assert.Equal(t, s.SegmentStatus, 0)
//}
//
//func TestMeta_ProcessSegmentModify(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// id := "0"
// value := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// var c = node.newCollection(int64(0), "test", "")
// c.newPartition("default")
//
// node.processSegmentCreate(id, value)
// var s = node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
// assert.Equal(t, s.SegmentStatus, 0)
//
// newValue := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177888," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// node.processSegmentModify(id, newValue)
// s = node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177888))
// assert.Equal(t, s.SegmentStatus, 0)
//}
//
//func TestMeta_ProcessCollectionModify(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// id := "0"
// value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processCollectionCreate(id, value)
// var c = node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//
// // TODO: use different index for testing processCollectionModify
// newValue := "{\"id\":0,\"name\":\"test_new\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processCollectionModify(id, newValue)
// c = node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//}
//
//func TestMeta_ProcessModify(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// key1 := "by-dev/collection/0"
// msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processCreate(key1, msg1)
// c := node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//
// key2 := "by-dev/segment/0"
// msg2 := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// node.processCreate(key2, msg2)
// s := node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
// assert.Equal(t, s.SegmentStatus, 0)
//
// // modify
// // TODO: use different index for testing processCollectionModify
// msg3 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processModify(key1, msg3)
// c = node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//
// msg4 := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177888," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// node.processModify(key2, msg4)
// s = node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177888))
// assert.Equal(t, s.SegmentStatus, 0)
//}
//
//func TestMeta_ProcessSegmentDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// id := "0"
// value := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// c := node.newCollection(int64(0), "test", "")
// c.newPartition("default")
//
// node.processSegmentCreate(id, value)
// s := node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
// assert.Equal(t, s.SegmentStatus, 0)
//
// node.processSegmentDelete("0")
// mapSize := len(node.SegmentsMap)
//
// assert.Equal(t, mapSize, 0)
//}
//
//func TestMeta_ProcessCollectionDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// id := "0"
// value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processCollectionCreate(id, value)
// c := node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//
// node.processCollectionDelete(id)
// collectionsSize := len(node.Collections)
//
// assert.Equal(t, collectionsSize, 0)
//}
//
//func TestMeta_ProcessDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// key1 := "by-dev/collection/0"
// msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
// "[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
// "{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
// "\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
// "\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
// "field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
// "field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
// "dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
// "6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
//
// node.processCreate(key1, msg1)
// c := node.Collections[0]
//
// assert.Equal(t, c.CollectionName, "test")
// assert.Equal(t, c.CollectionID, uint64(0))
//
// key2 := "by-dev/segment/0"
// msg2 := "{\"segment_id\":0,\"collection_id\":0," +
// "\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
// "\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
// "\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
//
// node.processCreate(key2, msg2)
// s := node.SegmentsMap[int64(0)]
//
// assert.Equal(t, s.SegmentID, int64(0))
// assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
// assert.Equal(t, s.SegmentStatus, 0)
//
// node.processDelete(key1)
// collectionsSize := len(node.Collections)
//
// assert.Equal(t, collectionsSize, 0)
//
// mapSize := len(node.SegmentsMap)
//
// assert.Equal(t, mapSize, 0)
//}
//
//func TestMeta_ProcessResp(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// err := node.InitFromMeta()
// assert.Nil(t, err)
//
// metaChan := node.kvBase.WatchWithPrefix("")
//
// select {
// case <-node.ctx.Done():
// return
// case resp := <-metaChan:
// _ = node.processResp(resp)
// }
//}
//
//func TestMeta_LoadCollections(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// err := node.InitFromMeta()
// assert.Nil(t, err)
//
// err2 := node.loadCollections()
// assert.Nil(t, err2)
//}
//
//func TestMeta_LoadSegments(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// err := node.InitFromMeta()
// assert.Nil(t, err)
//
// err2 := node.loadSegments()
// assert.Nil(t, err2)
//}
//
//func TestMeta_InitFromMeta(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// err := node.InitFromMeta()
// assert.Nil(t, err)
//}
//
//func TestMeta_RunMetaService(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// node := CreateQueryNode(ctx, 0, 0, nil)
//
// wg := sync.WaitGroup{}
// err := node.InitFromMeta()
//
// if err != nil {
// log.Printf("Init query node from meta failed")
// return
// }
//
// wg.Add(1)
// go node.RunMetaService(&wg)
// wg.Wait()
//
// node.Close()
//}

View File

@ -1,616 +0,0 @@
package reader
import (
"context"
"log"
"math"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/master/segment"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
)
func TestMeta_GetCollectionObjId(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "/collection/collection0"
var collectionObjID1 = GetCollectionObjID(key)
assert.Equal(t, collectionObjID1, "/collection/collection0")
key = "fakeKey"
var collectionObjID2 = GetCollectionObjID(key)
assert.Equal(t, collectionObjID2, "fakeKey")
}
func TestMeta_GetSegmentObjId(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "/segment/segment0"
var segmentObjID1 = GetSegmentObjID(key)
assert.Equal(t, segmentObjID1, "/segment/segment0")
key = "fakeKey"
var segmentObjID2 = GetSegmentObjID(key)
assert.Equal(t, segmentObjID2, "fakeKey")
}
func TestMeta_isCollectionObj(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "by-dev/collection/collection0"
var b1 = isCollectionObj(key)
assert.Equal(t, b1, true)
key = "by-dev/segment/segment0"
var b2 = isCollectionObj(key)
assert.Equal(t, b2, false)
}
func TestMeta_isSegmentObj(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "by-dev/segment/segment0"
var b1 = isSegmentObj(key)
assert.Equal(t, b1, true)
key = "by-dev/collection/collection0"
var b2 = isSegmentObj(key)
assert.Equal(t, b2, false)
}
func TestMeta_isSegmentChannelRangeInQueryNodeChannelRange(t *testing.T) {
conf.LoadConfig("config.yaml")
var s = segment.Segment{
SegmentID: int64(0),
CollectionID: int64(0),
PartitionTag: "partition0",
ChannelStart: 0,
ChannelEnd: 128,
OpenTimeStamp: uint64(0),
CloseTimeStamp: uint64(math.MaxUint64),
CollectionName: "collection0",
Rows: int64(0),
}
var b = isSegmentChannelRangeInQueryNodeChannelRange(&s)
assert.Equal(t, b, true)
s = segment.Segment{
SegmentID: int64(0),
CollectionID: int64(0),
PartitionTag: "partition0",
ChannelStart: 128,
ChannelEnd: 256,
OpenTimeStamp: uint64(0),
CloseTimeStamp: uint64(math.MaxUint64),
CollectionName: "collection0",
Rows: int64(0),
}
b = isSegmentChannelRangeInQueryNodeChannelRange(&s)
assert.Equal(t, b, false)
}
func TestMeta_PrintCollectionStruct(t *testing.T) {
var age = collection.FieldMeta{
FieldName: "age",
Type: schemapb.DataType_INT32,
DIM: int64(1),
}
var vec = collection.FieldMeta{
FieldName: "vec",
Type: schemapb.DataType_VECTOR_FLOAT,
DIM: int64(16),
}
var fieldMetas = []collection.FieldMeta{age, vec}
var c = collection.Collection{
ID: int64(0),
Name: "collection0",
CreateTime: uint64(0),
Schema: fieldMetas,
SegmentIDs: []int64{
0, 1, 2,
},
PartitionTags: []string{
"partition0",
},
GrpcMarshalString: "",
}
printCollectionStruct(&c)
}
func TestMeta_PrintSegmentStruct(t *testing.T) {
var s = segment.Segment{
SegmentID: int64(0),
CollectionID: int64(0),
PartitionTag: "partition0",
ChannelStart: 128,
ChannelEnd: 256,
OpenTimeStamp: uint64(0),
CloseTimeStamp: uint64(math.MaxUint64),
CollectionName: "collection0",
Rows: int64(0),
}
printSegmentStruct(&s)
}
func TestMeta_ProcessCollectionCreate(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionCreate(id, value)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
}
func TestMeta_ProcessSegmentCreate(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
c := node.NewCollection(int64(0), "test", "")
c.NewPartition("default")
node.processSegmentCreate(id, value)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessCreate(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
key1 := "by-dev/collection/0"
msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCreate(key1, msg1)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
key2 := "by-dev/segment/0"
msg2 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processCreate(key2, msg2)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessSegmentModify(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
var c = node.NewCollection(int64(0), "test", "")
c.NewPartition("default")
node.processSegmentCreate(id, value)
var s = node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
newValue := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177888," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processSegmentModify(id, newValue)
s = node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177888))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessCollectionModify(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionCreate(id, value)
var c = node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
// TODO: use different index for testing processCollectionModify
newValue := "{\"id\":0,\"name\":\"test_new\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionModify(id, newValue)
c = node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
}
func TestMeta_ProcessModify(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
key1 := "by-dev/collection/0"
msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCreate(key1, msg1)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
key2 := "by-dev/segment/0"
msg2 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processCreate(key2, msg2)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
// modify
// TODO: use different index for testing processCollectionModify
msg3 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processModify(key1, msg3)
c = node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
msg4 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177888," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processModify(key2, msg4)
s = node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177888))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessSegmentDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
c := node.NewCollection(int64(0), "test", "")
c.NewPartition("default")
node.processSegmentCreate(id, value)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
node.processSegmentDelete("0")
mapSize := len(node.SegmentsMap)
assert.Equal(t, mapSize, 0)
}
func TestMeta_ProcessCollectionDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionCreate(id, value)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
node.processCollectionDelete(id)
collectionsSize := len(node.Collections)
assert.Equal(t, collectionsSize, 0)
}
func TestMeta_ProcessDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
key1 := "by-dev/collection/0"
msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCreate(key1, msg1)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
key2 := "by-dev/segment/0"
msg2 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processCreate(key2, msg2)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentID, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
node.processDelete(key1)
collectionsSize := len(node.Collections)
assert.Equal(t, collectionsSize, 0)
mapSize := len(node.SegmentsMap)
assert.Equal(t, mapSize, 0)
}
func TestMeta_ProcessResp(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
metaChan := node.kvBase.WatchWithPrefix("")
select {
case <-node.ctx.Done():
return
case resp := <-metaChan:
_ = node.processResp(resp)
}
}
func TestMeta_LoadCollections(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
err2 := node.loadCollections()
assert.Nil(t, err2)
}
func TestMeta_LoadSegments(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
err2 := node.loadSegments()
assert.Nil(t, err2)
}
func TestMeta_InitFromMeta(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
}
func TestMeta_RunMetaService(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
node := CreateQueryNode(ctx, 0, 0, nil)
wg := sync.WaitGroup{}
err := node.InitFromMeta()
if err != nil {
log.Printf("Init query node from meta failed")
return
}
wg.Add(1)
go node.RunMetaService(&wg)
wg.Wait()
node.Close()
}

View File

@ -18,6 +18,8 @@ func (msNode *msgStreamNode) Name() string {
}
func (msNode *msgStreamNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do msgStreamNode operation")
if len(in) != 1 {
log.Println("Invalid operate message input in msgStreamNode")
// TODO: add error handling
@ -46,6 +48,7 @@ func (msNode *msgStreamNode) Operate(in []*Msg) []*Msg {
log.Println("Non supporting message type:", (*msg).Type())
}
}
var res Msg = &dmMsg
return []*Msg{&res}
}

View File

@ -19,10 +19,10 @@ type Partition struct {
Segments []*Segment
}
func (p *Partition) NewSegment(segmentID int64) *Segment {
func (p *Partition) newSegment(segmentID int64) *Segment {
/*
CSegmentBase
NewSegment(CPartition partition, unsigned long segment_id);
newSegment(CPartition partition, unsigned long segment_id);
*/
segmentPtr := C.NewSegment(p.PartitionPtr, C.ulong(segmentID))
@ -31,10 +31,10 @@ func (p *Partition) NewSegment(segmentID int64) *Segment {
return newSegment
}
func (p *Partition) DeleteSegment(node *QueryNode, segment *Segment) {
func (p *Partition) deleteSegment(node *QueryNode, segment *Segment) {
/*
void
DeleteSegment(CSegmentBase segment);
deleteSegment(CSegmentBase segment);
*/
cPtr := segment.SegmentPtr
C.DeleteSegment(cPtr)

View File

@ -9,16 +9,17 @@ import (
func TestPartition_NewSegment(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.NewSegment(0)
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, collection.CollectionID, int64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, node.Collections[0].Partitions[0].Segments[0].SegmentID, int64(0))
@ -27,22 +28,23 @@ func TestPartition_NewSegment(t *testing.T) {
assert.Equal(t, len(node.Collections[0].Partitions[0].Segments), 1)
assert.Equal(t, segment.SegmentID, int64(0))
assert.Equal(t, node.FoundSegmentBySegmentID(int64(0)), true)
assert.Equal(t, node.foundSegmentBySegmentID(int64(0)), true)
}
func TestPartition_DeleteSegment(t *testing.T) {
// 1. Construct node, collection, partition and segment
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.NewSegment(0)
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, collection.CollectionID, int64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, node.Collections[0].Partitions[0].Segments[0].SegmentID, int64(0))
@ -53,10 +55,10 @@ func TestPartition_DeleteSegment(t *testing.T) {
assert.Equal(t, segment.SegmentID, int64(0))
// 2. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
partition.deleteSegment(node, segment)
assert.Equal(t, len(collection.Partitions), 1)
assert.Equal(t, len(node.Collections), 1)
assert.Equal(t, len(node.Collections[0].Partitions[0].Segments), 0)
assert.Equal(t, node.FoundSegmentBySegmentID(int64(0)), false)
assert.Equal(t, node.foundSegmentBySegmentID(int64(0)), false)
}

View File

@ -15,179 +15,75 @@ import "C"
import (
"context"
"time"
"errors"
"strconv"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
type Timestamp = typeutil.Timestamp
type QueryNodeDataBuffer struct {
InsertDeleteBuffer []*msgPb.InsertOrDeleteMsg
SearchBuffer []*msgPb.SearchMsg
validInsertDeleteBuffer []bool
validSearchBuffer []bool
}
type QueryInfo struct {
NumQueries int64 `json:"num_queries"`
TopK int `json:"topK"`
FieldName string `json:"field_name"`
}
type MsgCounter struct {
InsertCounter int64
InsertTime time.Time
DeleteCounter int64
DeleteTime time.Time
SearchCounter int64
SearchTime time.Time
}
type InsertLog struct {
MsgLength int
DurationInMilliseconds int64
InsertTime time.Time
NumSince int64
Speed float64
}
type QueryNode struct {
// context
ctx context.Context
QueryNodeID uint64
Collections []*Collection
SegmentsMap map[int64]*Segment
messageClient *msgclient.ReaderMessageClient
queryNodeTimeSync *QueryNodeTime
buffer QueryNodeDataBuffer
deletePreprocessData DeletePreprocessData
deleteData DeleteData
insertData InsertData
kvBase *kv.EtcdKV
msgCounter *MsgCounter
InsertLogs []InsertLog
QueryNodeID uint64
pulsarURL string
Collections []*Collection
SegmentsMap map[int64]*Segment
queryNodeTime *QueryNodeTime
manipulationService *manipulationService
metaService *metaService
searchService *searchService
segmentService *segmentService
}
func NewQueryNode(ctx context.Context, queryNodeID uint64, timeSync uint64) *QueryNode {
mc := msgclient.ReaderMessageClient{}
queryNodeTimeSync := &QueryNodeTime{
ReadTimeSyncMin: timeSync,
ReadTimeSyncMax: timeSync,
WriteTimeSync: timeSync,
ServiceTimeSync: timeSync,
TSOTimeSync: timeSync,
}
func NewQueryNode(ctx context.Context, queryNodeID uint64, pulsarURL string) *QueryNode {
queryNodeTimeSync := &QueryNodeTime{}
segmentsMap := make(map[int64]*Segment)
buffer := QueryNodeDataBuffer{
InsertDeleteBuffer: make([]*msgPb.InsertOrDeleteMsg, 0),
SearchBuffer: make([]*msgPb.SearchMsg, 0),
validInsertDeleteBuffer: make([]bool, 0),
validSearchBuffer: make([]bool, 0),
}
msgCounter := MsgCounter{
InsertCounter: 0,
DeleteCounter: 0,
SearchCounter: 0,
}
return &QueryNode{
ctx: ctx,
QueryNodeID: queryNodeID,
Collections: nil,
SegmentsMap: segmentsMap,
messageClient: &mc,
queryNodeTimeSync: queryNodeTimeSync,
buffer: buffer,
msgCounter: &msgCounter,
ctx: ctx,
QueryNodeID: queryNodeID,
pulsarURL: pulsarURL,
Collections: nil,
SegmentsMap: segmentsMap,
queryNodeTime: queryNodeTimeSync,
manipulationService: nil,
metaService: nil,
searchService: nil,
segmentService: nil,
}
}
func (node *QueryNode) Start() {
node.manipulationService = newManipulationService(node.ctx, node, node.pulsarURL)
node.searchService = newSearchService(node.ctx, node, node.pulsarURL)
node.metaService = newMetaService(node.ctx, node)
node.segmentService = newSegmentService(node.ctx, node, node.pulsarURL)
go node.manipulationService.start()
go node.searchService.start()
go node.metaService.start()
node.segmentService.start()
}
func (node *QueryNode) Close() {
if node.messageClient != nil {
node.messageClient.Close()
}
if node.kvBase != nil {
node.kvBase.Close()
}
// TODO: close services
}
func CreateQueryNode(ctx context.Context, queryNodeID uint64, timeSync uint64, mc *msgclient.ReaderMessageClient) *QueryNode {
queryNodeTimeSync := &QueryNodeTime{
ReadTimeSyncMin: timeSync,
ReadTimeSyncMax: timeSync,
WriteTimeSync: timeSync,
ServiceTimeSync: timeSync,
TSOTimeSync: timeSync,
}
segmentsMap := make(map[int64]*Segment)
buffer := QueryNodeDataBuffer{
InsertDeleteBuffer: make([]*msgPb.InsertOrDeleteMsg, 0),
SearchBuffer: make([]*msgPb.SearchMsg, 0),
validInsertDeleteBuffer: make([]bool, 0),
validSearchBuffer: make([]bool, 0),
}
msgCounter := MsgCounter{
InsertCounter: 0,
InsertTime: time.Now(),
DeleteCounter: 0,
DeleteTime: time.Now(),
SearchCounter: 0,
SearchTime: time.Now(),
}
return &QueryNode{
ctx: ctx,
QueryNodeID: queryNodeID,
Collections: nil,
SegmentsMap: segmentsMap,
messageClient: mc,
queryNodeTimeSync: queryNodeTimeSync,
buffer: buffer,
msgCounter: &msgCounter,
InsertLogs: make([]InsertLog, 0),
}
}
func (node *QueryNode) QueryNodeDataInit() {
deletePreprocessData := DeletePreprocessData{
deleteRecords: make([]*DeleteRecord, 0),
count: 0,
}
deleteData := DeleteData{
deleteIDs: make(map[int64][]int64),
deleteTimestamps: make(map[int64][]uint64),
deleteOffset: make(map[int64]int64),
}
insertData := InsertData{
insertIDs: make(map[int64][]int64),
insertTimestamps: make(map[int64][]uint64),
// insertRecords: make(map[int64][][]byte),
insertOffset: make(map[int64]int64),
}
node.deletePreprocessData = deletePreprocessData
node.deleteData = deleteData
node.insertData = insertData
}
func (node *QueryNode) NewCollection(collectionID int64, collectionName string, schemaConfig string) *Collection {
func (node *QueryNode) newCollection(collectionID int64, collectionName string, schemaConfig string) *Collection {
/*
CCollection
newCollection(const char* collection_name, const char* schema_conf);
*/
cName := C.CString(collectionName)
cSchema := C.CString(schemaConfig)
collection := C.NewCollection(cName, cSchema)
@ -198,10 +94,10 @@ func (node *QueryNode) NewCollection(collectionID int64, collectionName string,
return newCollection
}
func (node *QueryNode) DeleteCollection(collection *Collection) {
func (node *QueryNode) deleteCollection(collection *Collection) {
/*
void
DeleteCollection(CCollection collection);
deleteCollection(CCollection collection);
*/
cPtr := collection.CollectionPtr
C.DeleteCollection(cPtr)
@ -222,3 +118,64 @@ func (node *QueryNode) DeleteCollection(collection *Collection) {
node.Collections = tmpCollections
}
/************************************** util functions ***************************************/
// Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs
//func (node *QueryNode) GetKey2Segments() (*[]int64, *[]uint64, *[]int64) {
// var entityIDs = make([]int64, 0)
// var timestamps = make([]uint64, 0)
// var segmentIDs = make([]int64, 0)
//
// var key2SegMsg = node.messageClient.Key2SegMsg
// for _, msg := range key2SegMsg {
// if msg.SegmentId == nil {
// segmentIDs = append(segmentIDs, -1)
// entityIDs = append(entityIDs, msg.Uid)
// timestamps = append(timestamps, msg.Timestamp)
// } else {
// for _, segmentID := range msg.SegmentId {
// segmentIDs = append(segmentIDs, segmentID)
// entityIDs = append(entityIDs, msg.Uid)
// timestamps = append(timestamps, msg.Timestamp)
// }
// }
// }
//
// return &entityIDs, &timestamps, &segmentIDs
//}
func (node *QueryNode) getCollectionByID(collectionID int64) *Collection {
for _, collection := range node.Collections {
if collection.CollectionID == collectionID {
return collection
}
}
return nil
}
func (node *QueryNode) getCollectionByCollectionName(collectionName string) (*Collection, error) {
for _, collection := range node.Collections {
if collection.CollectionName == collectionName {
return collection, nil
}
}
return nil, errors.New("Cannot found collection: " + collectionName)
}
func (node *QueryNode) getSegmentBySegmentID(segmentID int64) (*Segment, error) {
targetSegment, ok := node.SegmentsMap[segmentID]
if !ok {
return nil, errors.New("cannot found segment with id = " + strconv.FormatInt(segmentID, 10))
}
return targetSegment, nil
}
func (node *QueryNode) foundSegmentBySegmentID(segmentID int64) bool {
_, ok := node.SegmentsMap[segmentID]
return ok
}

View File

@ -1,85 +1,85 @@
package reader
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
)
func TestQueryNode_CreateQueryNode(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
}
func TestQueryNode_NewQueryNode(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
node := NewQueryNode(ctx, 0, 0)
assert.NotNil(t, node)
}
func TestQueryNode_Close(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
node.Close()
}
func TestQueryNode_QueryNodeDataInit(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
node.QueryNodeDataInit()
assert.NotNil(t, node.deletePreprocessData)
assert.NotNil(t, node.insertData)
assert.NotNil(t, node.deleteData)
}
func TestQueryNode_NewCollection(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
var collection = node.NewCollection(0, "collection0", "")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, len(node.Collections), 1)
}
func TestQueryNode_DeleteCollection(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
var collection = node.NewCollection(0, "collection0", "")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, len(node.Collections), 1)
node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
}
//import (
// "context"
// "testing"
//
// "github.com/stretchr/testify/assert"
// "github.com/zilliztech/milvus-distributed/internal/conf"
//)
//
//func TestQueryNode_CreateQueryNode(t *testing.T) {
// conf.LoadConfig("config.yaml")
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// node := CreateQueryNode(ctx, 0, 0, nil)
// assert.NotNil(t, node)
//}
//
//func TestQueryNode_NewQueryNode(t *testing.T) {
// conf.LoadConfig("config.yaml")
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// node := NewQueryNode(ctx, 0, 0)
// assert.NotNil(t, node)
//}
//
//func TestQueryNode_Close(t *testing.T) {
// conf.LoadConfig("config.yaml")
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// node := CreateQueryNode(ctx, 0, 0, nil)
// assert.NotNil(t, node)
//
// node.Close()
//}
//
//func TestQueryNode_QueryNodeDataInit(t *testing.T) {
// conf.LoadConfig("config.yaml")
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// node := CreateQueryNode(ctx, 0, 0, nil)
// assert.NotNil(t, node)
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//}
//
//func TestQueryNode_NewCollection(t *testing.T) {
// conf.LoadConfig("config.yaml")
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// node := CreateQueryNode(ctx, 0, 0, nil)
// assert.NotNil(t, node)
//
// var collection = node.newCollection(0, "collection0", "")
//
// assert.Equal(t, collection.CollectionName, "collection0")
// assert.Equal(t, len(node.Collections), 1)
//}
//
//func TestQueryNode_DeleteCollection(t *testing.T) {
// conf.LoadConfig("config.yaml")
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// node := CreateQueryNode(ctx, 0, 0, nil)
// assert.NotNil(t, node)
//
// var collection = node.newCollection(0, "collection0", "")
//
// assert.Equal(t, collection.CollectionName, "collection0")
// assert.Equal(t, len(node.Collections), 1)
//
// node.deleteCollection(collection)
//
// assert.Equal(t, len(node.Collections), 0)
//}

View File

@ -2,130 +2,10 @@ package reader
import (
"context"
"fmt"
"log"
"sync"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
)
func StartQueryNode(ctx context.Context, pulsarURL string) {
mc := msgclient.ReaderMessageClient{}
mc.InitClient(ctx, pulsarURL)
node := NewQueryNode(ctx, 0, pulsarURL)
mc.ReceiveMessage()
qn := CreateQueryNode(ctx, 0, 0, &mc)
// Segments Services
go qn.SegmentStatisticService()
wg := sync.WaitGroup{}
err := qn.InitFromMeta()
if err != nil {
log.Printf("Init query node from meta failed")
return
}
wg.Add(3)
go qn.RunMetaService(&wg)
go qn.RunInsertDelete(&wg)
go qn.RunSearch(&wg)
wg.Wait()
qn.Close()
}
func (node *QueryNode) RunInsertDelete(wg *sync.WaitGroup) {
const Debug = true
const CountInsertMsgBaseline = 1000 * 1000
var BaselineCounter int64 = 0
if Debug {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
default:
var msgLen = node.PrepareBatchMsg()
var timeRange = TimeRange{node.messageClient.TimeSyncStart(), node.messageClient.TimeSyncEnd()}
assert.NotEqual(nil, 0, timeRange.timestampMin)
assert.NotEqual(nil, 0, timeRange.timestampMax)
if node.msgCounter.InsertCounter/CountInsertMsgBaseline != BaselineCounter {
node.WriteQueryLog()
BaselineCounter = node.msgCounter.InsertCounter / CountInsertMsgBaseline
}
if msgLen[0] == 0 && len(node.buffer.InsertDeleteBuffer) <= 0 {
node.queryNodeTimeSync.updateSearchServiceTime(timeRange)
continue
}
node.QueryNodeDataInit()
node.MessagesPreprocess(node.messageClient.InsertOrDeleteMsg, timeRange)
//fmt.Println("MessagesPreprocess Done")
node.WriterDelete()
node.PreInsertAndDelete()
//fmt.Println("PreInsertAndDelete Done")
node.DoInsertAndDelete()
//fmt.Println("DoInsertAndDelete Done")
node.queryNodeTimeSync.updateSearchServiceTime(timeRange)
}
}
} else {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
default:
var msgLen = node.PrepareBatchMsg()
var timeRange = TimeRange{node.messageClient.TimeSyncStart(), node.messageClient.TimeSyncEnd()}
assert.NotEqual(nil, 0, timeRange.timestampMin)
assert.NotEqual(nil, 0, timeRange.timestampMax)
if msgLen[0] == 0 && len(node.buffer.InsertDeleteBuffer) <= 0 {
node.queryNodeTimeSync.updateSearchServiceTime(timeRange)
continue
}
node.QueryNodeDataInit()
node.MessagesPreprocess(node.messageClient.InsertOrDeleteMsg, timeRange)
//fmt.Println("MessagesPreprocess Done")
node.WriterDelete()
node.PreInsertAndDelete()
//fmt.Println("PreInsertAndDelete Done")
node.DoInsertAndDelete()
//fmt.Println("DoInsertAndDelete Done")
node.queryNodeTimeSync.updateSearchServiceTime(timeRange)
}
}
}
}
func (node *QueryNode) RunSearch(wg *sync.WaitGroup) {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
case msg := <-node.messageClient.GetSearchChan():
node.messageClient.SearchMsg = node.messageClient.SearchMsg[:0]
node.messageClient.SearchMsg = append(node.messageClient.SearchMsg, msg)
//for {
//if node.messageClient.SearchMsg[0].Timestamp < node.queryNodeTimeSync.ServiceTimeSync {
var status = node.Search(node.messageClient.SearchMsg)
fmt.Println("Do Search done")
if status.ErrorCode != 0 {
fmt.Println("Search Failed")
node.PublishFailedSearchResult()
}
//break
//}
//}
default:
}
}
node.Start()
}

View File

@ -1,96 +1,96 @@
package reader
import (
"context"
"strconv"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
)
const ctxTimeInMillisecond = 10
// NOTE: start pulsar and etcd before test
func TestReader_startQueryNode(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
StartQueryNode(ctx, pulsarAddr)
// To make sure to get here
assert.Equal(t, 0, 0)
}
// NOTE: start pulsar before test
func TestReader_RunInsertDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc)
wg := sync.WaitGroup{}
wg.Add(1)
go node.RunInsertDelete(&wg)
wg.Wait()
node.Close()
// To make sure to get here
assert.Equal(t, 0, 0)
}
// NOTE: start pulsar before test
func TestReader_RunSearch(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc)
wg := sync.WaitGroup{}
wg.Add(1)
go node.RunSearch(&wg)
wg.Wait()
node.Close()
// To make sure to get here
assert.Equal(t, 0, 0)
}
//import (
// "context"
// "strconv"
// "sync"
// "testing"
// "time"
//
// "github.com/stretchr/testify/assert"
// "github.com/zilliztech/milvus-distributed/internal/conf"
// "github.com/zilliztech/milvus-distributed/internal/msgclient"
//)
//
//const ctxTimeInMillisecond = 10
//
//// NOTE: start pulsar and etcd before test
//func TestReader_startQueryNode(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// StartQueryNode(ctx, pulsarAddr)
//
// // To make sure to get here
// assert.Equal(t, 0, 0)
//}
//
//// NOTE: start pulsar before test
//func TestReader_RunInsertDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// mc.ReceiveMessage()
//
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// wg := sync.WaitGroup{}
//
// wg.Add(1)
// go node.RunInsertDelete(&wg)
// wg.Wait()
//
// node.Close()
//
// // To make sure to get here
// assert.Equal(t, 0, 0)
//}
//
//// NOTE: start pulsar before test
//func TestReader_RunSearch(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// mc.ReceiveMessage()
//
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// wg := sync.WaitGroup{}
//
// wg.Add(1)
// go node.RunSearch(&wg)
// wg.Wait()
//
// node.Close()
//
// // To make sure to get here
// assert.Equal(t, 0, 0)
//}

View File

@ -1,13 +1,5 @@
package reader
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
type ResultEntityIds []UniqueID
type SearchResult struct {
@ -15,27 +7,28 @@ type SearchResult struct {
ResultDistances []float32
}
func (node *QueryNode) PublishSearchResult(results *msgpb.QueryResult) commonpb.Status {
var ctx = context.Background()
node.messageClient.SendResult(ctx, *results, results.ProxyId)
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (node *QueryNode) PublishFailedSearchResult() commonpb.Status {
var results = msgpb.QueryResult{}
var ctx = context.Background()
node.messageClient.SendResult(ctx, results, results.ProxyId)
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (node *QueryNode) PublicStatistic(statisticData *[]internalpb.SegmentStatistics) commonpb.Status {
var ctx = context.Background()
node.messageClient.SendSegmentsStatistic(ctx, statisticData)
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
//
//func (node *QueryNode) PublishSearchResult(results *msgpb.QueryResult) commonpb.Status {
// var ctx = context.Background()
//
// node.messageClient.SendResult(ctx, *results, results.ProxyId)
//
// return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
//}
//
//func (node *QueryNode) PublishFailedSearchResult() commonpb.Status {
// var results = msgpb.QueryResult{}
//
// var ctx = context.Background()
//
// node.messageClient.SendResult(ctx, results, results.ProxyId)
// return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
//}
//
//func (node *QueryNode) PublicStatistic(statisticData *[]internalpb.SegmentStatistics) commonpb.Status {
// var ctx = context.Background()
//
// node.messageClient.SendSegmentsStatistic(ctx, statisticData)
//
// return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
//}

View File

@ -1,127 +1,127 @@
package reader
import (
"context"
"strconv"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
//masterPb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
// NOTE: start pulsar before test
func TestResult_PublishSearchResult(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment
const N = 10
var entityIDs = msgPb.Entities{
Ids: make([]int64, N),
}
var result = msgPb.QueryResult{
Entities: &entityIDs,
Distances: make([]float32, N),
}
for i := 0; i < N; i++ {
result.Entities.Ids = append(result.Entities.Ids, int64(i))
result.Distances = append(result.Distances, float32(i))
}
status := node.PublishSearchResult(&result)
assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close()
}
// NOTE: start pulsar before test
func TestResult_PublishFailedSearchResult(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment
status := node.PublishFailedSearchResult()
assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close()
}
// NOTE: start pulsar before test
func TestResult_PublicStatistic(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment
var statisticData = make([]internalpb.SegmentStatistics, 0)
for segmentID, segment := range node.SegmentsMap {
currentMemSize := segment.GetMemSize()
stat := internalpb.SegmentStatistics{
SegmentId: segmentID,
MemorySize: currentMemSize,
}
statisticData = append(statisticData, stat)
}
status := node.PublicStatistic(&statisticData)
assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close()
}
//import (
// "context"
// "strconv"
// "testing"
// "time"
//
// "github.com/stretchr/testify/assert"
// "github.com/zilliztech/milvus-distributed/internal/conf"
// "github.com/zilliztech/milvus-distributed/internal/msgclient"
//
// //masterPb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
// "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
// msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
//)
//
//// NOTE: start pulsar before test
//func TestResult_PublishSearchResult(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// // Construct node, collection, partition and segment
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
// node.SegmentsMap[0] = segment
//
// const N = 10
// var entityIDs = msgPb.Entities{
// Ids: make([]int64, N),
// }
// var result = msgPb.QueryResult{
// Entities: &entityIDs,
// Distances: make([]float32, N),
// }
// for i := 0; i < N; i++ {
// result.Entities.Ids = append(result.Entities.Ids, int64(i))
// result.Distances = append(result.Distances, float32(i))
// }
//
// status := node.PublishSearchResult(&result)
// assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
//
// node.Close()
//}
//
//// NOTE: start pulsar before test
//func TestResult_PublishFailedSearchResult(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// // Construct node, collection, partition and segment
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
// node.SegmentsMap[0] = segment
//
// status := node.PublishFailedSearchResult()
// assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
//
// node.Close()
//}
//
//// NOTE: start pulsar before test
//func TestResult_PublicStatistic(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// // Construct node, collection, partition and segment
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
// node.SegmentsMap[0] = segment
//
// var statisticData = make([]internalpb.SegmentStatistics, 0)
//
// for segmentID, segment := range node.SegmentsMap {
// currentMemSize := segment.getMemSize()
// stat := internalpb.SegmentStatistics{
// SegmentId: segmentID,
// MemorySize: currentMemSize,
// }
// statisticData = append(statisticData, stat)
// }
//
// status := node.PublicStatistic(&statisticData)
// assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
//
// node.Close()
//}

View File

@ -1,120 +0,0 @@
package reader
import (
"fmt"
"sort"
"github.com/zilliztech/milvus-distributed/internal/conf"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status {
type SearchResultTmp struct {
ResultID int64
ResultDistance float32
}
node.msgCounter.SearchCounter += int64(len(searchMessages))
// Traverse all messages in the current messageClient.
// TODO: Do not receive batched search requests
for _, msg := range searchMessages {
var clientID = msg.ClientId
var searchTimestamp = msg.Timestamp
// ServiceTimeSync update by TimeSync, which is get from proxy.
// Proxy send this timestamp per `conf.Config.Timesync.Interval` milliseconds.
// However, timestamp of search request (searchTimestamp) is precision time.
// So the ServiceTimeSync is always less than searchTimestamp.
// Here, we manually make searchTimestamp's logic time minus `conf.Config.Timesync.Interval` milliseconds.
// Which means `searchTimestamp.logicTime = searchTimestamp.logicTime - conf.Config.Timesync.Interval`.
var logicTimestamp = searchTimestamp << 46 >> 46
searchTimestamp = (searchTimestamp>>18-uint64(conf.Config.Timesync.Interval+600))<<18 + logicTimestamp
var vector = msg.Records
// We now only the first Json is valid.
var queryJSON = msg.Json[0]
// 1. Timestamp check
// TODO: return or wait? Or adding graceful time
if searchTimestamp > node.queryNodeTimeSync.ServiceTimeSync {
fmt.Println("Invalid query time, timestamp = ", searchTimestamp>>18, ", SearchTimeSync = ", node.queryNodeTimeSync.ServiceTimeSync>>18)
return msgPb.Status{ErrorCode: 1}
}
// 2. Get query information from query json
query := node.QueryJSON2Info(&queryJSON)
// 2d slice for receiving multiple queries's results
var resultsTmp = make([][]SearchResultTmp, query.NumQueries)
for i := 0; i < int(query.NumQueries); i++ {
resultsTmp[i] = make([]SearchResultTmp, 0)
}
// 3. Do search in all segments
for _, segment := range node.SegmentsMap {
if segment.GetRowCount() <= 0 {
// Skip empty segment
continue
}
//fmt.Println("Search in segment:", segment.SegmentID, ",segment rows:", segment.GetRowCount())
var res, err = segment.SegmentSearch(query, searchTimestamp, vector)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
for i := 0; i < int(query.NumQueries); i++ {
for j := i * query.TopK; j < (i+1)*query.TopK; j++ {
resultsTmp[i] = append(resultsTmp[i], SearchResultTmp{
ResultID: res.ResultIds[j],
ResultDistance: res.ResultDistances[j],
})
}
}
}
// 4. Reduce results
for _, rTmp := range resultsTmp {
sort.Slice(rTmp, func(i, j int) bool {
return rTmp[i].ResultDistance < rTmp[j].ResultDistance
})
}
for _, rTmp := range resultsTmp {
if len(rTmp) > query.TopK {
rTmp = rTmp[:query.TopK]
}
}
var entities = msgPb.Entities{
Ids: make([]int64, 0),
}
var results = msgPb.QueryResult{
Status: &msgPb.Status{
ErrorCode: 0,
},
Entities: &entities,
Distances: make([]float32, 0),
QueryId: msg.Uid,
ProxyId: clientID,
}
for _, rTmp := range resultsTmp {
for _, res := range rTmp {
results.Entities.Ids = append(results.Entities.Ids, res.ResultID)
results.Distances = append(results.Distances, res.ResultDistance)
results.Scores = append(results.Distances, float32(0))
}
}
// Send numQueries to RowNum.
results.RowNum = query.NumQueries
// 5. publish result to pulsar
//fmt.Println(results.Entities.Ids)
//fmt.Println(results.Distances)
node.PublishSearchResult(&results)
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}

View File

@ -2,14 +2,182 @@ package reader
import (
"context"
"encoding/json"
"fmt"
"log"
"sort"
"github.com/zilliztech/milvus-distributed/internal/conf"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
)
type searchService struct {
ctx context.Context
queryNodeTime *QueryNodeTime
msgStream *msgstream.PulsarMsgStream
ctx context.Context
pulsarURL string
node *QueryNode
msgStream *msgstream.PulsarMsgStream
}
func (ss *searchService) Start() {}
type queryInfo struct {
NumQueries int64 `json:"num_queries"`
TopK int `json:"topK"`
FieldName string `json:"field_name"`
}
func newSearchService(ctx context.Context, node *QueryNode, pulsarURL string) *searchService {
return &searchService{
ctx: ctx,
pulsarURL: pulsarURL,
node: node,
msgStream: nil,
}
}
func (ss *searchService) start() {
// TODO: init pulsar message stream
for {
select {
case <-ss.ctx.Done():
return
//case msg := <-node.messageClient.GetSearchChan():
// ss.messageClient.SearchMsg = node.messageClient.SearchMsg[:0]
// ss.messageClient.SearchMsg = append(node.messageClient.SearchMsg, msg)
// // TODO: add serviceTime check
// var status = node.search(node.messageClient.SearchMsg)
// fmt.Println("Do search done")
// if status.ErrorCode != 0 {
// fmt.Println("search Failed")
// node.PublishFailedSearchResult()
// }
}
}
}
func (ss *searchService) search(searchMessages []*msgPb.SearchMsg) msgPb.Status {
type SearchResultTmp struct {
ResultID int64
ResultDistance float32
}
// Traverse all messages in the current messageClient.
// TODO: Do not receive batched search requests
for _, msg := range searchMessages {
var clientID = msg.ClientId
var searchTimestamp = msg.Timestamp
// ServiceTimeSync update by TimeSync, which is get from proxy.
// Proxy send this timestamp per `conf.Config.Timesync.Interval` milliseconds.
// However, timestamp of search request (searchTimestamp) is precision time.
// So the ServiceTimeSync is always less than searchTimestamp.
// Here, we manually make searchTimestamp's logic time minus `conf.Config.Timesync.Interval` milliseconds.
// Which means `searchTimestamp.logicTime = searchTimestamp.logicTime - conf.Config.Timesync.Interval`.
var logicTimestamp = searchTimestamp << 46 >> 46
searchTimestamp = (searchTimestamp>>18-uint64(conf.Config.Timesync.Interval+600))<<18 + logicTimestamp
var vector = msg.Records
// We now only the first Json is valid.
var queryJSON = msg.Json[0]
// 1. Timestamp check
// TODO: return or wait? Or adding graceful time
if searchTimestamp > ss.node.queryNodeTime.ServiceTimeSync {
fmt.Println("Invalid query time, timestamp = ", searchTimestamp>>18, ", SearchTimeSync = ", ss.node.queryNodeTime.ServiceTimeSync>>18)
return msgPb.Status{ErrorCode: 1}
}
// 2. Get query information from query json
query := ss.queryJSON2Info(&queryJSON)
// 2d slice for receiving multiple queries's results
var resultsTmp = make([][]SearchResultTmp, query.NumQueries)
for i := 0; i < int(query.NumQueries); i++ {
resultsTmp[i] = make([]SearchResultTmp, 0)
}
// 3. Do search in all segments
for _, segment := range ss.node.SegmentsMap {
if segment.getRowCount() <= 0 {
// Skip empty segment
continue
}
//fmt.Println("search in segment:", segment.SegmentID, ",segment rows:", segment.getRowCount())
var res, err = segment.segmentSearch(query, searchTimestamp, vector)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
for i := 0; i < int(query.NumQueries); i++ {
for j := i * query.TopK; j < (i+1)*query.TopK; j++ {
resultsTmp[i] = append(resultsTmp[i], SearchResultTmp{
ResultID: res.ResultIds[j],
ResultDistance: res.ResultDistances[j],
})
}
}
}
// 4. Reduce results
for _, rTmp := range resultsTmp {
sort.Slice(rTmp, func(i, j int) bool {
return rTmp[i].ResultDistance < rTmp[j].ResultDistance
})
}
for _, rTmp := range resultsTmp {
if len(rTmp) > query.TopK {
rTmp = rTmp[:query.TopK]
}
}
var entities = msgPb.Entities{
Ids: make([]int64, 0),
}
var results = msgPb.QueryResult{
Status: &msgPb.Status{
ErrorCode: 0,
},
Entities: &entities,
Distances: make([]float32, 0),
QueryId: msg.Uid,
ProxyId: clientID,
}
for _, rTmp := range resultsTmp {
for _, res := range rTmp {
results.Entities.Ids = append(results.Entities.Ids, res.ResultID)
results.Distances = append(results.Distances, res.ResultDistance)
results.Scores = append(results.Distances, float32(0))
}
}
// Send numQueries to RowNum.
results.RowNum = query.NumQueries
// 5. publish result to pulsar
//fmt.Println(results.Entities.Ids)
//fmt.Println(results.Distances)
ss.publishSearchResult(&results)
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (ss *searchService) publishSearchResult(res *msgPb.QueryResult) {
// TODO: add result publish
}
func (ss *searchService) queryJSON2Info(queryJSON *string) *queryInfo {
var query queryInfo
var err = json.Unmarshal([]byte(*queryJSON), &query)
if err != nil {
log.Fatal("Unmarshal query json failed")
return nil
}
return &query
}

View File

@ -0,0 +1,140 @@
package reader
//import (
// "context"
// "encoding/binary"
// "math"
// "strconv"
// "sync"
// "testing"
//
// "github.com/stretchr/testify/assert"
// "github.com/zilliztech/milvus-distributed/internal/conf"
// "github.com/zilliztech/milvus-distributed/internal/msgclient"
// msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
//)
//
//// NOTE: start pulsar before test
//func TestSearch_Search(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
//
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// mc.ReceiveMessage()
//
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// var collection = node.newCollection(0, "collection0", "")
// _ = collection.newPartition("partition0")
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records [][]byte
// for i := 0; i < N; i++ {
// records = append(records, rawData)
// }
//
// insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
//
// for i := 0; i < msgLength; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i),
// ChannelId: 0,
// Op: msgPb.OpType_INSERT,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//
// node.MessagesPreprocess(insertDeleteMessages, timeRange)
//
// assert.Equal(t, len(node.insertData.insertIDs), msgLength)
// assert.Equal(t, len(node.insertData.insertTimestamps), msgLength)
// assert.Equal(t, len(node.insertData.insertRecords), msgLength)
// assert.Equal(t, len(node.insertData.insertOffset), 0)
//
// assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
// assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
//
// assert.Equal(t, len(node.SegmentsMap), 10)
// assert.Equal(t, len(node.Collections[0].Partitions[0].Segments), 10)
//
// node.PreInsertAndDelete()
//
// assert.Equal(t, len(node.insertData.insertOffset), msgLength)
//
// wg := sync.WaitGroup{}
// for segmentID := range node.insertData.insertRecords {
// wg.Add(1)
// go node.DoInsert(segmentID, &wg)
// }
// wg.Wait()
//
// var queryRawData = make([]float32, 0)
// for i := 0; i < DIM; i++ {
// queryRawData = append(queryRawData, float32(i))
// }
//
// var queryJSON = "{\"field_name\":\"fakevec\",\"num_queries\":1,\"topK\":10}"
// searchMsg1 := msgPb.SearchMsg{
// CollectionName: "collection0",
// Records: &msgPb.VectorRowRecord{
// FloatData: queryRawData,
// },
// PartitionTag: []string{"partition0"},
// Uid: int64(0),
// Timestamp: uint64(0),
// ClientId: int64(0),
// ExtraParams: nil,
// Json: []string{queryJSON},
// }
// searchMessages := []*msgPb.SearchMsg{&searchMsg1}
//
// node.queryNodeTime.updateSearchServiceTime(timeRange)
// assert.Equal(t, node.queryNodeTime.ServiceTimeSync, timeRange.timestampMax)
//
// status := node.search(searchMessages)
// assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
//
// node.Close()
//}

View File

@ -1,140 +0,0 @@
package reader
import (
"context"
"encoding/binary"
"math"
"strconv"
"sync"
"testing"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
// NOTE: start pulsar before test
func TestSearch_Search(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc)
var collection = node.NewCollection(0, "collection0", "")
_ = collection.NewPartition("partition0")
const msgLength = 10
const DIM = 16
const N = 3
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
var rawData []byte
for _, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
rawData = append(rawData, buf...)
}
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records [][]byte
for i := 0; i < N; i++ {
records = append(records, rawData)
}
insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
for i := 0; i < msgLength; i++ {
msg := msgPb.InsertOrDeleteMsg{
CollectionName: "collection0",
RowsData: &msgPb.RowData{
Blob: rawData,
},
Uid: int64(i),
PartitionTag: "partition0",
Timestamp: Timestamp(i + 1000),
SegmentId: UniqueID(i),
ChannelId: 0,
Op: msgPb.OpType_INSERT,
ClientId: 0,
ExtraParams: nil,
}
insertDeleteMessages = append(insertDeleteMessages, &msg)
}
timeRange := TimeRange{
timestampMin: 0,
timestampMax: math.MaxUint64,
}
node.QueryNodeDataInit()
assert.NotNil(t, node.deletePreprocessData)
assert.NotNil(t, node.insertData)
assert.NotNil(t, node.deleteData)
node.MessagesPreprocess(insertDeleteMessages, timeRange)
assert.Equal(t, len(node.insertData.insertIDs), msgLength)
assert.Equal(t, len(node.insertData.insertTimestamps), msgLength)
assert.Equal(t, len(node.insertData.insertRecords), msgLength)
assert.Equal(t, len(node.insertData.insertOffset), 0)
assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
assert.Equal(t, len(node.SegmentsMap), 10)
assert.Equal(t, len(node.Collections[0].Partitions[0].Segments), 10)
node.PreInsertAndDelete()
assert.Equal(t, len(node.insertData.insertOffset), msgLength)
wg := sync.WaitGroup{}
for segmentID := range node.insertData.insertRecords {
wg.Add(1)
go node.DoInsert(segmentID, &wg)
}
wg.Wait()
var queryRawData = make([]float32, 0)
for i := 0; i < DIM; i++ {
queryRawData = append(queryRawData, float32(i))
}
var queryJSON = "{\"field_name\":\"fakevec\",\"num_queries\":1,\"topK\":10}"
searchMsg1 := msgPb.SearchMsg{
CollectionName: "collection0",
Records: &msgPb.VectorRowRecord{
FloatData: queryRawData,
},
PartitionTag: []string{"partition0"},
Uid: UniqueID(0),
Timestamp: Timestamp(0),
ClientId: UniqueID(0),
ExtraParams: nil,
Json: []string{queryJSON},
}
searchMessages := []*msgPb.SearchMsg{&searchMsg1}
node.queryNodeTimeSync.updateSearchServiceTime(timeRange)
assert.Equal(t, node.queryNodeTimeSync.ServiceTimeSync, timeRange.timestampMax)
status := node.Search(searchMessages)
assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close()
}

View File

@ -13,7 +13,9 @@ package reader
*/
import "C"
import (
"github.com/stretchr/testify/assert"
"strconv"
"unsafe"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
@ -23,83 +25,34 @@ import (
type IntPrimaryKey = typeutil.IntPrimaryKey
const SegmentLifetime = 20000
const (
//SegmentOpened = 0
//SegmentClosed = 1
//SegmentIndexing = 2
//SegmentIndexed = 3
)
type Segment struct {
SegmentPtr C.CSegmentBase
SegmentID UniqueID
SegmentCloseTime Timestamp
LastMemSize int64
SegmentStatus int
recentlyModified bool
}
//func (s *Segment) GetStatus() int {
// /*
// bool
// IsOpened(CSegmentBase c_segment);
// */
// var isOpened = C.IsOpened(s.SegmentPtr)
// if isOpened {
// return SegmentOpened
// } else {
// return SegmentClosed
// }
//}
func (s *Segment) GetRowCount() int64 {
func (s *Segment) getRowCount() int64 {
/*
long int
GetRowCount(CSegmentBase c_segment);
getRowCount(CSegmentBase c_segment);
*/
var rowCount = C.GetRowCount(s.SegmentPtr)
return int64(rowCount)
}
func (s *Segment) GetDeletedCount() int64 {
func (s *Segment) getDeletedCount() int64 {
/*
long int
GetDeletedCount(CSegmentBase c_segment);
getDeletedCount(CSegmentBase c_segment);
*/
var deletedCount = C.GetDeletedCount(s.SegmentPtr)
return int64(deletedCount)
}
//func (s *Segment) CloseSegment(collection* Collection) error {
// /*
// int
// Close(CSegmentBase c_segment);
// */
// fmt.Println("Closing segment :", s.SegmentID)
//
// var status = C.Close(s.SegmentPtr)
// s.SegmentStatus = SegmentClosed
//
// if status != 0 {
// return errors.New("Close segment failed, error code = " + strconv.Itoa(int(status)))
// }
//
// // Build index after closing segment
// //s.SegmentStatus = SegmentIndexing
// //fmt.Println("Building index...")
// //s.BuildIndex(collection)
//
// // TODO: remove redundant segment indexed status
// // Change segment status to indexed
// //s.SegmentStatus = SegmentIndexed
// //fmt.Println("Segment closed and indexed")
//
// fmt.Println("Segment closed")
// return nil
//}
func (s *Segment) GetMemSize() int64 {
func (s *Segment) getMemSize() int64 {
/*
long int
GetMemoryUsageInBytes(CSegmentBase c_segment);
@ -110,7 +63,7 @@ func (s *Segment) GetMemSize() int64 {
}
////////////////////////////////////////////////////////////////////////////
func (s *Segment) SegmentPreInsert(numOfRecords int) int64 {
func (s *Segment) segmentPreInsert(numOfRecords int) int64 {
/*
long int
PreInsert(CSegmentBase c_segment, long int size);
@ -120,7 +73,7 @@ func (s *Segment) SegmentPreInsert(numOfRecords int) int64 {
return int64(offset)
}
func (s *Segment) SegmentPreDelete(numOfRecords int) int64 {
func (s *Segment) segmentPreDelete(numOfRecords int) int64 {
/*
long int
PreDelete(CSegmentBase c_segment, long int size);
@ -130,7 +83,7 @@ func (s *Segment) SegmentPreDelete(numOfRecords int) int64 {
return int64(offset)
}
func (s *Segment) SegmentInsert(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp, records *[]*commonpb.Blob) error {
func (s *Segment) segmentInsert(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp, records *[]*commonpb.Blob) error {
/*
int
Insert(CSegmentBase c_segment,
@ -143,42 +96,42 @@ func (s *Segment) SegmentInsert(offset int64, entityIDs *[]UniqueID, timestamps
signed long int count);
*/
// Blobs to one big blob
//var numOfRow = len(*entityIDs)
//var sizeofPerRow = len((*records)[0])
//
//assert.Equal(nil, numOfRow, len(*records))
//
//var rawData = make([]byte, numOfRow*sizeofPerRow)
//var copyOffset = 0
//for i := 0; i < len(*records); i++ {
// copy(rawData[copyOffset:], (*records)[i])
// copyOffset += sizeofPerRow
//}
//
//var cOffset = C.long(offset)
//var cNumOfRows = C.long(numOfRow)
//var cEntityIdsPtr = (*C.long)(&(*entityIDs)[0])
//var cTimestampsPtr = (*C.ulong)(&(*timestamps)[0])
//var cSizeofPerRow = C.int(sizeofPerRow)
//var cRawDataVoidPtr = unsafe.Pointer(&rawData[0])
//
//var status = C.Insert(s.SegmentPtr,
// cOffset,
// cNumOfRows,
// cEntityIdsPtr,
// cTimestampsPtr,
// cRawDataVoidPtr,
// cSizeofPerRow,
// cNumOfRows)
//
//if status != 0 {
// return errors.New("Insert failed, error code = " + strconv.Itoa(int(status)))
//}
var numOfRow = len(*entityIDs)
var sizeofPerRow = len((*records)[0].Value)
assert.Equal(nil, numOfRow, len(*records))
var rawData = make([]byte, numOfRow*sizeofPerRow)
var copyOffset = 0
for i := 0; i < len(*records); i++ {
copy(rawData[copyOffset:], (*records)[i].Value)
copyOffset += sizeofPerRow
}
var cOffset = C.long(offset)
var cNumOfRows = C.long(numOfRow)
var cEntityIdsPtr = (*C.long)(&(*entityIDs)[0])
var cTimestampsPtr = (*C.ulong)(&(*timestamps)[0])
var cSizeofPerRow = C.int(sizeofPerRow)
var cRawDataVoidPtr = unsafe.Pointer(&rawData[0])
var status = C.Insert(s.SegmentPtr,
cOffset,
cNumOfRows,
cEntityIdsPtr,
cTimestampsPtr,
cRawDataVoidPtr,
cSizeofPerRow,
cNumOfRows)
if status != 0 {
return errors.New("Insert failed, error code = " + strconv.Itoa(int(status)))
}
return nil
}
func (s *Segment) SegmentDelete(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp) error {
func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps *[]Timestamp) error {
/*
int
Delete(CSegmentBase c_segment,
@ -201,10 +154,10 @@ func (s *Segment) SegmentDelete(offset int64, entityIDs *[]UniqueID, timestamps
return nil
}
func (s *Segment) SegmentSearch(query *QueryInfo, timestamp Timestamp, vectorRecord *msgPb.VectorRowRecord) (*SearchResult, error) {
func (s *Segment) segmentSearch(query *queryInfo, timestamp Timestamp, vectorRecord *msgPb.VectorRowRecord) (*SearchResult, error) {
/*
int
Search(CSegmentBase c_segment,
search(CSegmentBase c_segment,
CQueryInfo c_query_info,
unsigned long timestamp,
float* query_raw_data,
@ -241,10 +194,10 @@ func (s *Segment) SegmentSearch(query *QueryInfo, timestamp Timestamp, vectorRec
var status = C.Search(s.SegmentPtr, cQuery, cTimestamp, cQueryRawData, cQueryRawDataLength, cResultIds, cResultDistances)
if status != 0 {
return nil, errors.New("Search failed, error code = " + strconv.Itoa(int(status)))
return nil, errors.New("search failed, error code = " + strconv.Itoa(int(status)))
}
//fmt.Println("Search Result---- Ids =", resultIds, ", Distances =", resultDistances)
//fmt.Println("search Result---- Ids =", resultIds, ", Distances =", resultDistances)
return &SearchResult{ResultIds: resultIds, ResultDistances: resultDistances}, nil
}

View File

@ -1,59 +1,53 @@
package reader
import (
"context"
"fmt"
"log"
"strconv"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
//func (node *QueryNode) SegmentsManagement() {
// //node.queryNodeTimeSync.updateTSOTimeSync()
// //var timeNow = node.queryNodeTimeSync.TSOTimeSync
//
// timeNow := node.messageClient.GetTimeNow() >> 18
//
// for _, collection := range node.Collections {
// for _, partition := range collection.Partitions {
// for _, segment := range partition.Segments {
// if segment.SegmentStatus != SegmentOpened {
// continue
// }
//
// // fmt.Println("timeNow = ", timeNow, "SegmentCloseTime = ", segment.SegmentCloseTime)
// if timeNow >= segment.SegmentCloseTime {
// go segment.CloseSegment(collection)
// }
// }
// }
// }
//}
type segmentService struct {
ctx context.Context
msgStream *msgstream.PulsarMsgStream
node *QueryNode
}
//func (node *QueryNode) SegmentManagementService() {
// sleepMillisecondTime := 1000
// fmt.Println("do segments management in ", strconv.Itoa(sleepMillisecondTime), "ms")
// for {
// select {
// case <-node.ctx.Done():
// return
// default:
// time.Sleep(time.Duration(sleepMillisecondTime) * time.Millisecond)
// node.SegmentsManagement()
// }
// }
//}
func newSegmentService(ctx context.Context, node *QueryNode, pulsarAddress string) *segmentService {
// TODO: add pulsar message stream init
func (node *QueryNode) SegmentStatistic(sleepMillisecondTime int) {
return &segmentService{
ctx: ctx,
node: node,
}
}
func (sService *segmentService) start() {
sleepMillisecondTime := 1000
fmt.Println("do segments statistic in ", strconv.Itoa(sleepMillisecondTime), "ms")
for {
select {
case <-sService.ctx.Done():
return
default:
time.Sleep(time.Duration(sleepMillisecondTime) * time.Millisecond)
sService.sendSegmentStatistic()
}
}
}
func (sService *segmentService) sendSegmentStatistic() {
var statisticData = make([]internalpb.SegmentStatistics, 0)
for segmentID, segment := range node.SegmentsMap {
currentMemSize := segment.GetMemSize()
for segmentID, segment := range sService.node.SegmentsMap {
currentMemSize := segment.getMemSize()
segment.LastMemSize = currentMemSize
segmentNumOfRows := segment.GetRowCount()
segmentNumOfRows := segment.getRowCount()
stat := internalpb.SegmentStatistics{
// TODO: set master pb's segment id type from uint64 to int64
@ -67,22 +61,9 @@ func (node *QueryNode) SegmentStatistic(sleepMillisecondTime int) {
// fmt.Println("Publish segment statistic")
// fmt.Println(statisticData)
var status = node.PublicStatistic(&statisticData)
if status.ErrorCode != commonpb.ErrorCode_SUCCESS {
log.Printf("Publish segments statistic failed")
}
sService.publicStatistic(&statisticData)
}
func (node *QueryNode) SegmentStatisticService() {
sleepMillisecondTime := 1000
fmt.Println("do segments statistic in ", strconv.Itoa(sleepMillisecondTime), "ms")
for {
select {
case <-node.ctx.Done():
return
default:
time.Sleep(time.Duration(sleepMillisecondTime) * time.Millisecond)
node.SegmentStatistic(sleepMillisecondTime)
}
}
func (sService *segmentService) publicStatistic(statistic *[]internalpb.SegmentStatistics) {
// TODO: publish statistic
}

View File

@ -1,100 +1,70 @@
package reader
import (
"context"
"strconv"
"testing"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
)
//func TestSegmentManagement_SegmentsManagement(t *testing.T) {
//import (
// "context"
// "strconv"
// "testing"
// "time"
//
// "github.com/zilliztech/milvus-distributed/internal/conf"
// "github.com/zilliztech/milvus-distributed/internal/msgclient"
//)
//
//// NOTE: start pulsar before test
//func TestSegmentManagement_SegmentStatistic(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// mc.ReceiveMessage()
//
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// // Construct node, collection, partition and segment
// ctx := context.Background()
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.NewCollection(0, "collection0", "")
// var partition = collection.NewPartition("partition0")
// var segment = partition.NewSegment(0)
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
// node.SegmentsMap[0] = segment
//
// node.SegmentsManagement()
// node.SegmentStatistic(1000)
//
// node.Close()
//}
//func TestSegmentManagement_SegmentService(t *testing.T) {
//
//// NOTE: start pulsar before test
//func TestSegmentManagement_SegmentStatisticService(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, _ := context.WithDeadline(context.Background(), d)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// mc.ReceiveMessage()
//
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// // Construct node, collection, partition and segment
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.NewCollection(0, "collection0", "")
// var partition = collection.NewPartition("partition0")
// var segment = partition.NewSegment(0)
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
// node.SegmentsMap[0] = segment
//
// node.SegmentManagementService()
// node.SegmentStatisticService()
//
// node.Close()
//}
// NOTE: start pulsar before test
func TestSegmentManagement_SegmentStatistic(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment
node.SegmentStatistic(1000)
node.Close()
}
// NOTE: start pulsar before test
func TestSegmentManagement_SegmentStatisticService(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment
node.SegmentStatisticService()
node.Close()
}

View File

@ -4,6 +4,7 @@ import (
"context"
"encoding/binary"
"fmt"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"math"
"testing"
@ -14,10 +15,11 @@ import (
func TestSegment_ConstructorAndDestructor(t *testing.T) {
// 1. Construct node, collection, partition and segment
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -27,9 +29,9 @@ func TestSegment_ConstructorAndDestructor(t *testing.T) {
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -40,10 +42,11 @@ func TestSegment_ConstructorAndDestructor(t *testing.T) {
func TestSegment_SegmentInsert(t *testing.T) {
// 1. Construct node, collection, partition and segment
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -53,8 +56,8 @@ func TestSegment_SegmentInsert(t *testing.T) {
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps
//ids := []int64{1, 2, 3}
//timestamps := []uint64{0, 0, 0}
ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0}
// 3. Create records, use schema below:
// schema_tmp->AddField("fakeVec", DataType::VECTOR_FLOAT, 16);
@ -71,23 +74,26 @@ func TestSegment_SegmentInsert(t *testing.T) {
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records [][]byte
var records []*commonpb.Blob
for i := 0; i < N; i++ {
records = append(records, rawData)
blob := &commonpb.Blob {
Value: rawData,
}
records = append(records, blob)
}
// 4. Do PreInsert
var offset = segment.SegmentPreInsert(N)
var offset = segment.segmentPreInsert(N)
assert.GreaterOrEqual(t, offset, int64(0))
// 5. Do Insert
//var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
//assert.NoError(t, err)
var err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
// 6. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -98,10 +104,11 @@ func TestSegment_SegmentInsert(t *testing.T) {
func TestSegment_SegmentDelete(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -115,17 +122,17 @@ func TestSegment_SegmentDelete(t *testing.T) {
timestamps := []uint64{0, 0, 0}
// 3. Do PreDelete
var offset = segment.SegmentPreDelete(10)
var offset = segment.segmentPreDelete(10)
assert.GreaterOrEqual(t, offset, int64(0))
// 4. Do Delete
var err = segment.SegmentDelete(offset, &ids, &timestamps)
var err = segment.segmentDelete(offset, &ids, &timestamps)
assert.NoError(t, err)
// 5. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -136,10 +143,11 @@ func TestSegment_SegmentDelete(t *testing.T) {
func TestSegment_SegmentSearch(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -167,20 +175,23 @@ func TestSegment_SegmentSearch(t *testing.T) {
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records [][]byte
var records []*commonpb.Blob
for i := 0; i < N; i++ {
blob := &commonpb.Blob {
Value: rawData,
}
ids = append(ids, int64(i))
timestamps = append(timestamps, uint64(i+1))
records = append(records, rawData)
records = append(records, blob)
}
// 4. Do PreInsert
var offset = segment.SegmentPreInsert(N)
var offset = segment.segmentPreInsert(N)
assert.GreaterOrEqual(t, offset, int64(0))
// 5. Do Insert
//var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
//assert.NoError(t, err)
var err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
// 6. Do search
var queryJSON = "{\"field_name\":\"fakevec\",\"num_queries\":1,\"topK\":10}"
@ -191,15 +202,17 @@ func TestSegment_SegmentSearch(t *testing.T) {
var vectorRecord = msgPb.VectorRowRecord{
FloatData: queryRawData,
}
query := node.QueryJSON2Info(&queryJSON)
var searchRes, searchErr = segment.SegmentSearch(query, timestamps[N/2], &vectorRecord)
sService := searchService{}
query := sService.queryJSON2Info(&queryJSON)
var searchRes, searchErr = segment.segmentSearch(query, timestamps[N/2], &vectorRecord)
assert.NoError(t, searchErr)
fmt.Println(searchRes)
// 7. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -210,10 +223,11 @@ func TestSegment_SegmentSearch(t *testing.T) {
func TestSegment_SegmentPreInsert(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -223,13 +237,13 @@ func TestSegment_SegmentPreInsert(t *testing.T) {
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Do PreInsert
var offset = segment.SegmentPreInsert(10)
var offset = segment.segmentPreInsert(10)
assert.GreaterOrEqual(t, offset, int64(0))
// 3. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -240,10 +254,11 @@ func TestSegment_SegmentPreInsert(t *testing.T) {
func TestSegment_SegmentPreDelete(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -253,13 +268,13 @@ func TestSegment_SegmentPreDelete(t *testing.T) {
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Do PreDelete
var offset = segment.SegmentPreDelete(10)
var offset = segment.segmentPreDelete(10)
assert.GreaterOrEqual(t, offset, int64(0))
// 3. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -267,55 +282,14 @@ func TestSegment_SegmentPreDelete(t *testing.T) {
node.Close()
}
// Segment util functions test
////////////////////////////////////////////////////////////////////////////
//func TestSegment_GetStatus(t *testing.T) {
// ctx := context.Background()
// // 1. Construct node, collection, partition and segment
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.NewCollection(0, "collection0", "")
// var partition = collection.NewPartition("partition0")
// var segment = partition.NewSegment(0)
//
// // 2. Get segment status
// var status = segment.GetStatus()
// assert.Equal(t, status, SegmentOpened)
//
// // 3. Destruct collection, partition and segment
// partition.DeleteSegment(segment)
// collection.DeletePartition(partition)
// node.DeleteCollection(collection)
//
// node.Close()
//}
//func TestSegment_Close(t *testing.T) {
// ctx := context.Background()
// // 1. Construct node, collection, partition and segment
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.NewCollection(0, "collection0", "")
// var partition = collection.NewPartition("partition0")
// var segment = partition.NewSegment(0)
//
// // 2. Close segment
// var err = segment.CloseSegment(collection)
// assert.NoError(t, err)
//
// // 3. Destruct collection, partition and segment
// partition.DeleteSegment(segment)
// collection.DeletePartition(partition)
// node.DeleteCollection(collection)
//
// node.Close()
//}
func TestSegment_GetRowCount(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -326,7 +300,7 @@ func TestSegment_GetRowCount(t *testing.T) {
// 2. Create ids and timestamps
ids := []int64{1, 2, 3}
//timestamps := []uint64{0, 0, 0}
timestamps := []uint64{0, 0, 0}
// 3. Create records, use schema below:
// schema_tmp->AddField("fakeVec", DataType::VECTOR_FLOAT, 16);
@ -343,27 +317,30 @@ func TestSegment_GetRowCount(t *testing.T) {
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records [][]byte
var records []*commonpb.Blob
for i := 0; i < N; i++ {
records = append(records, rawData)
blob := &commonpb.Blob {
Value: rawData,
}
records = append(records, blob)
}
// 4. Do PreInsert
var offset = segment.SegmentPreInsert(N)
var offset = segment.segmentPreInsert(N)
assert.GreaterOrEqual(t, offset, int64(0))
// 5. Do Insert
//var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
//assert.NoError(t, err)
var err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
// 6. Get segment row count
var rowCount = segment.GetRowCount()
var rowCount = segment.getRowCount()
assert.Equal(t, rowCount, int64(len(ids)))
// 7. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -374,10 +351,11 @@ func TestSegment_GetRowCount(t *testing.T) {
func TestSegment_GetDeletedCount(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -391,22 +369,22 @@ func TestSegment_GetDeletedCount(t *testing.T) {
timestamps := []uint64{0, 0, 0}
// 3. Do PreDelete
var offset = segment.SegmentPreDelete(10)
var offset = segment.segmentPreDelete(10)
assert.GreaterOrEqual(t, offset, int64(0))
// 4. Do Delete
var err = segment.SegmentDelete(offset, &ids, &timestamps)
var err = segment.segmentDelete(offset, &ids, &timestamps)
assert.NoError(t, err)
// 5. Get segment deleted count
var deletedCount = segment.GetDeletedCount()
var deletedCount = segment.getDeletedCount()
// TODO: assert.Equal(t, deletedCount, len(ids))
assert.Equal(t, deletedCount, int64(0))
// 6. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -417,10 +395,11 @@ func TestSegment_GetDeletedCount(t *testing.T) {
func TestSegment_GetMemSize(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
var collection = node.newCollection(0, "collection0", "")
var partition = collection.newPartition("partition0")
var segment = partition.newSegment(0)
node.SegmentsMap[int64(0)] = segment
@ -430,8 +409,8 @@ func TestSegment_GetMemSize(t *testing.T) {
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps
//ids := []int64{1, 2, 3}
//timestamps := []uint64{0, 0, 0}
ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0}
// 3. Create records, use schema below:
// schema_tmp->AddField("fakeVec", DataType::VECTOR_FLOAT, 16);
@ -448,27 +427,30 @@ func TestSegment_GetMemSize(t *testing.T) {
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records [][]byte
var records []*commonpb.Blob
for i := 0; i < N; i++ {
records = append(records, rawData)
blob := &commonpb.Blob {
Value: rawData,
}
records = append(records, blob)
}
// 4. Do PreInsert
var offset = segment.SegmentPreInsert(N)
var offset = segment.segmentPreInsert(N)
assert.GreaterOrEqual(t, offset, int64(0))
// 5. Do Insert
//var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
//assert.NoError(t, err)
var err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
// 6. Get memory usage in bytes
var memSize = segment.GetMemSize()
assert.Equal(t, memSize, uint64(2785280))
var memSize = segment.getMemSize()
assert.Equal(t, memSize, int64(2785280))
// 7. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
partition.deleteSegment(node, segment)
collection.deletePartition(node, partition)
node.deleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
@ -476,68 +458,68 @@ func TestSegment_GetMemSize(t *testing.T) {
node.Close()
}
func TestSegment_RealSchemaTest(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
//var schemaString = "id: 6873737669791618215\nname: \"collection0\"\nschema: \u003c\n " +
// "field_metas: \u003c\n field_name: \"age\"\n type: INT32\n dim: 1\n \u003e\n " +
// "field_metas: \u003c\n field_name: \"field_1\"\n type: VECTOR_FLOAT\n dim: 16\n \u003e\n" +
// "\u003e\ncreate_time: 1600416765\nsegment_ids: 6873737669791618215\npartition_tags: \"default\"\n"
var schemaString = "id: 6875229265736357360\nname: \"collection0\"\nschema: \u003c\n " +
"field_metas: \u003c\n field_name: \"field_3\"\n type: INT32\n dim: 1\n \u003e\n " +
"field_metas: \u003c\n field_name: \"field_vec\"\n type: VECTOR_FLOAT\n dim: 16\n " +
"\u003e\n\u003e\ncreate_time: 1600764055\nsegment_ids: 6875229265736357360\npartition_tags: \"default\"\n"
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", schemaString)
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentID, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps
//ids := []int64{1, 2, 3}
//timestamps := []uint64{0, 0, 0}
// 3. Create records, use schema below:
// schema_tmp->AddField("fakeVec", DataType::VECTOR_FLOAT, 16);
// schema_tmp->AddField("age", DataType::INT32);
const DIM = 16
const N = 3
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
var rawData []byte
for _, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
rawData = append(rawData, buf...)
}
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records [][]byte
for i := 0; i < N; i++ {
records = append(records, rawData)
}
// 4. Do PreInsert
var offset = segment.SegmentPreInsert(N)
assert.GreaterOrEqual(t, offset, int64(0))
// 5. Do Insert
//var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
//assert.NoError(t, err)
// 6. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close()
}
//func TestSegment_RealSchemaTest(t *testing.T) {
// ctx := context.Background()
// // 1. Construct node, collection, partition and segment
// var schemaString = "id: 6875229265736357360\nname: \"collection0\"\nschema: \u003c\n " +
// "field_metas: \u003c\n field_name: \"field_3\"\n type: INT32\n dim: 1\n \u003e\n " +
// "field_metas: \u003c\n field_name: \"field_vec\"\n type: VECTOR_FLOAT\n dim: 16\n " +
// "\u003e\n\u003e\ncreate_time: 1600764055\nsegment_ids: 6875229265736357360\npartition_tags: \"default\"\n"
// pulsarUrl := "pulsar://localhost:6650"
// node := NewQueryNode(ctx, 0, pulsarUrl)
// var collection = node.newCollection(0, "collection0", schemaString)
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
//
// node.SegmentsMap[int64(0)] = segment
//
// assert.Equal(t, collection.CollectionName, "collection0")
// assert.Equal(t, partition.PartitionName, "partition0")
// assert.Equal(t, segment.SegmentID, int64(0))
// assert.Equal(t, len(node.SegmentsMap), 1)
//
// // 2. Create ids and timestamps
// ids := []int64{1, 2, 3}
// timestamps := []uint64{0, 0, 0}
//
// // 3. Create records, use schema below:
// // schema_tmp->AddField("fakeVec", DataType::VECTOR_FLOAT, 16);
// // schema_tmp->AddField("age", DataType::INT32);
// const DIM = 16
// const N = 3
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records []*commonpb.Blob
// for i := 0; i < N; i++ {
// blob := &commonpb.Blob {
// Value: rawData,
// }
// records = append(records, blob)
// }
//
// // 4. Do PreInsert
// var offset = segment.segmentPreInsert(N)
// assert.GreaterOrEqual(t, offset, int64(0))
//
// // 5. Do Insert
// var err = segment.segmentInsert(offset, &ids, &timestamps, &records)
// assert.NoError(t, err)
//
// // 6. Destruct collection, partition and segment
// partition.deleteSegment(node, segment)
// collection.deletePartition(node, partition)
// node.deleteCollection(collection)
//
// assert.Equal(t, len(node.Collections), 0)
// assert.Equal(t, len(node.SegmentsMap), 0)
//
// node.Close()
//}

View File

@ -1,18 +1,22 @@
package reader
import "log"
import (
"log"
)
type serviceTimeNode struct {
BaseNode
queryNodeTime *QueryNodeTime
node *QueryNode
serviceTimeMsg serviceTimeMsg
}
func (stNode *serviceTimeNode) Name() string {
return "iNode"
return "stNode"
}
func (stNode *serviceTimeNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do serviceTimeNode operation")
if len(in) != 1 {
log.Println("Invalid operate message input in serviceTimeNode")
// TODO: add error handling
@ -24,16 +28,17 @@ func (stNode *serviceTimeNode) Operate(in []*Msg) []*Msg {
// TODO: add error handling
}
stNode.queryNodeTime.updateSearchServiceTime(serviceTimeMsg.timeRange)
stNode.node.queryNodeTime.updateSearchServiceTime(serviceTimeMsg.timeRange)
return nil
}
func newServiceTimeNode() *serviceTimeNode {
func newServiceTimeNode(node *QueryNode) *serviceTimeNode {
baseNode := BaseNode{}
baseNode.SetMaxQueueLength(maxQueueLength)
baseNode.SetMaxParallelism(maxParallelism)
return &serviceTimeNode{
BaseNode: baseNode,
node: node,
}
}

View File

@ -1,151 +0,0 @@
package reader
import (
"encoding/json"
"errors"
"fmt"
"os"
"strconv"
"time"
log "github.com/apache/pulsar/pulsar-client-go/logutil"
)
// Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs
func (node *QueryNode) GetKey2Segments() (*[]UniqueID, *[]Timestamp, *[]UniqueID) {
var entityIDs = make([]UniqueID, 0)
var timestamps = make([]Timestamp, 0)
var segmentIDs = make([]UniqueID, 0)
var key2SegMsg = node.messageClient.Key2SegMsg
for _, msg := range key2SegMsg {
if msg.SegmentId == nil {
segmentIDs = append(segmentIDs, -1)
entityIDs = append(entityIDs, msg.Uid)
timestamps = append(timestamps, msg.Timestamp)
} else {
for _, segmentID := range msg.SegmentId {
segmentIDs = append(segmentIDs, segmentID)
entityIDs = append(entityIDs, msg.Uid)
timestamps = append(timestamps, msg.Timestamp)
}
}
}
return &entityIDs, &timestamps, &segmentIDs
}
func (node *QueryNode) GetCollectionByID(collectionID UniqueID) *Collection {
for _, collection := range node.Collections {
if collection.CollectionID == collectionID {
return collection
}
}
return nil
}
func (node *QueryNode) GetCollectionByCollectionName(collectionName string) (*Collection, error) {
for _, collection := range node.Collections {
if collection.CollectionName == collectionName {
return collection, nil
}
}
return nil, errors.New("Cannot found collection: " + collectionName)
}
func (node *QueryNode) GetSegmentBySegmentID(segmentID UniqueID) (*Segment, error) {
targetSegment := node.SegmentsMap[segmentID]
if targetSegment == nil {
return nil, errors.New("cannot found segment with id = " + strconv.FormatInt(segmentID, 10))
}
return targetSegment, nil
}
func (node *QueryNode) FoundSegmentBySegmentID(segmentID UniqueID) bool {
_, ok := node.SegmentsMap[segmentID]
return ok
}
func (c *Collection) GetPartitionByName(partitionName string) (partition *Partition) {
for _, partition := range c.Partitions {
if partition.PartitionName == partitionName {
return partition
}
}
return nil
// TODO: remove from c.Partitions
}
func (node *QueryNode) QueryLog(length int) {
node.msgCounter.InsertCounter += int64(length)
timeNow := time.Now()
duration := timeNow.Sub(node.msgCounter.InsertTime)
speed := float64(length) / duration.Seconds()
insertLog := InsertLog{
MsgLength: length,
DurationInMilliseconds: duration.Milliseconds(),
InsertTime: timeNow,
NumSince: node.msgCounter.InsertCounter,
Speed: speed,
}
node.InsertLogs = append(node.InsertLogs, insertLog)
node.msgCounter.InsertTime = timeNow
}
func (node *QueryNode) WriteQueryLog() {
f, err := os.OpenFile("/tmp/query_node_insert.txt", os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0644)
if err != nil {
log.Fatal(err)
}
// write logs
for _, insertLog := range node.InsertLogs {
insertLogJSON, err := json.Marshal(&insertLog)
if err != nil {
log.Fatal(err)
}
writeString := string(insertLogJSON) + "\n"
fmt.Println(writeString)
_, err2 := f.WriteString(writeString)
if err2 != nil {
log.Fatal(err2)
}
}
// reset InsertLogs buffer
node.InsertLogs = make([]InsertLog, 0)
err = f.Close()
if err != nil {
log.Fatal(err)
}
fmt.Println("write log done")
}
func (node *QueryNode) PrepareBatchMsg() []int {
var msgLen = node.messageClient.PrepareBatchMsg()
return msgLen
}
func (node *QueryNode) QueryJSON2Info(queryJSON *string) *QueryInfo {
var query QueryInfo
var err = json.Unmarshal([]byte(*queryJSON), &query)
if err != nil {
log.Fatal("Unmarshal query json failed")
return nil
}
//fmt.Println(query)
return &query
}

View File

@ -1,202 +1,202 @@
package reader
import (
"context"
"strconv"
"testing"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/stretchr/testify/assert"
)
// NOTE: start pulsar before test
func TestUtilFunctions_GetKey2Segments(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel := context.WithDeadline(context.Background(), d)
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc)
node.messageClient.PrepareKey2SegmentMsg()
const msgLength = 10
for i := 0; i < msgLength; i++ {
key2SegMsg := msgPb.Key2SegMsg{
Uid: UniqueID(i),
Timestamp: Timestamp(i + 1000),
SegmentId: []UniqueID{UniqueID(i)},
}
node.messageClient.Key2SegMsg = append(node.messageClient.Key2SegMsg, &key2SegMsg)
}
entityIDs, timestamps, segmentIDs := node.GetKey2Segments()
assert.Equal(t, len(*entityIDs), msgLength)
assert.Equal(t, len(*timestamps), msgLength)
assert.Equal(t, len(*segmentIDs), msgLength)
node.Close()
}
func TestUtilFunctions_GetCollectionByID(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentID, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
c := node.GetCollectionByID(int64(0))
assert.Equal(t, c.CollectionName, "collection0")
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close()
}
func TestUtilFunctions_GetCollectionByCollectionName(t *testing.T) {
ctx := context.Background()
// 1. Construct node, and collections
node := NewQueryNode(ctx, 0, 0)
var _ = node.NewCollection(0, "collection0", "")
// 2. Get collection by collectionName
var c0, err = node.GetCollectionByCollectionName("collection0")
assert.NoError(t, err)
assert.Equal(t, c0.CollectionName, "collection0")
c0 = node.GetCollectionByID(0)
assert.NotNil(t, c0)
assert.Equal(t, c0.CollectionID, uint64(0))
node.Close()
}
func TestUtilFunctions_GetSegmentBySegmentID(t *testing.T) {
ctx := context.Background()
// 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment
// 2. Get segment by segment id
var s0, err = node.GetSegmentBySegmentID(0)
assert.NoError(t, err)
assert.Equal(t, s0.SegmentID, int64(0))
node.Close()
}
func TestUtilFunctions_FoundSegmentBySegmentID(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentID, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
b1 := node.FoundSegmentBySegmentID(int64(0))
assert.Equal(t, b1, true)
b2 := node.FoundSegmentBySegmentID(int64(1))
assert.Equal(t, b2, false)
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close()
}
func TestUtilFunctions_GetPartitionByName(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var p = collection.GetPartitionByName("partition0")
assert.Equal(t, p.PartitionName, "partition0")
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
node.Close()
}
// NOTE: start pulsar before test
func TestUtilFunctions_PrepareBatchMsg(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
mc := msgclient.ReaderMessageClient{}
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc)
node.PrepareBatchMsg()
node.Close()
}
func TestUtilFunctions_QueryJson2Info(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var queryJSON = "{\"field_name\":\"age\",\"num_queries\":1,\"topK\":10}"
info := node.QueryJSON2Info(&queryJSON)
assert.Equal(t, info.FieldName, "age")
assert.Equal(t, info.NumQueries, int64(1))
assert.Equal(t, info.TopK, 10)
node.Close()
}
//import (
// "context"
// "strconv"
// "testing"
// "time"
//
// "github.com/zilliztech/milvus-distributed/internal/conf"
// "github.com/zilliztech/milvus-distributed/internal/msgclient"
// msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
//
// "github.com/stretchr/testify/assert"
//)
//
//// NOTE: start pulsar before test
//func TestUtilFunctions_GetKey2Segments(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// mc.ReceiveMessage()
//
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// node.messageClient.PrepareKey2SegmentMsg()
//
// const msgLength = 10
//
// for i := 0; i < msgLength; i++ {
// key2SegMsg := msgPb.Key2SegMsg{
// Uid: int64(i),
// Timestamp: uint64(i + 1000),
// SegmentId: []int64{int64(i)},
// }
// node.messageClient.Key2SegMsg = append(node.messageClient.Key2SegMsg, &key2SegMsg)
// }
//
// entityIDs, timestamps, segmentIDs := node.GetKey2Segments()
//
// assert.Equal(t, len(*entityIDs), msgLength)
// assert.Equal(t, len(*timestamps), msgLength)
// assert.Equal(t, len(*segmentIDs), msgLength)
//
// node.Close()
//}
//
//func TestUtilFunctions_GetCollectionByID(t *testing.T) {
// ctx := context.Background()
//
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
//
// node.SegmentsMap[int64(0)] = segment
//
// assert.Equal(t, collection.CollectionName, "collection0")
// assert.Equal(t, partition.PartitionName, "partition0")
// assert.Equal(t, segment.SegmentID, int64(0))
// assert.Equal(t, len(node.SegmentsMap), 1)
//
// c := node.getCollectionByID(int64(0))
// assert.Equal(t, c.CollectionName, "collection0")
//
// partition.deleteSegment(node, segment)
// collection.deletePartition(node, partition)
// node.deleteCollection(collection)
//
// assert.Equal(t, len(node.Collections), 0)
// assert.Equal(t, len(node.SegmentsMap), 0)
//
// node.Close()
//}
//
//func TestUtilFunctions_GetCollectionByCollectionName(t *testing.T) {
// ctx := context.Background()
// // 1. Construct node, and collections
// node := NewQueryNode(ctx, 0, 0)
// var _ = node.newCollection(0, "collection0", "")
//
// // 2. Get collection by collectionName
// var c0, err = node.getCollectionByCollectionName("collection0")
// assert.NoError(t, err)
// assert.Equal(t, c0.CollectionName, "collection0")
//
// c0 = node.getCollectionByID(0)
// assert.NotNil(t, c0)
// assert.Equal(t, c0.CollectionID, uint64(0))
//
// node.Close()
//}
//
//func TestUtilFunctions_GetSegmentBySegmentID(t *testing.T) {
// ctx := context.Background()
//
// // 1. Construct node, collection, partition and segment
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
// node.SegmentsMap[0] = segment
//
// // 2. Get segment by segment id
// var s0, err = node.getSegmentBySegmentID(0)
// assert.NoError(t, err)
// assert.Equal(t, s0.SegmentID, int64(0))
//
// node.Close()
//}
//
//func TestUtilFunctions_FoundSegmentBySegmentID(t *testing.T) {
// ctx := context.Background()
//
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(0)
//
// node.SegmentsMap[int64(0)] = segment
//
// assert.Equal(t, collection.CollectionName, "collection0")
// assert.Equal(t, partition.PartitionName, "partition0")
// assert.Equal(t, segment.SegmentID, int64(0))
// assert.Equal(t, len(node.SegmentsMap), 1)
//
// b1 := node.foundSegmentBySegmentID(int64(0))
// assert.Equal(t, b1, true)
//
// b2 := node.foundSegmentBySegmentID(int64(1))
// assert.Equal(t, b2, false)
//
// partition.deleteSegment(node, segment)
// collection.deletePartition(node, partition)
// node.deleteCollection(collection)
//
// assert.Equal(t, len(node.Collections), 0)
// assert.Equal(t, len(node.SegmentsMap), 0)
//
// node.Close()
//}
//
//func TestUtilFunctions_GetPartitionByName(t *testing.T) {
// ctx := context.Background()
//
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
//
// var p = collection.getPartitionByName("partition0")
// assert.Equal(t, p.PartitionName, "partition0")
//
// collection.deletePartition(node, partition)
// node.deleteCollection(collection)
//
// node.Close()
//}
//
//// NOTE: start pulsar before test
//func TestUtilFunctions_PrepareBatchMsg(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// ctx, cancel := context.WithCancel(context.Background())
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
//
// mc.InitClient(ctx, pulsarAddr)
// mc.ReceiveMessage()
//
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// node.PrepareBatchMsg()
// node.Close()
//}
//
//func TestUtilFunctions_QueryJson2Info(t *testing.T) {
// ctx := context.Background()
// node := NewQueryNode(ctx, 0, 0)
//
// var queryJSON = "{\"field_name\":\"age\",\"num_queries\":1,\"topK\":10}"
// info := node.queryJSON2Info(&queryJSON)
//
// assert.Equal(t, info.FieldName, "age")
// assert.Equal(t, info.NumQueries, int64(1))
// assert.Equal(t, info.TopK, 10)
//
// node.Close()
//}

View File

@ -76,6 +76,7 @@ func (fg *TimeTickedFlowGraph) SetStartNode(nodeName string) error {
}
fg.startNode = startNode
fg.startNode.inputChannels = []chan *Msg{make(chan *Msg, maxQueueLength)}
return nil
}

View File

@ -2,7 +2,7 @@ package flowgraph
import (
"context"
"fmt"
"log"
"sync"
)
@ -49,7 +49,14 @@ func (nodeCtx *nodeCtx) Start(ctx context.Context, wg *sync.WaitGroup) {
n := *nodeCtx.node
res := n.Operate(inputs)
wg := sync.WaitGroup{}
for i := 0; i < len(nodeCtx.downstreamInputChanIdx); i++ {
downstreamLength := len(nodeCtx.downstreamInputChanIdx)
if len(nodeCtx.downstream) < downstreamLength {
log.Fatal("nodeCtx.downstream length = ", len(nodeCtx.downstream))
}
if len(res) < downstreamLength {
log.Fatal("node result length = ", len(res))
}
for i := 0; i < downstreamLength; i++ {
wg.Add(1)
go nodeCtx.downstream[i].ReceiveMsg(&wg, res[i], nodeCtx.downstreamInputChanIdx[(*nodeCtx.downstream[i].node).Name()])
}
@ -66,7 +73,8 @@ func (nodeCtx *nodeCtx) Close() {
func (nodeCtx *nodeCtx) ReceiveMsg(wg *sync.WaitGroup, msg *Msg, inputChanIdx int) {
nodeCtx.inputChannels[inputChanIdx] <- msg
fmt.Println("node:", (*nodeCtx.node).Name(), "receive to input channel ", inputChanIdx)
// fmt.Println((*nodeCtx.node).Name(), "receive to input channel ", inputChanIdx)
wg.Done()
}