Refactor proxy framework and add CreateCollection Task

Signed-off-by: dragondriver <jiquan.long@zilliz.com>
pull/4973/head^2
dragondriver 2020-11-05 18:01:33 +08:00 committed by yefu.chen
parent e45df02874
commit c732c267e1
27 changed files with 655 additions and 4165 deletions

View File

@ -1,18 +1,62 @@
package main
import (
"flag"
"go.uber.org/zap"
"log"
"os"
"os/signal"
"syscall"
"context"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/proxy"
)
func main() {
cfg, err := proxy.ReadProxyOptionsFromConfig()
var yamlFile string
flag.StringVar(&yamlFile, "yaml", "", "yaml file")
flag.Parse()
flag.Usage()
log.Println("yaml file: ", yamlFile)
conf.LoadConfig(yamlFile)
// Creates server.
ctx, cancel := context.WithCancel(context.Background())
svr, err := proxy.CreateProxy(ctx)
if err != nil {
log.Fatalf("read proxy options form config file , error = %v", err)
log.Print("create server failed", zap.Error(err))
}
err = proxy.StartProxy(cfg)
if err != nil {
log.Fatalf("start proxy failed, error = %v", err)
sc := make(chan os.Signal, 1)
signal.Notify(sc,
syscall.SIGHUP,
syscall.SIGINT,
syscall.SIGTERM,
syscall.SIGQUIT)
var sig os.Signal
go func() {
sig = <-sc
cancel()
}()
if err := svr.Run(); err != nil {
log.Fatal("run server failed", zap.Error(err))
}
<-ctx.Done()
log.Print("Got signal to exit", zap.String("signal", sig.String()))
//svr.Close()
switch sig {
case syscall.SIGTERM:
exit(0)
default:
exit(1)
}
}
func exit(code int) {
os.Exit(code)
}

View File

@ -429,9 +429,6 @@ func NewUnmarshalDispatcher() *UnmarshalDispatcher
```
#### 5.4 Time Ticked Flow Graph
###### 5.4.1 Flow Graph States
@ -547,8 +544,10 @@ type KVBase interface {
Save(key, value string) error
MultiSave(kvs map[string]string) error
Remove(key string) error
MultiRemove(keys []string) error
MultiSaveAndRemove(saves map[string]string, removals []string) error
Watch(key string) clientv3.WatchChan
WatchWithPrefix(key string) clientv3.WatchChan
LoadWithPrefix(key string) ( []string, []string, error)
@ -1207,7 +1206,6 @@ func newHardTimeTickBarrier(ctx context.Context, ttStream *MsgStream, peerIds []
<img src="./figs/time_sync_msg_producer.png" width=700>
```go
type timeSyncMsgProducer struct {
proxyTtBarrier *softTimeTickBarrier

View File

@ -1,567 +0,0 @@
package msgclient
import (
"context"
"fmt"
"log"
"strconv"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/timesync"
)
type MessageType int
const (
InsertOrDelete MessageType = 0
Search MessageType = 1
SearchById MessageType = 2
TimeSync MessageType = 3
Key2Seg MessageType = 4
Statistics MessageType = 5
)
type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
type ReaderMessageClient struct {
// context
ctx context.Context
// timesync
timeSyncCfg *timesync.TimeSyncCfg
// message channel
searchChan chan *msgpb.SearchMsg
Key2SegChan chan *msgpb.Key2SegMsg
// pulsar
client pulsar.Client
//searchResultProducer pulsar.Producer
searchResultProducers map[UniqueID]pulsar.Producer
segmentsStatisticProducer pulsar.Producer
searchConsumer pulsar.Consumer
key2segConsumer pulsar.Consumer
// batch messages
InsertOrDeleteMsg []*msgpb.InsertOrDeleteMsg
Key2SegMsg []*msgpb.Key2SegMsg
SearchMsg []*msgpb.SearchMsg
timestampBatchStart Timestamp
timestampBatchEnd Timestamp
batchIDLen int
//client id
MessageClientID int
}
func (mc *ReaderMessageClient) GetTimeNow() Timestamp {
return mc.timestampBatchEnd
}
func (mc *ReaderMessageClient) TimeSyncStart() Timestamp {
return mc.timestampBatchStart
}
func (mc *ReaderMessageClient) TimeSyncEnd() Timestamp {
return mc.timestampBatchEnd
}
func (mc *ReaderMessageClient) SendResult(ctx context.Context, msg msgpb.QueryResult, producerKey UniqueID) {
var msgBuffer, _ = proto.Marshal(&msg)
if _, err := mc.searchResultProducers[producerKey].Send(ctx, &pulsar.ProducerMessage{
Payload: msgBuffer,
}); err != nil {
log.Fatal(err)
}
}
func (mc *ReaderMessageClient) SendSegmentsStatistic(ctx context.Context, statisticData *[]internalpb.SegmentStatistics) {
for _, data := range *statisticData {
var stat, _ = proto.Marshal(&data)
if _, err := mc.segmentsStatisticProducer.Send(ctx, &pulsar.ProducerMessage{
Payload: stat,
}); err != nil {
log.Fatal(err)
}
}
}
func (mc *ReaderMessageClient) GetSearchChan() <-chan *msgpb.SearchMsg {
return mc.searchChan
}
func (mc *ReaderMessageClient) receiveSearchMsg() {
for {
select {
case <-mc.ctx.Done():
return
default:
searchMsg := msgpb.SearchMsg{}
msg, err := mc.searchConsumer.Receive(mc.ctx)
if err != nil {
log.Println(err)
continue
}
err = proto.Unmarshal(msg.Payload(), &searchMsg)
if err != nil {
log.Fatal(err)
}
mc.searchChan <- &searchMsg
mc.searchConsumer.Ack(msg)
}
}
}
func (mc *ReaderMessageClient) receiveKey2SegMsg() {
for {
select {
case <-mc.ctx.Done():
return
default:
key2SegMsg := msgpb.Key2SegMsg{}
msg, err := mc.key2segConsumer.Receive(mc.ctx)
if err != nil {
log.Println(err)
continue
}
err = proto.Unmarshal(msg.Payload(), &key2SegMsg)
if err != nil {
log.Fatal(err)
}
mc.Key2SegChan <- &key2SegMsg
mc.key2segConsumer.Ack(msg)
}
}
}
func (mc *ReaderMessageClient) ReceiveMessage() {
err := mc.timeSyncCfg.Start()
if err != nil {
log.Println(err)
}
go mc.receiveSearchMsg()
go mc.receiveKey2SegMsg()
}
func (mc *ReaderMessageClient) creatProducer(topicName string) pulsar.Producer {
producer, err := mc.client.CreateProducer(pulsar.ProducerOptions{
Topic: topicName,
})
if err != nil {
log.Fatal(err)
}
return producer
}
func (mc *ReaderMessageClient) createConsumer(topicName string) pulsar.Consumer {
consumer, err := mc.client.Subscribe(pulsar.ConsumerOptions{
Topic: topicName,
SubscriptionName: "reader" + strconv.Itoa(mc.MessageClientID),
})
if err != nil {
log.Fatal(err)
}
return consumer
}
func (mc *ReaderMessageClient) createClient(url string) pulsar.Client {
if conf.Config.Pulsar.Authentication {
// create client with Authentication
client, err := pulsar.NewClient(pulsar.ClientOptions{
URL: url,
Authentication: pulsar.NewAuthenticationToken(conf.Config.Pulsar.Token),
})
if err != nil {
log.Fatal(err)
}
return client
}
// create client without Authentication
client, err := pulsar.NewClient(pulsar.ClientOptions{
URL: url,
})
if err != nil {
log.Fatal(err)
}
return client
}
func (mc *ReaderMessageClient) InitClient(ctx context.Context, url string) {
// init context
mc.ctx = ctx
//create client
mc.client = mc.createClient(url)
mc.MessageClientID = conf.Config.Reader.ClientId
//create producer
mc.searchResultProducers = make(map[UniqueID]pulsar.Producer)
proxyIdList := conf.Config.Master.ProxyIdList
searchResultTopicName := "SearchResult-"
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
key2SegTopicName = "Key2Seg-" + conf.Config.Pulsar.User
// timeSyncTopicName = "TimeSync-" + conf.Config.Pulsar.User
insertOrDeleteTopicName = "InsertOrDelete-" + conf.Config.Pulsar.User + "-"
}
for _, key := range proxyIdList {
topic := searchResultTopicName
topic = topic + strconv.Itoa(int(key))
mc.searchResultProducers[key] = mc.creatProducer(topic)
}
//mc.searchResultProducer = mc.creatProducer("SearchResult")
SegmentsStatisticTopicName := conf.Config.Master.PulsarTopic
mc.segmentsStatisticProducer = mc.creatProducer(SegmentsStatisticTopicName)
//create consumer
mc.searchConsumer = mc.createConsumer(searchTopicName)
mc.key2segConsumer = mc.createConsumer(key2SegTopicName)
// init channel
mc.searchChan = make(chan *msgpb.SearchMsg, conf.Config.Reader.SearchChanSize)
mc.Key2SegChan = make(chan *msgpb.Key2SegMsg, conf.Config.Reader.Key2SegChanSize)
mc.InsertOrDeleteMsg = make([]*msgpb.InsertOrDeleteMsg, 0)
mc.Key2SegMsg = make([]*msgpb.Key2SegMsg, 0)
//init timesync
timeSyncTopic := timeSyncTopicName
timeSyncSubName := "reader" + strconv.Itoa(mc.MessageClientID)
readTopics := make([]string, 0)
for i := conf.Config.Reader.TopicStart; i < conf.Config.Reader.TopicEnd; i++ {
str := insertOrDeleteTopicName
str = str + strconv.Itoa(i)
readTopics = append(readTopics, str)
}
readSubName := "reader" + strconv.Itoa(mc.MessageClientID)
readerQueueSize := timesync.WithReaderQueueSize(conf.Config.Reader.ReaderQueueSize)
timeSync, err := timesync.NewTimeSync(ctx,
timeSyncTopic,
timeSyncSubName,
readTopics,
readSubName,
proxyIdList,
conf.Config.Reader.StopFlag,
readerQueueSize)
if err != nil {
log.Fatal(err)
}
mc.timeSyncCfg = timeSync.(*timesync.TimeSyncCfg)
mc.timeSyncCfg.RoleType = timesync.Reader
mc.timestampBatchStart = 0
mc.timestampBatchEnd = 0
mc.batchIDLen = 0
}
func (mc *ReaderMessageClient) Close() {
if mc.client != nil {
mc.client.Close()
}
for key, _ := range mc.searchResultProducers {
if mc.searchResultProducers[key] != nil {
mc.searchResultProducers[key].Close()
}
}
if mc.segmentsStatisticProducer != nil {
mc.segmentsStatisticProducer.Close()
}
if mc.searchConsumer != nil {
mc.searchConsumer.Close()
}
if mc.key2segConsumer != nil {
mc.key2segConsumer.Close()
}
if mc.timeSyncCfg != nil {
mc.timeSyncCfg.Close()
}
}
func (mc *ReaderMessageClient) prepareMsg(messageType MessageType, msgLen int) {
switch messageType {
case InsertOrDelete:
for i := 0; i < msgLen; i++ {
msg := <-mc.timeSyncCfg.InsertOrDelete()
mc.InsertOrDeleteMsg = append(mc.InsertOrDeleteMsg, msg)
}
case TimeSync:
mc.timestampBatchStart = mc.timestampBatchEnd
mc.batchIDLen = 0
for i := 0; i < msgLen; i++ {
msg, ok := <-mc.timeSyncCfg.TimeSync()
if !ok {
fmt.Println("cnn't get data from timesync chan")
}
if i == msgLen-1 {
mc.timestampBatchEnd = msg.Timestamp
}
mc.batchIDLen += int(msg.NumRecorders)
}
}
}
func (mc *ReaderMessageClient) PrepareBatchMsg() []int {
// assume the channel not full
mc.InsertOrDeleteMsg = mc.InsertOrDeleteMsg[:0]
mc.batchIDLen = 0
// get the length of every channel
timeLen := mc.timeSyncCfg.TimeSyncChanLen()
// get message from channel to slice
if timeLen > 0 {
mc.prepareMsg(TimeSync, timeLen)
mc.prepareMsg(InsertOrDelete, mc.batchIDLen)
}
return []int{mc.batchIDLen, timeLen}
}
func (mc *ReaderMessageClient) PrepareKey2SegmentMsg() {
mc.Key2SegMsg = mc.Key2SegMsg[:0]
msgLen := len(mc.Key2SegChan)
for i := 0; i < msgLen; i++ {
msg := <-mc.Key2SegChan
mc.Key2SegMsg = append(mc.Key2SegMsg, msg)
}
}
type WriterMessageClient struct {
// timesync
timeSyncCfg *timesync.TimeSyncCfg
//message channel
searchByIdChan chan *msgpb.EntityIdentity
// pulsar
client pulsar.Client
key2segProducer pulsar.Producer
searchByIdConsumer pulsar.Consumer
// batch messages
InsertMsg []*msgpb.InsertOrDeleteMsg
DeleteMsg []*msgpb.InsertOrDeleteMsg
timestampBatchStart uint64
timestampBatchEnd uint64
batchIDLen int
//client id
MessageClientID int
}
func (mc *WriterMessageClient) Send(ctx context.Context, msg msgpb.Key2SegMsg) {
var msgBuffer, _ = proto.Marshal(&msg)
if _, err := mc.key2segProducer.Send(ctx, &pulsar.ProducerMessage{
Payload: msgBuffer,
}); err != nil {
log.Fatal(err)
}
}
func (mc *WriterMessageClient) TimeSync() uint64 {
return mc.timestampBatchEnd
}
func (mc *WriterMessageClient) SearchByIdChan() chan *msgpb.EntityIdentity {
return mc.searchByIdChan
}
func (mc *WriterMessageClient) receiveSearchByIdMsg() {
for {
searchByIdMsg := msgpb.EntityIdentity{}
msg, err := mc.searchByIdConsumer.Receive(context.Background())
err = proto.Unmarshal(msg.Payload(), &searchByIdMsg)
if err != nil {
log.Fatal(err)
}
mc.searchByIdChan <- &searchByIdMsg
mc.searchByIdConsumer.Ack(msg)
}
}
func (mc *WriterMessageClient) ReceiveMessage() {
err := mc.timeSyncCfg.Start()
if err != nil {
log.Fatal(err)
}
go mc.receiveSearchByIdMsg()
}
func (mc *WriterMessageClient) creatProducer(topicName string) pulsar.Producer {
producer, err := mc.client.CreateProducer(pulsar.ProducerOptions{
Topic: topicName,
})
if err != nil {
log.Fatal(err)
}
return producer
}
func (mc *WriterMessageClient) createConsumer(topicName string) pulsar.Consumer {
consumer, err := mc.client.Subscribe(pulsar.ConsumerOptions{
Topic: topicName,
SubscriptionName: "writer" + strconv.Itoa(mc.MessageClientID),
})
if err != nil {
log.Fatal(err)
}
return consumer
}
func (mc *WriterMessageClient) createClient(url string) pulsar.Client {
if conf.Config.Pulsar.Authentication {
// create client with Authentication
client, err := pulsar.NewClient(pulsar.ClientOptions{
URL: url,
Authentication: pulsar.NewAuthenticationToken(conf.Config.Pulsar.Token),
})
if err != nil {
log.Fatal(err)
}
return client
}
// create client without Authentication
client, err := pulsar.NewClient(pulsar.ClientOptions{
URL: url,
})
if err != nil {
log.Fatal(err)
}
return client
}
func (mc *WriterMessageClient) InitClient(url string) {
//create client
mc.client = mc.createClient(url)
mc.MessageClientID = conf.Config.Writer.ClientId
key2SegTopicName := "Key2Seg"
searchByIdTopicName := "SearchById"
timeSyncTopicName := "TimeSync"
insertOrDeleteTopicName := "InsertOrDelete-"
if conf.Config.Pulsar.Authentication {
key2SegTopicName = "Key2Seg-" + conf.Config.Pulsar.User
searchByIdTopicName = "search-" + conf.Config.Pulsar.User
// timeSyncTopicName = "TimeSync-" + conf.Config.Pulsar.User
insertOrDeleteTopicName = "InsertOrDelete-" + conf.Config.Pulsar.User + "-"
}
//create producer
mc.key2segProducer = mc.creatProducer(key2SegTopicName)
//create consumer
mc.searchByIdConsumer = mc.createConsumer(searchByIdTopicName)
//init channel
mc.searchByIdChan = make(chan *msgpb.EntityIdentity, conf.Config.Writer.SearchByIdChanSize)
//init msg slice
mc.InsertMsg = make([]*msgpb.InsertOrDeleteMsg, 0)
mc.DeleteMsg = make([]*msgpb.InsertOrDeleteMsg, 0)
//init timesync
timeSyncTopic := timeSyncTopicName
timeSyncSubName := "writer" + strconv.Itoa(mc.MessageClientID)
readTopics := make([]string, 0)
for i := conf.Config.Writer.TopicStart; i < conf.Config.Writer.TopicEnd; i++ {
str := insertOrDeleteTopicName
str = str + strconv.Itoa(i)
readTopics = append(readTopics, str)
}
readSubName := "writer" + strconv.Itoa(mc.MessageClientID)
proxyIdList := conf.Config.Master.ProxyIdList
readerQueueSize := timesync.WithReaderQueueSize(conf.Config.Reader.ReaderQueueSize)
timeSync, err := timesync.NewTimeSync(context.Background(),
timeSyncTopic,
timeSyncSubName,
readTopics,
readSubName,
proxyIdList,
conf.Config.Writer.StopFlag,
readerQueueSize)
if err != nil {
log.Fatal(err)
}
mc.timeSyncCfg = timeSync.(*timesync.TimeSyncCfg)
mc.timeSyncCfg.RoleType = timesync.Writer
mc.timestampBatchStart = 0
mc.timestampBatchEnd = 0
mc.batchIDLen = 0
}
func (mc *WriterMessageClient) Close() {
mc.client.Close()
mc.key2segProducer.Close()
mc.searchByIdConsumer.Close()
mc.timeSyncCfg.Close()
}
func (mc *WriterMessageClient) PrepareMsg(messageType MessageType, msgLen int) {
switch messageType {
case InsertOrDelete:
for i := 0; i < msgLen; i++ {
msg := <-mc.timeSyncCfg.InsertOrDelete()
if msg.Op == msgpb.OpType_INSERT {
mc.InsertMsg = append(mc.InsertMsg, msg)
} else {
mc.DeleteMsg = append(mc.DeleteMsg, msg)
}
}
case TimeSync:
mc.timestampBatchStart = mc.timestampBatchEnd
mc.batchIDLen = 0
for i := 0; i < msgLen; i++ {
msg := <-mc.timeSyncCfg.TimeSync()
if i == msgLen-1 {
mc.timestampBatchEnd = msg.Timestamp
}
mc.batchIDLen += int(msg.NumRecorders)
}
}
}
func (mc *WriterMessageClient) PrepareBatchMsg() int {
// assume the channel not full
mc.InsertMsg = mc.InsertMsg[:0]
mc.DeleteMsg = mc.DeleteMsg[:0]
mc.batchIDLen = 0
// get the length of every channel
timeLen := len(mc.timeSyncCfg.TimeSync())
// get message from channel to slice
if timeLen > 0 {
mc.PrepareMsg(TimeSync, timeLen)
mc.PrepareMsg(InsertOrDelete, mc.batchIDLen)
}
//return []int{insertOrDeleteLen, searchLen, timeLen}
return mc.batchIDLen
}

View File

@ -47,7 +47,7 @@ func GetMarshaler(MsgType MsgType) *TsMsgMarshaler {
type InsertMarshaler struct{}
func (im *InsertMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
insertTask := (*input).(InsertTask)
insertTask := (*input).(*InsertMsg)
insertRequest := &insertTask.InsertRequest
mb, err := proto.Marshal(insertRequest)
if err != nil {
@ -59,12 +59,12 @@ func (im *InsertMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
func (im *InsertMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
insertRequest := internalPb.InsertRequest{}
err := proto.Unmarshal(input, &insertRequest)
insertTask := InsertTask{InsertRequest: insertRequest}
insertMsg := &InsertMsg{InsertRequest: insertRequest}
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
}
var tsMsg TsMsg = insertTask
var tsMsg TsMsg = insertMsg
return &tsMsg, commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
}
@ -73,8 +73,8 @@ func (im *InsertMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
type DeleteMarshaler struct{}
func (dm *DeleteMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
deleteTask := (*input).(DeleteTask)
deleteRequest := &deleteTask.DeleteRequest
deleteMsg := (*input).(*DeleteMsg)
deleteRequest := &deleteMsg.DeleteRequest
mb, err := proto.Marshal(deleteRequest)
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
@ -85,11 +85,11 @@ func (dm *DeleteMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
func (dm *DeleteMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
deleteRequest := internalPb.DeleteRequest{}
err := proto.Unmarshal(input, &deleteRequest)
deleteTask := DeleteTask{DeleteRequest: deleteRequest}
deleteMsg := &DeleteMsg{DeleteRequest: deleteRequest}
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
}
var tsMsg TsMsg = deleteTask
var tsMsg TsMsg = deleteMsg
return &tsMsg, commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
}
@ -98,8 +98,8 @@ func (dm *DeleteMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
type SearchMarshaler struct{}
func (sm *SearchMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
searchTask := (*input).(SearchTask)
searchRequest := &searchTask.SearchRequest
searchMsg := (*input).(*SearchMsg)
searchRequest := &searchMsg.SearchRequest
mb, err := proto.Marshal(searchRequest)
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
@ -110,11 +110,11 @@ func (sm *SearchMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
func (sm *SearchMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
searchRequest := internalPb.SearchRequest{}
err := proto.Unmarshal(input, &searchRequest)
searchTask := SearchTask{SearchRequest: searchRequest}
searchMsg := &SearchMsg{SearchRequest: searchRequest}
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
}
var tsMsg TsMsg = searchTask
var tsMsg TsMsg = searchMsg
return &tsMsg, commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
}
@ -123,8 +123,8 @@ func (sm *SearchMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
type SearchResultMarshaler struct{}
func (srm *SearchResultMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
searchResultTask := (*input).(SearchResultTask)
searchResult := &searchResultTask.SearchResult
searchResultMsg := (*input).(*SearchResultMsg)
searchResult := &searchResultMsg.SearchResult
mb, err := proto.Marshal(searchResult)
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
@ -135,11 +135,11 @@ func (srm *SearchResultMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status
func (srm *SearchResultMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
searchResult := internalPb.SearchResult{}
err := proto.Unmarshal(input, &searchResult)
searchResultTask := SearchResultTask{SearchResult: searchResult}
searchResultMsg := &SearchResultMsg{SearchResult: searchResult}
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
}
var tsMsg TsMsg = searchResultTask
var tsMsg TsMsg = searchResultMsg
return &tsMsg, commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
}
@ -148,9 +148,9 @@ func (srm *SearchResultMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Stat
type TimeTickMarshaler struct{}
func (tm *TimeTickMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
timeTickTask := (*input).(TimeTickTask)
timeTickMsg := &timeTickTask.TimeTickMsg
mb, err := proto.Marshal(timeTickMsg)
timeTickMsg := (*input).(*TimeTickMsg)
timeTick := &timeTickMsg.TimeTickMsg
mb, err := proto.Marshal(timeTick)
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
}
@ -160,35 +160,10 @@ func (tm *TimeTickMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
func (tm *TimeTickMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
timeTickMsg := internalPb.TimeTickMsg{}
err := proto.Unmarshal(input, &timeTickMsg)
timeTickTask := TimeTickTask{TimeTickMsg: timeTickMsg}
timeTick := &TimeTickMsg{TimeTickMsg: timeTickMsg}
if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
}
var tsMsg TsMsg = timeTickTask
var tsMsg TsMsg = timeTick
return &tsMsg, commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
}
///////////////////////////////////////Key2Seg///////////////////////////////////////////////
//
//type Key2SegMarshaler struct{}
//
//func (km *Key2SegMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
// key2SegTask := (*input).(Key2SegTask)
// key2SegMsg := &key2SegTask.Key2SegMsg
// mb, err := proto.Marshal(key2SegMsg)
// if err != nil{
// return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
// }
// return mb, commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
//}
//
//func (km *Key2SegMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
// key2SegMsg := internalPb.Key2SegMsg{}
// err := proto.Unmarshal(input, &key2SegMsg)
// key2SegTask := Key2SegTask{key2SegMsg}
// if err != nil{
// return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
// }
// var tsMsg TsMsg = key2SegTask
// return &tsMsg, commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
//}

View File

@ -26,6 +26,11 @@ func repackFunc(msgs []*TsMsg, hashKeys [][]int32) map[int32]*MsgPack {
func getTsMsg(msgType MsgType, reqId UniqueID, hashValue int32) *TsMsg {
var tsMsg TsMsg
baseMsg := BaseMsg{
beginTs: 0,
endTs: 0,
HashValues: []int32{hashValue},
}
switch msgType {
case internalPb.MsgType_kInsert:
insertRequest := internalPb.InsertRequest{
@ -38,8 +43,8 @@ func getTsMsg(msgType MsgType, reqId UniqueID, hashValue int32) *TsMsg {
ProxyId: 1,
Timestamps: []Timestamp{1},
}
insertMsg := InsertTask{
HashValues: []int32{hashValue},
insertMsg := &InsertMsg{
BaseMsg: baseMsg,
InsertRequest: insertRequest,
}
tsMsg = insertMsg
@ -53,8 +58,8 @@ func getTsMsg(msgType MsgType, reqId UniqueID, hashValue int32) *TsMsg {
Timestamps: []Timestamp{1},
PrimaryKeys: []IntPrimaryKey{1},
}
deleteMsg := DeleteTask{
HashValues: []int32{hashValue},
deleteMsg := &DeleteMsg{
BaseMsg: baseMsg,
DeleteRequest: deleteRequest,
}
tsMsg = deleteMsg
@ -66,13 +71,14 @@ func getTsMsg(msgType MsgType, reqId UniqueID, hashValue int32) *TsMsg {
Timestamp: 1,
ResultChannelId: 1,
}
searchMsg := SearchTask{
HashValues: []int32{hashValue},
searchMsg := &SearchMsg{
BaseMsg: baseMsg,
SearchRequest: searchRequest,
}
tsMsg = searchMsg
case internalPb.MsgType_kSearchResult:
searchResult := internalPb.SearchResult{
MsgType: internalPb.MsgType_kSearchResult,
Status: &commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS},
ReqId: reqId,
ProxyId: 1,
@ -80,18 +86,19 @@ func getTsMsg(msgType MsgType, reqId UniqueID, hashValue int32) *TsMsg {
Timestamp: 1,
ResultChannelId: 1,
}
searchResultMsg := SearchResultTask{
HashValues: []int32{hashValue},
searchResultMsg := &SearchResultMsg{
BaseMsg: baseMsg,
SearchResult: searchResult,
}
tsMsg = searchResultMsg
case internalPb.MsgType_kTimeTick:
timeTickResult := internalPb.TimeTickMsg{
MsgType: internalPb.MsgType_kTimeTick,
PeerId: reqId,
Timestamp: 1,
}
timeTickMsg := TimeTickTask{
HashValues: []int32{hashValue},
timeTickMsg := &TimeTickMsg{
BaseMsg: baseMsg,
TimeTickMsg: timeTickResult,
}
tsMsg = timeTickMsg
@ -161,21 +168,22 @@ func TestStream_Insert(t *testing.T) {
consumerSubName := "subInsert"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 3, 3))
//run stream
initStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName, &msgPack, internalPb.MsgType_kInsert, internalPb.MsgType_kInsert, false)
}
func TestStream_Delete(t *testing.T) {
func
TestStream_Delete(t *testing.T) {
pulsarAddress := "pulsar://localhost:6650"
producerChannels := []string{"delete"}
consumerChannels := []string{"delete"}
consumerSubName := "subDelete"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kDelete, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kDelete, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kDelete, 3, 3))
//run stream
@ -189,21 +197,22 @@ func TestStream_Search(t *testing.T) {
consumerSubName := "subSearch"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearch, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearch, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearch, 3, 3))
//run stream
initStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName, &msgPack, internalPb.MsgType_kSearch, internalPb.MsgType_kSearch, false)
}
func TestStream_SearchResult(t *testing.T) {
func
TestStream_SearchResult(t *testing.T) {
pulsarAddress := "pulsar://localhost:6650"
producerChannels := []string{"search"}
consumerChannels := []string{"search"}
consumerSubName := "subSearch"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearchResult, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearchResult, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearchResult, 3, 3))
//run stream
@ -217,7 +226,7 @@ func TestStream_TimeTick(t *testing.T) {
consumerSubName := "subSearch"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 3, 3))
//run stream
@ -231,7 +240,7 @@ func TestStream_BroadCast(t *testing.T) {
consumerSubName := "subInsert"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 3, 3))
//run stream

View File

@ -14,8 +14,8 @@ func TestNewStream_Insert(t *testing.T) {
consumerSubName := "subInsert"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 3, 3))
inputStream := NewInputStream(pulsarAddress, producerChannels, false)
outputStream := NewOutputStream(pulsarAddress, 100, 100, consumerChannels, consumerSubName, false)
@ -52,8 +52,8 @@ func TestNewStream_Delete(t *testing.T) {
consumerSubName := "subDelete"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kDelete, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kDelete, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kDelete, 3, 3))
inputStream := NewInputStream(pulsarAddress, producerChannels, false)
outputStream := NewOutputStream(pulsarAddress, 100, 100, consumerChannels, consumerSubName, false)
@ -90,8 +90,8 @@ func TestNewStream_Search(t *testing.T) {
consumerSubName := "subSearch"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearch, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearch, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearch, 3, 3))
inputStream := NewInputStream(pulsarAddress, producerChannels, false)
outputStream := NewOutputStream(pulsarAddress, 100, 100, consumerChannels, consumerSubName, false)
@ -128,8 +128,8 @@ func TestNewStream_SearchResult(t *testing.T) {
consumerSubName := "subInsert"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearchResult, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearchResult, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kSearchResult, 3, 3))
inputStream := NewInputStream(pulsarAddress, producerChannels, false)
outputStream := NewOutputStream(pulsarAddress, 100, 100, consumerChannels, consumerSubName, false)
@ -166,8 +166,8 @@ func TestNewStream_TimeTick(t *testing.T) {
consumerSubName := "subInsert"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kTimeTick, 3, 3))
inputStream := NewInputStream(pulsarAddress, producerChannels, false)
outputStream := NewOutputStream(pulsarAddress, 100, 100, consumerChannels, consumerSubName, false)
@ -203,8 +203,8 @@ func TestNewTtStream_Insert_TimeSync(t *testing.T) {
consumerSubName := "subInsert"
msgPack := MsgPack{}
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 0, 0))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 1, 1))
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(internalPb.MsgType_kInsert, 3, 3))
insertRequest := internalPb.InsertRequest{
MsgType: internalPb.MsgType_kTimeTick,
@ -216,8 +216,8 @@ func TestNewTtStream_Insert_TimeSync(t *testing.T) {
ProxyId: 1,
Timestamps: []Timestamp{1},
}
insertMsg := InsertTask{
HashValues: []int32{2},
insertMsg := &InsertMsg{
BaseMsg: BaseMsg{HashValues: []int32{2}},
InsertRequest: insertRequest,
}
var tsMsg TsMsg = insertMsg
@ -245,7 +245,7 @@ func TestNewTtStream_Insert_TimeSync(t *testing.T) {
fmt.Println("msg type: ", (*v).Type(), ", msg value: ", *v)
}
}
if receiveCount+1 >= len(msgPack.Msgs) {
if receiveCount + 1 >= len(msgPack.Msgs) {
break
}
}

View File

@ -7,184 +7,86 @@ import (
type MsgType = internalPb.MsgType
type TsMsg interface {
SetTs(ts Timestamp)
BeginTs() Timestamp
EndTs() Timestamp
Type() MsgType
HashKeys() []int32
}
/////////////////////////////////////////Insert//////////////////////////////////////////
type InsertTask struct {
type BaseMsg struct {
BeginTimestamp Timestamp
EndTimestamp Timestamp
HashValues []int32
}
func (bm *BaseMsg) BeginTs() Timestamp{
return bm.BeginTimestamp
}
func (bm *BaseMsg) EndTs() Timestamp {
return bm.EndTimestamp
}
func (bm *BaseMsg) HashKeys() []int32 {
return bm.HashValues
}
/////////////////////////////////////////Insert//////////////////////////////////////////
type InsertMsg struct {
BaseMsg
internalPb.InsertRequest
}
func (it InsertTask) SetTs(ts Timestamp) {
// TODO::
}
func (it InsertTask) BeginTs() Timestamp {
timestamps := it.Timestamps
var beginTs Timestamp
for _, v := range timestamps {
beginTs = Timestamp(v)
break
}
for _, v := range timestamps {
if beginTs > Timestamp(v) {
beginTs = Timestamp(v)
}
}
return beginTs
}
func (it InsertTask) EndTs() Timestamp {
timestamps := it.Timestamps
var endTs Timestamp
for _, v := range timestamps {
endTs = Timestamp(v)
break
}
for _, v := range timestamps {
if endTs < Timestamp(v) {
endTs = Timestamp(v)
}
}
return endTs
}
func (it InsertTask) Type() MsgType {
func (it *InsertMsg) Type() MsgType {
return it.MsgType
}
func (it InsertTask) HashKeys() []int32 {
return it.HashValues
}
/////////////////////////////////////////Delete//////////////////////////////////////////
type DeleteTask struct {
HashValues []int32
type DeleteMsg struct {
BaseMsg
internalPb.DeleteRequest
}
func (dt DeleteTask) SetTs(ts Timestamp) {
// TODO::
}
func (dt DeleteTask) BeginTs() Timestamp {
timestamps := dt.Timestamps
var beginTs Timestamp
for _, v := range timestamps {
beginTs = Timestamp(v)
break
}
for _, v := range timestamps {
if beginTs > Timestamp(v) {
beginTs = Timestamp(v)
}
}
return beginTs
}
func (dt DeleteTask) EndTs() Timestamp {
timestamps := dt.Timestamps
var endTs Timestamp
for _, v := range timestamps {
endTs = Timestamp(v)
break
}
for _, v := range timestamps {
if endTs < Timestamp(v) {
endTs = Timestamp(v)
}
}
return endTs
}
func (dt DeleteTask) Type() MsgType {
func (dt *DeleteMsg) Type() MsgType {
return dt.MsgType
}
func (dt DeleteTask) HashKeys() []int32 {
return dt.HashValues
}
/////////////////////////////////////////Search//////////////////////////////////////////
type SearchTask struct {
HashValues []int32
type SearchMsg struct {
BaseMsg
internalPb.SearchRequest
}
func (st SearchTask) SetTs(ts Timestamp) {
st.Timestamp = uint64(ts)
}
func (st SearchTask) BeginTs() Timestamp {
return Timestamp(st.Timestamp)
}
func (st SearchTask) EndTs() Timestamp {
return Timestamp(st.Timestamp)
}
func (st SearchTask) Type() MsgType {
func (st *SearchMsg) Type() MsgType {
return st.MsgType
}
func (st SearchTask) HashKeys() []int32 {
return st.HashValues
}
/////////////////////////////////////////SearchResult//////////////////////////////////////////
type SearchResultTask struct {
HashValues []int32
type SearchResultMsg struct {
BaseMsg
internalPb.SearchResult
}
func (srt SearchResultTask) SetTs(ts Timestamp) {
srt.Timestamp = uint64(ts)
}
func (srt SearchResultTask) BeginTs() Timestamp {
return Timestamp(srt.Timestamp)
}
func (srt SearchResultTask) EndTs() Timestamp {
return Timestamp(srt.Timestamp)
}
func (srt SearchResultTask) Type() MsgType {
func (srt *SearchResultMsg) Type() MsgType {
return srt.MsgType
}
func (srt SearchResultTask) HashKeys() []int32 {
return srt.HashValues
}
/////////////////////////////////////////TimeTick//////////////////////////////////////////
type TimeTickTask struct {
HashValues []int32
type TimeTickMsg struct {
BaseMsg
internalPb.TimeTickMsg
}
func (tst TimeTickTask) SetTs(ts Timestamp) {
tst.Timestamp = uint64(ts)
}
func (tst TimeTickTask) BeginTs() Timestamp {
return Timestamp(tst.Timestamp)
}
func (tst TimeTickTask) EndTs() Timestamp {
return Timestamp(tst.Timestamp)
}
func (tst TimeTickTask) Type() MsgType {
func (tst *TimeTickMsg) Type() MsgType {
return tst.MsgType
}
func (tst TimeTickTask) HashKeys() []int32 {
return tst.HashValues
}
///////////////////////////////////////////Key2Seg//////////////////////////////////////////
//type Key2SegMsg struct {
// BaseMsg
// internalPb.Key2SegMsg
//}
//
//func (k2st *Key2SegMsg) Type() MsgType {
// return
//}

View File

@ -0,0 +1,137 @@
package proxy
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"log"
"github.com/pingcap/goleveldb/leveldb/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
)
func (p *Proxy) Insert(ctx context.Context, in *servicepb.RowBatch) (*servicepb.IntegerRangeResponse, error) {
it := &InsertTask{
baseInsertTask: baseInsertTask{
BaseMsg: msgstream.BaseMsg{
HashValues: in.HashKeys,
},
InsertRequest: internalpb.InsertRequest{
MsgType: internalpb.MsgType_kInsert,
CollectionName: in.CollectionName,
PartitionTag: in.PartitionTag,
RowData: in.RowData,
},
},
done: make(chan error),
resultChan: make(chan *servicepb.IntegerRangeResponse),
manipulationMsgStream: p.manipulationMsgStream,
}
it.ctx, it.cancel = context.WithCancel(ctx)
// TODO: req_id, segment_id, channel_id, proxy_id, timestamps, row_ids
defer it.cancel()
var t task = it
p.taskSch.DmQueue.Enqueue(&t)
for {
select {
case <-ctx.Done():
log.Print("insert timeout!")
return &servicepb.IntegerRangeResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "insert timeout!",
},
}, errors.New("insert timeout!")
case result := <-it.resultChan:
return result, nil
}
}
}
func (p *Proxy) CreateCollection(ctx context.Context, req *schemapb.CollectionSchema) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (p *Proxy) DropCollection(ctx context.Context, req *servicepb.CollectionName) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (p *Proxy) HasCollection(ctx context.Context, req *servicepb.CollectionName) (*servicepb.BoolResponse, error) {
return &servicepb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
Value: true,
}, nil
}
func (p *Proxy) DescribeCollection(ctx context.Context, req *servicepb.CollectionName) (*servicepb.CollectionDescription, error) {
return &servicepb.CollectionDescription{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}
func (p *Proxy) ShowCollections(ctx context.Context, req *commonpb.Empty) (*servicepb.StringListResponse, error) {
return &servicepb.StringListResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}
func (p *Proxy) CreatePartition(ctx context.Context, in *servicepb.PartitionName) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (p *Proxy) DropPartition(ctx context.Context, in *servicepb.PartitionName) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (p *Proxy) HasPartition(ctx context.Context, in *servicepb.PartitionName) (*servicepb.BoolResponse, error) {
return &servicepb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
Value: true,
}, nil
}
func (p *Proxy) DescribePartition(ctx context.Context, in *servicepb.PartitionName) (*servicepb.PartitionDescription, error) {
return &servicepb.PartitionDescription{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}
func (p *Proxy) ShowPartitions(ctx context.Context, req *servicepb.CollectionName) (*servicepb.StringListResponse, error) {
return &servicepb.StringListResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}

View File

@ -1,173 +0,0 @@
package proxy
import (
"fmt"
"log"
"sync"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
type manipulationReq struct {
stats []commonpb.Status
msgs []*pb.ManipulationReqMsg
wg sync.WaitGroup
proxy *proxyServer
}
// TsMsg interfaces
func (req *manipulationReq) Ts() (Timestamp, error) {
if req.msgs == nil {
return 0, errors.New("No typed manipulation request message in ")
}
return req.msgs[0].Timestamp, nil
}
func (req *manipulationReq) SetTs(ts Timestamp) {
for _, msg := range req.msgs {
msg.Timestamp = ts
}
}
// BaseRequest interfaces
func (req *manipulationReq) Type() pb.ReqType {
if req.msgs == nil {
return 0
}
return req.msgs[0].ReqType
}
// TODO: use a ProcessReq function to wrap details?
// like func (req *manipulationReq) ProcessReq() commonpb.Status{
// req.PreExecute()
// req.Execute()
// req.PostExecute()
// req.WaitToFinish()
//}
func (req *manipulationReq) PreExecute() commonpb.Status {
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (req *manipulationReq) Execute() commonpb.Status {
req.proxy.reqSch.manipulationsChan <- req
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (req *manipulationReq) PostExecute() commonpb.Status { // send into pulsar
req.wg.Add(1)
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (req *manipulationReq) WaitToFinish() commonpb.Status { // wait until send into pulsar
req.wg.Wait()
for _, stat := range req.stats {
if stat.ErrorCode != commonpb.ErrorCode_SUCCESS {
return stat
}
}
// update timestamp if necessary
ts, _ := req.Ts()
req.proxy.reqSch.mTimestampMux.Lock()
defer req.proxy.reqSch.mTimestampMux.Unlock()
if req.proxy.reqSch.mTimestamp <= ts {
req.proxy.reqSch.mTimestamp = ts
} else {
log.Printf("there is some wrong with m_timestamp, it goes back, current = %d, previous = %d", ts, req.proxy.reqSch.mTimestamp)
}
return req.stats[0]
}
func (s *proxyServer) restartManipulationRoutine(bufSize int) error {
s.reqSch.manipulationsChan = make(chan *manipulationReq, bufSize)
pulsarClient, err := pulsar.NewClient(pulsar.ClientOptions{URL: s.pulsarAddr})
if err != nil {
return err
}
readers := make([]pulsar.Producer, len(s.readerTopics))
for i, t := range s.readerTopics {
p, err := pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: t})
if err != nil {
return err
}
readers[i] = p
}
deleter, err := pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: s.deleteTopic})
if err != nil {
return err
}
go func() {
for {
select {
case <-s.ctx.Done():
deleter.Close()
for _, r := range readers {
r.Close()
}
pulsarClient.Close()
return
case ip := <-s.reqSch.manipulationsChan:
ts, err := s.getTimestamp(1)
if err != nil {
log.Printf("get time stamp failed")
ip.stats[0] = commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR}
ip.wg.Done()
break
}
ip.SetTs(ts[0])
wg := sync.WaitGroup{}
for i, mq := range ip.msgs {
mq := mq
i := i
wg.Add(1)
go func() {
defer wg.Done()
mb, err := proto.Marshal(mq)
if err != nil {
log.Printf("Marshal ManipulationReqMsg failed, error = %v", err)
ip.stats[i] = commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: fmt.Sprintf("Marshal ManipulationReqMsg failed, error=%v", err),
}
return
}
switch ip.Type() {
case pb.ReqType_kInsert:
if _, err := readers[mq.ChannelId].Send(s.ctx, &pulsar.ProducerMessage{Payload: mb}); err != nil {
log.Printf("post into puslar failed, error = %v", err)
ip.stats[i] = commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: fmt.Sprintf("Post into puslar failed, error=%v", err.Error()),
}
return
}
case pb.ReqType_kDeleteEntityByID:
if _, err = deleter.Send(s.ctx, &pulsar.ProducerMessage{Payload: mb}); err != nil {
log.Printf("post into pulsar filed, error = %v", err)
ip.stats[i] = commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: fmt.Sprintf("Post into puslar failed, error=%v", err.Error()),
}
return
}
default:
log.Printf("post unexpect ReqType = %d", ip.Type())
return
}
}()
}
wg.Wait()
ip.wg.Done()
break
}
}
}()
return nil
}

View File

@ -1,58 +0,0 @@
package proxy
import (
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
)
type insertTask struct {
baseTask
// SegIdAssigner, RowIdAllocator
rowBatch *servicepb.RowBatch
resultChan chan *servicepb.IntegerRangeResponse
pulsarMsgStream *msgstream.PulsarMsgStream
}
func (it *insertTask) PreExecute() error {
return nil
}
func (it *insertTask) Execute() error {
ts := it.GetTs()
insertRequest := internalpb.InsertRequest{
MsgType: internalpb.MsgType_kInsert,
ReqId: it.ReqId,
CollectionName: it.rowBatch.CollectionName,
PartitionTag: it.rowBatch.PartitionTag,
SegmentId: 1, // TODO: use SegIdAssigner instead
// TODO: ChannelID
ProxyId: it.ProxyId,
Timestamps: []Timestamp{ts},
RowIds: []UniqueID{1}, // TODO: use RowIdAllocator instead
RowData: it.rowBatch.RowData,
}
pulsarInsertTask := msgstream.InsertTask{
InsertRequest: insertRequest,
}
var tsMsg msgstream.TsMsg = &pulsarInsertTask
msgPack := &msgstream.MsgPack{
BeginTs: ts,
EndTs: ts,
}
msgPack.Msgs = append(msgPack.Msgs, &tsMsg)
it.pulsarMsgStream.Produce(msgPack)
return nil
}
func (it *insertTask) PostExecute() error {
return nil
}
func (it *insertTask) WaitToFinish() error {
return nil
}
func (it *insertTask) Notify() error {
return nil
}

View File

@ -0,0 +1,22 @@
package mock
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
mpb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
)
type testMasterServer struct {
mpb.UnimplementedMasterServer
}
func (s *testMasterServer) CreateCollection(ctx context.Context, in *internalpb.CreateCollectionRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
}, nil
}

136
internal/proxy/proxy.go Normal file
View File

@ -0,0 +1,136 @@
package proxy
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"google.golang.org/grpc"
"log"
"math/rand"
"net"
"sync"
"time"
)
type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
type Proxy struct {
ctx context.Context
proxyLoopCtx context.Context
proxyLoopCancel func()
proxyLoopWg sync.WaitGroup
servicepb.UnimplementedMilvusServiceServer
grpcServer *grpc.Server
masterConn *grpc.ClientConn
masterClient masterpb.MasterClient
taskSch *TaskScheduler
manipulationMsgStream *msgstream.PulsarMsgStream
queryMsgStream *msgstream.PulsarMsgStream
}
func CreateProxy(ctx context.Context) (*Proxy, error) {
rand.Seed(time.Now().UnixNano())
m := &Proxy{
ctx: ctx,
}
return m, nil
}
func (p *Proxy) grpcLoop() {
defer p.proxyLoopWg.Done()
// TODO: use address in config instead
lis, err := net.Listen("tcp", "5053")
if err != nil {
log.Fatalf("Proxy grpc server fatal error=%v", err)
}
p.grpcServer = grpc.NewServer()
servicepb.RegisterMilvusServiceServer(p.grpcServer, p)
if err = p.grpcServer.Serve(lis); err != nil {
log.Fatalf("Proxy grpc server fatal error=%v", err)
}
ctx, cancel := context.WithCancel(p.proxyLoopCtx)
defer cancel()
for {
select {
case <-ctx.Done():
log.Print("proxy is closed...")
return
}
}
}
func (p *Proxy) pulsarMsgStreamLoop() {
defer p.proxyLoopWg.Done()
p.manipulationMsgStream = &msgstream.PulsarMsgStream{}
p.queryMsgStream = &msgstream.PulsarMsgStream{}
// TODO: config, RepackFunc
p.manipulationMsgStream.Start()
p.queryMsgStream.Start()
ctx, cancel := context.WithCancel(p.proxyLoopCtx)
defer cancel()
for {
select {
case <-ctx.Done():
log.Print("proxy is closed...")
return
}
}
}
func (p *Proxy) scheduleLoop() {
defer p.proxyLoopWg.Done()
p.taskSch = &TaskScheduler{}
p.taskSch.Start(p.ctx)
defer p.taskSch.Close()
ctx, cancel := context.WithCancel(p.proxyLoopCtx)
defer cancel()
for {
select {
case <-ctx.Done():
log.Print("proxy is closed...")
return
}
}
}
func (p *Proxy) connectMaster() error {
log.Printf("Connected to master, master_addr=%s", "127.0.0.1:5053")
ctx, _ := context.WithTimeout(context.Background(), 2*time.Second)
conn, err := grpc.DialContext(ctx, "127.0.0.1:5053", grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
log.Printf("Connect to master failed, error= %v", err)
return err
}
log.Printf("Connected to master, master_addr=%s", "127.0.0.1:5053")
p.masterConn = conn
p.masterClient = masterpb.NewMasterClient(conn)
return nil
}
func (p *Proxy) startProxyLoop(ctx context.Context) {
p.proxyLoopCtx, p.proxyLoopCancel = context.WithCancel(ctx)
p.proxyLoopWg.Add(3)
p.connectMaster()
go p.grpcLoop()
go p.pulsarMsgStreamLoop()
go p.scheduleLoop()
}
func (p *Proxy) Run() error {
p.startProxyLoop(p.ctx)
p.proxyLoopWg.Wait()
return nil
}

View File

@ -1,94 +0,0 @@
package proxy
import (
"context"
"log"
"net"
"sync"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"google.golang.org/grpc"
)
type proxyInstance struct {
servicepb.UnimplementedMilvusServiceServer
grpcServer *grpc.Server
taskSch *taskScheduler
taskChan chan *task
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
}
func (ins *proxyInstance) Insert(ctx context.Context, req *servicepb.RowBatch) (*servicepb.IntegerRangeResponse, error) {
return &servicepb.IntegerRangeResponse{}, nil
}
func (ins *proxyInstance) StartGrpcServer() error {
// TODO: use address in config instead
lis, err := net.Listen("tcp", "127.0.0.1")
if err != nil {
return err
}
go func() {
ins.wg.Add(1)
defer ins.wg.Done()
server := grpc.NewServer()
servicepb.RegisterMilvusServiceServer(server, ins)
err := server.Serve(lis)
if err != nil {
log.Fatalf("Proxy grpc server fatal error=%v", err)
}
}()
return nil
}
func (ins *proxyInstance) restartSchedulerRoutine(bufSize int) error {
ins.taskChan = make(chan *task, bufSize)
go func() {
for {
select {
case t := <-ins.taskChan:
switch (*t).Type() {
case internalpb.MsgType_kInsert:
ins.taskSch.DmQueue.Enqueue(t)
default:
return
}
default:
return
}
}
}()
return nil
}
func (ins *proxyInstance) restartForwardRoutine() error {
return nil
}
func startProxyInstance(ins *proxyInstance) error {
if err := ins.restartSchedulerRoutine(1024); err != nil {
return err
}
if err := ins.restartForwardRoutine(); err != nil {
return err
}
return ins.StartGrpcServer()
}
func StartProxyInstance() error {
ins := &proxyInstance{}
err := startProxyInstance(ins)
if err != nil {
return nil
}
ins.wg.Wait()
return nil
}

View File

@ -1,175 +0,0 @@
package proxy
import (
"context"
"fmt"
"strconv"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
etcd "go.etcd.io/etcd/clientv3"
)
type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
type IntPrimaryKey = typeutil.IntPrimaryKey
type BaseRequest interface {
Type() internalpb.MsgType
PreExecute() commonpb.Status
Execute() commonpb.Status
PostExecute() commonpb.Status
WaitToFinish() commonpb.Status
}
type ProxyOptions struct {
//proxy server
address string //grpc server address
master_address string //master server addess
collectionMetaRootPath string // etcd root path,read metas of collections and segments from etcd
pulsarAddr string // pulsar address for reader
readerTopicsPrefix string
numReadTopics int
deleteTopic string
queryTopic string
resultTopic string
resultGroup string
numReaderNode int
proxyId UniqueID //start from 1
etcdEndpoints []string
//timestamporacle
tsoRootPath string //etcd root path, store timestamp into this key
tsoSaveInterval uint64
//timetick
timeTickInterval uint64
timeTickTopic string
timeTickPeerId UniqueID //start from 1
// inner member
proxyServer *proxyServer
tso *allocator.TimestampAllocator
timeTick *timeTick
ctx context.Context
cancel context.CancelFunc
}
func ReadProxyOptionsFromConfig() (*ProxyOptions, error) {
conf.LoadConfig("config.yaml")
etcdRootPath := conf.Config.Etcd.Rootpath
if etcdRootPath[len(etcdRootPath)-1] == '/' {
etcdRootPath = etcdRootPath[0 : len(etcdRootPath)-1]
}
return &ProxyOptions{
address: conf.Config.Proxy.Network.Address + ":" + strconv.Itoa(conf.Config.Proxy.Network.Port),
master_address: conf.Config.Master.Address + ":" + strconv.Itoa(int(conf.Config.Master.Port)),
collectionMetaRootPath: etcdRootPath,
pulsarAddr: "pulsar://" + conf.Config.Pulsar.Address + ":" + strconv.Itoa(int(conf.Config.Pulsar.Port)),
readerTopicsPrefix: conf.Config.Proxy.PulsarTopics.ReaderTopicPrefix,
numReadTopics: conf.Config.Proxy.PulsarTopics.NumReaderTopics,
deleteTopic: conf.Config.Proxy.PulsarTopics.DeleteTopic,
queryTopic: conf.Config.Proxy.PulsarTopics.QueryTopic,
resultTopic: conf.Config.Proxy.PulsarTopics.ResultTopic,
resultGroup: conf.Config.Proxy.PulsarTopics.ResultGroup,
numReaderNode: conf.Config.Proxy.NumReaderNodes,
proxyId: UniqueID(conf.Config.Proxy.ProxyId),
etcdEndpoints: []string{conf.Config.Etcd.Address + ":" + strconv.Itoa(int(conf.Config.Etcd.Port))},
tsoRootPath: etcdRootPath,
tsoSaveInterval: uint64(conf.Config.Proxy.TosSaveInterval),
timeTickInterval: uint64(conf.Config.Proxy.TimeTickInterval),
timeTickTopic: conf.Config.Proxy.PulsarTopics.TimeTickTopic,
timeTickPeerId: UniqueID(conf.Config.Proxy.ProxyId),
}, nil
}
func StartProxy(opt *ProxyOptions) error {
//global context
opt.ctx, opt.cancel = context.WithCancel(context.Background())
///////////////////// timestamporacle //////////////////////////
//etcdTso, err := etcd.New(etcd.Config{Endpoints: opt.etcdEndpoints})
//if err != nil {
// return err
//}
//tso := &timestampOracle{
// client: etcdTso,
// ctx: opt.ctx,
// rootPath: opt.tsoRootPath,
// saveInterval: opt.tsoSaveInterval,
//}
//tso.Restart(opt.proxyId)
tso, _ := allocator.NewTimestampAllocator(opt.ctx)
/////////////////// proxy server ///////////////////////////////
//readerTopics, send insert and delete message into these topics
readerTopics := make([]string, 0, opt.numReadTopics)
for i := 0; i < opt.numReadTopics; i++ {
readerTopics = append(readerTopics, opt.readerTopicsPrefix+strconv.Itoa(i))
}
etcdProxy, err := etcd.New(etcd.Config{Endpoints: opt.etcdEndpoints})
if err != nil {
return err
}
srv := &proxyServer{
address: opt.address,
masterAddress: opt.master_address,
rootPath: opt.collectionMetaRootPath,
pulsarAddr: opt.pulsarAddr,
readerTopics: readerTopics,
deleteTopic: opt.deleteTopic,
queryTopic: opt.queryTopic,
resultTopic: opt.resultTopic,
resultGroup: opt.resultTopic,
numReaderNode: opt.numReaderNode,
proxyId: opt.proxyId,
getTimestamp: tso.Alloc,
client: etcdProxy,
ctx: opt.ctx,
}
err = startProxyServer(srv)
if err != nil {
return err
}
////////////////////////// time tick /////////////////////////////////
ttClient, err := pulsar.NewClient(pulsar.ClientOptions{URL: opt.pulsarAddr})
if err != nil {
return err
}
ttProducer, err := ttClient.CreateProducer(pulsar.ProducerOptions{Topic: opt.timeTickTopic})
if err != nil {
return err
}
tt := &timeTick{
interval: opt.timeTickInterval,
pulsarProducer: ttProducer,
peer_id: opt.timeTickPeerId,
ctx: opt.ctx,
areRequestsDelivered: func(ts Timestamp) bool { return srv.reqSch.AreRequestsDelivered(ts, 2) },
getTimestamp: func() (Timestamp, error) {
ts, st := tso.AllocOne()
return ts, st
},
}
err = tt.Restart()
if err != nil {
return fmt.Errorf("timeTick Restart Failed")
}
opt.proxyServer = srv
opt.tso = tso
opt.timeTick = tt
srv.wg.Wait()
return nil
}

View File

@ -1,455 +0,0 @@
package proxy
import (
"context"
"encoding/binary"
"encoding/json"
"os"
"sort"
"strconv"
"testing"
"time"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
etcd "go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
)
const (
tsoKeyPath string = "/timestampOracle"
)
var timeAllocator *allocator.TimestampAllocator
func TestMain(m *testing.M) {
timeAllocator, _ = allocator.NewTimestampAllocator(context.Background())
exitCode := m.Run()
timeAllocator.Close()
os.Exit(exitCode)
}
func TestProxyNode(t *testing.T) {
startTestMaster("localhost:11000", t)
testOpt := ProxyOptions{
address: "localhost:11001",
master_address: "localhost:11000",
collectionMetaRootPath: "/collections/meta",
pulsarAddr: "pulsar://localhost:6650",
readerTopicsPrefix: "reader-",
numReadTopics: 2,
deleteTopic: "deleteT",
queryTopic: "queryT",
resultTopic: "resultT",
resultGroup: "resultG",
numReaderNode: 2,
proxyId: 1,
etcdEndpoints: []string{"127.0.0.1:2379"},
tsoRootPath: "/tso",
tsoSaveInterval: 200,
timeTickInterval: 200,
timeTickTopic: "timetick",
timeTickPeerId: 1,
}
if err := StartProxy(&testOpt); err != nil {
t.Fatal(err)
}
startTime := uint64(time.Now().UnixNano()) / uint64(1e6)
t.Logf("start time stamp = %d", startTime)
etcdClient, err := etcd.New(etcd.Config{Endpoints: testOpt.etcdEndpoints})
assert.Nil(t, err)
//defer etcdClient.Close()
pulsarClient, err := pulsar.NewClient(pulsar.ClientOptions{URL: testOpt.pulsarAddr})
assert.Nil(t, err)
defer pulsarClient.Close()
go func() {
time.Sleep(time.Second)
for {
ts, err := etcdClient.Get(testOpt.ctx, testOpt.tsoRootPath+tsoKeyPath)
assert.Nil(t, err)
if len(ts.Kvs) != 1 {
t.Fatalf("save tso into etcd falied")
}
value, err := strconv.ParseUint(string(ts.Kvs[0].Value), 10, 64)
assert.Nil(t, err)
curValue, err := testOpt.tso.AllocOne()
curTS, err := timeAllocator.AllocOne()
assert.Equalf(t, err, nil, "%s", "allocator failed")
curTime, _ := tsoutil.ParseTS(curTS)
t.Logf("current time stamp = %d, saved time stamp = %d", curTime, value)
assert.GreaterOrEqual(t, curValue, value)
assert.GreaterOrEqual(t, value, startTime)
time.Sleep(time.Duration(testOpt.tsoSaveInterval) * time.Millisecond)
}
}()
tickComsumer, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topic: testOpt.timeTickTopic,
SubscriptionName: testOpt.timeTickTopic + "G",
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
assert.Nil(t, err)
defer tickComsumer.Close()
reader, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topics: testOpt.proxyServer.readerTopics,
SubscriptionName: testOpt.readerTopicsPrefix + "G",
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
assert.Nil(t, err)
defer reader.Close()
query, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topic: testOpt.queryTopic,
SubscriptionName: testOpt.queryTopic + "G",
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
assert.Nil(t, err)
defer query.Close()
deleteC, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topic: testOpt.deleteTopic,
SubscriptionName: testOpt.deleteTopic + "G",
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
assert.Nil(t, err)
defer deleteC.Close()
result, err := pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: testOpt.resultTopic})
assert.Nil(t, err)
defer result.Close()
tick := time.Tick(500 * time.Millisecond)
// read pulsar channel until empty
func() {
cnt := 0
for {
select {
case <-tick:
cnt++
if cnt >= 3 {
return
}
case cm, ok := <-tickComsumer.Chan():
assert.Truef(t, ok, "time tick consumer topic has closed")
tickComsumer.AckID(cm.ID())
case cm, ok := <-reader.Chan():
assert.Truef(t, ok, "reader comsumer topic has closed")
reader.AckID(cm.ID())
case cm, ok := <-deleteC.Chan():
assert.Truef(t, ok, "delete topic has closed")
deleteC.AckID(cm.ID())
case cm, ok := <-query.Chan():
assert.Truef(t, ok, "query topic has closed")
query.AckID(cm.ID())
}
}
}()
go func() {
lastT, _ := tsoutil.ParseTS(startTime)
for {
cm, ok := <-tickComsumer.Chan()
assert.Truef(t, ok, "time tick consumer topic has closed")
tickComsumer.AckID(cm.ID())
var tsm pb.TimeSyncMsg
if err := proto.Unmarshal(cm.Payload(), &tsm); err != nil {
t.Fatal(err)
}
curT, _ := tsoutil.ParseTS(tsm.Timestamp)
t.Logf("time tick = %d", curT)
assert.Greater(t, curT, lastT)
lastT = curT
}
}()
cm100 := etcdpb.CollectionMeta{
Id: 100,
Schema: nil,
CreateTime: 0,
SegmentIds: []int64{101, 102},
PartitionTags: nil,
}
sm101 := etcdpb.SegmentMeta{
SegmentId: 101,
CollectionId: 100,
ChannelStart: 0,
ChannelEnd: 1,
}
sm102 := etcdpb.SegmentMeta{
SegmentId: 102,
CollectionId: 100,
ChannelStart: 1,
ChannelEnd: 2,
}
if cm100b, err := json.Marshal(&cm100); err != nil {
t.Fatal(err)
} else if _, err := etcdClient.Put(testOpt.ctx, testOpt.collectionMetaRootPath+"/"+keyCollectionPath+"/100", string(cm100b)); err != nil {
t.Fatal(err)
}
if sm101b, err := json.Marshal(&sm101); err != nil {
t.Fatal(err)
} else if _, err := etcdClient.Put(testOpt.ctx, testOpt.collectionMetaRootPath+"/"+keySegmentPath+"/101", string(sm101b)); err != nil {
t.Fatal(err)
}
if sm102b, err := json.Marshal(&sm102); err != nil {
t.Fatal(err)
} else if _, err := etcdClient.Put(testOpt.ctx, testOpt.collectionMetaRootPath+"/"+keySegmentPath+"/102", string(sm102b)); err != nil {
t.Fatal(err)
}
ctx1, _ := context.WithTimeout(testOpt.ctx, time.Second)
grpcConn, err := grpc.DialContext(ctx1, testOpt.address, grpc.WithInsecure(), grpc.WithBlock())
assert.Nil(t, err)
defer grpcConn.Close()
proxyClient := pb.NewMilvusServiceClient(grpcConn)
insertParm := pb.InsertParam{
CollectionName: "cm100",
Schema: nil,
RowsData: []*pb.RowData{
{Blob: uint64ToBytes(10)},
{Blob: uint64ToBytes(11)},
{Blob: uint64ToBytes(12)},
{Blob: uint64ToBytes(13)},
{Blob: uint64ToBytes(14)},
{Blob: uint64ToBytes(15)},
},
EntityIdArray: []UniqueID{10, 11, 12, 13, 14, 15},
PartitionTag: "",
ExtraParams: nil,
}
deleteParm := pb.DeleteByIDParam{
CollectionName: "cm100",
IdArray: []UniqueID{20, 21},
}
searchParm := pb.SearchParam{
CollectionName: "cm100",
VectorParam: nil,
Dsl: "",
PartitionTag: nil,
ExtraParams: nil,
}
go func() {
cm, ok := <-query.Chan()
assert.Truef(t, ok, "query topic has closed")
query.AckID(cm.ID())
var qm pb.QueryReqMsg
if err := proto.Unmarshal(cm.Payload(), &qm); err != nil {
t.Fatal(err)
}
assert.Equal(t, qm.ProxyId, testOpt.proxyId)
assert.Equal(t, qm.CollectionName, "cm100")
physicalTime, _ := tsoutil.ParseTS(qm.Timestamp)
t.Logf("query time stamp = %d", physicalTime)
assert.Greater(t, physicalTime, startTime)
r1 := pb.QueryResult{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Entities: &pb.Entities{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Ids: []UniqueID{11, 13, 15},
ValidRow: []bool{true, true, true},
RowsData: []*pb.RowData{
{Blob: uint64ToBytes(11)},
{Blob: uint64ToBytes(13)},
{Blob: uint64ToBytes(15)},
},
},
RowNum: 3,
Scores: []float32{11, 13, 15},
Distances: []float32{11, 13, 15},
ExtraParams: nil,
QueryId: qm.QueryId,
ProxyId: qm.ProxyId,
}
r2 := pb.QueryResult{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Entities: &pb.Entities{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Ids: []UniqueID{12, 14, 16},
ValidRow: []bool{true, false, true},
RowsData: []*pb.RowData{
{Blob: uint64ToBytes(12)},
{Blob: uint64ToBytes(14)},
{Blob: uint64ToBytes(16)},
},
},
RowNum: 3,
Scores: []float32{12, 14, 16},
Distances: []float32{12, 14, 16},
ExtraParams: nil,
QueryId: qm.QueryId,
ProxyId: qm.ProxyId,
}
if b1, err := proto.Marshal(&r1); err != nil {
t.Fatal(err)
} else if _, err := result.Send(testOpt.ctx, &pulsar.ProducerMessage{Payload: b1}); err != nil {
t.Fatal(err)
}
if b2, err := proto.Marshal(&r2); err != nil {
t.Fatal(err)
} else if _, err := result.Send(testOpt.ctx, &pulsar.ProducerMessage{Payload: b2}); err != nil {
t.Fatal(err)
}
}()
insertR, err := proxyClient.Insert(testOpt.ctx, &insertParm)
assert.Nil(t, err)
assert.Equalf(t, insertR.Status.ErrorCode, pb.ErrorCode_SUCCESS, "%s", insertR.Status.Reason)
assert.Equal(t, len(insertR.EntityIdArray), 6)
sort.Slice(insertR.EntityIdArray, func(i, j int) bool {
return insertR.EntityIdArray[i] < insertR.EntityIdArray[j]
})
for i := 0; i < len(insertR.EntityIdArray); i++ {
assert.Equal(t, insertR.EntityIdArray[i], int64(i+10))
}
var insertPrimaryKey []IntPrimaryKey
readerM1, ok := <-reader.Chan()
assert.True(t, ok)
reader.AckID(readerM1.ID())
var m1 pb.ManipulationReqMsg
if err := proto.UnmarshalMerge(readerM1.Payload(), &m1); err != nil {
t.Fatal(err)
}
assert.Equal(t, m1.CollectionName, "cm100")
assert.Equal(t, len(m1.PrimaryKeys), len(m1.RowsData))
physicalTime, _ := tsoutil.ParseTS(m1.Timestamp)
t.Logf("reader time stamp = %d", physicalTime)
assert.GreaterOrEqual(t, physicalTime, startTime)
for i, k := range m1.PrimaryKeys {
insertPrimaryKey = append(insertPrimaryKey, k)
rowValue := binary.LittleEndian.Uint64(m1.RowsData[i].Blob)
t.Logf("insert primary key = %d, row data= %d", k, rowValue)
assert.Equal(t, k, rowValue)
}
readerM2, ok := <-reader.Chan()
assert.True(t, ok)
reader.AckID(readerM2.ID())
var m2 pb.ManipulationReqMsg
if err := proto.UnmarshalMerge(readerM2.Payload(), &m2); err != nil {
t.Fatal(err)
}
assert.Equal(t, m2.CollectionName, "cm100")
assert.Equal(t, len(m2.PrimaryKeys), len(m2.RowsData))
physicalTime, _ = tsoutil.ParseTS(m2.Timestamp)
t.Logf("reader time stamp = %d", physicalTime)
t.Logf("read time stamp = %d", physicalTime)
assert.GreaterOrEqual(t, physicalTime, startTime)
for i, k := range m2.PrimaryKeys {
insertPrimaryKey = append(insertPrimaryKey, k)
rowValue := binary.LittleEndian.Uint64(m2.RowsData[i].Blob)
t.Logf("insert primary key = %d, row data= %d", k, rowValue)
assert.Equal(t, k, rowValue)
}
sort.Slice(insertPrimaryKey, func(i, j int) bool {
return insertPrimaryKey[i] < insertPrimaryKey[j]
})
assert.Equal(t, len(insertPrimaryKey), 6)
for i := 0; i < len(insertPrimaryKey); i++ {
assert.Equal(t, insertPrimaryKey[i], uint64(i+10))
}
deleteR, err := proxyClient.DeleteByID(testOpt.ctx, &deleteParm)
assert.Nil(t, err)
assert.Equal(t, deleteR.ErrorCode, pb.ErrorCode_SUCCESS)
deleteM, ok := <-deleteC.Chan()
assert.True(t, ok)
deleteC.AckID(deleteM.ID())
var dm pb.ManipulationReqMsg
if err := proto.UnmarshalMerge(deleteM.Payload(), &dm); err != nil {
t.Fatal(err)
}
assert.Equal(t, dm.CollectionName, "cm100")
assert.Equal(t, len(dm.PrimaryKeys), 2)
physicalTime, _ = tsoutil.ParseTS(m1.Timestamp)
t.Logf("reader time stamp = %d", physicalTime)
t.Logf("delete time stamp = %d", physicalTime)
assert.GreaterOrEqual(t, physicalTime, startTime)
for i := 0; i < len(dm.PrimaryKeys); i++ {
assert.Equal(t, dm.PrimaryKeys[i], uint64(i+20))
}
searchR, err := proxyClient.Search(testOpt.ctx, &searchParm)
assert.Nil(t, err)
assert.Equal(t, searchR.Status.ErrorCode, pb.ErrorCode_SUCCESS)
assert.Equal(t, searchR.Entities.Status.ErrorCode, pb.ErrorCode_SUCCESS)
assert.Equal(t, len(searchR.Entities.Ids), 3)
assert.Equal(t, searchR.Entities.Ids, []int64{16, 15, 13})
assert.Equal(t, len(searchR.Entities.ValidRow), 3)
assert.Equal(t, searchR.Entities.ValidRow, []bool{true, true, true})
assert.Equal(t, len(searchR.Entities.RowsData), 3)
assert.Equal(t, searchR.Entities.RowsData, []*pb.RowData{
{Blob: uint64ToBytes(16)},
{Blob: uint64ToBytes(15)},
{Blob: uint64ToBytes(13)},
})
assert.Equal(t, len(searchR.Scores), 3)
assert.Equal(t, searchR.Scores, []float32{16, 15, 13})
assert.Equal(t, len(searchR.Distances), 3)
assert.Equal(t, searchR.Distances, []float32{16, 15, 13})
time.Sleep(time.Second)
}
func TestReadProxyOptionsFromConfig(t *testing.T) {
conf, err := ReadProxyOptionsFromConfig()
assert.Nil(t, err)
t.Log(conf.address)
t.Log(conf.master_address)
t.Log(conf.collectionMetaRootPath)
t.Log(conf.pulsarAddr)
t.Log(conf.readerTopicsPrefix)
t.Log(conf.numReadTopics)
t.Log(conf.deleteTopic)
t.Log(conf.queryTopic)
t.Log(conf.resultTopic)
t.Log(conf.resultGroup)
t.Log(conf.numReaderNode)
t.Log(conf.proxyId)
t.Log(conf.etcdEndpoints)
t.Log(conf.tsoRootPath)
t.Log(conf.tsoSaveInterval)
t.Log(conf.timeTickInterval)
t.Log(conf.timeTickTopic)
t.Log(conf.timeTickPeerId)
}

View File

@ -1,173 +0,0 @@
package proxy
import (
"log"
"sync"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
)
type queryReq struct {
internalpb.SearchRequest
result []*internalpb.SearchResult
wg sync.WaitGroup
proxy *proxyServer
}
// BaseRequest interfaces
func (req *queryReq) Type() internalpb.MsgType {
return req.MsgType
}
func (req *queryReq) PreExecute() commonpb.Status {
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (req *queryReq) Execute() commonpb.Status {
req.proxy.reqSch.queryChan <- req
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (req *queryReq) PostExecute() commonpb.Status { // send into pulsar
req.wg.Add(1)
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (req *queryReq) WaitToFinish() commonpb.Status { // wait unitl send into pulsar
req.wg.Wait()
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
}
func (s *proxyServer) restartQueryRoutine(buf_size int) error {
s.reqSch.queryChan = make(chan *queryReq, buf_size)
pulsarClient, err := pulsar.NewClient(pulsar.ClientOptions{URL: s.pulsarAddr})
if err != nil {
return nil
}
query, err := pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: s.queryTopic})
if err != nil {
return err
}
result, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topic: s.resultTopic,
SubscriptionName: s.resultGroup,
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
if err != nil {
return err
}
resultMap := make(map[UniqueID]*queryReq)
go func() {
defer result.Close()
defer query.Close()
defer pulsarClient.Close()
for {
select {
case <-s.ctx.Done():
return
case qm := <-s.reqSch.queryChan:
ts, err := s.getTimestamp(1)
if err != nil {
log.Printf("get time stamp failed")
break
}
qm.Timestamp = uint64(ts[0])
qb, err := proto.Marshal(qm)
if err != nil {
log.Printf("Marshal QueryReqMsg failed, error = %v", err)
continue
}
if _, err := query.Send(s.ctx, &pulsar.ProducerMessage{Payload: qb}); err != nil {
log.Printf("post into puslar failed, error = %v", err)
}
s.reqSch.qTimestampMux.Lock()
if s.reqSch.qTimestamp <= ts[0] {
s.reqSch.qTimestamp = ts[0]
} else {
log.Printf("there is some wrong with q_timestamp, it goes back, current = %d, previous = %d", ts[0], s.reqSch.qTimestamp)
}
s.reqSch.qTimestampMux.Unlock()
resultMap[qm.ReqId] = qm
//log.Printf("start search, query id = %d", qm.QueryId)
case cm, ok := <-result.Chan():
if !ok {
log.Printf("consumer of result topic has closed")
return
}
var rm internalpb.SearchResult
if err := proto.Unmarshal(cm.Message.Payload(), &rm); err != nil {
log.Printf("Unmarshal QueryReqMsg failed, error = %v", err)
break
}
if rm.ProxyId != s.proxyId {
break
}
qm, ok := resultMap[rm.ReqId]
if !ok {
log.Printf("unknown query id = %d", rm.ReqId)
break
}
qm.result = append(qm.result, &rm)
if len(qm.result) == s.numReaderNode {
qm.wg.Done()
delete(resultMap, rm.ReqId)
}
result.AckID(cm.ID())
}
}
}()
return nil
}
//func (s *proxyServer) reduceResult(query *queryReq) *servicepb.QueryResult {
//}
func (s *proxyServer) reduceResults(query *queryReq) *servicepb.QueryResult {
var results []*internalpb.SearchResult
var status commonpb.Status
status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
for _, r := range query.result {
status = *r.Status
if status.ErrorCode == commonpb.ErrorCode_SUCCESS {
results = append(results, r)
} else {
break
}
}
if len(results) != s.numReaderNode {
status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
}
if status.ErrorCode != commonpb.ErrorCode_SUCCESS {
result := servicepb.QueryResult{
Status: &status,
}
return &result
}
if s.numReaderNode == 1 {
result := servicepb.QueryResult{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
Hits: results[0].Hits,
}
return &result
}
return &servicepb.QueryResult{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
}
}

View File

@ -1,58 +0,0 @@
package proxy
import (
"sync"
)
type requestScheduler struct {
//definitions requestQueue
//manipulations requestQueue
manipulationsChan chan *manipulationReq // manipulation queue
mTimestamp Timestamp
mTimestampMux sync.Mutex
//queries requestQueue
queryChan chan *queryReq
qTimestamp Timestamp
qTimestampMux sync.Mutex
}
// @param selection
// bit_0 = 1: select definition queue
// bit_1 = 1: select manipulation queue
// bit_2 = 1: select query queue
// example: if mode = 3, then both definition and manipulation queues are selected
func (rs *requestScheduler) AreRequestsDelivered(ts Timestamp, selection uint32) bool {
r1 := func() bool {
if selection&uint32(2) == 0 {
return true
}
rs.mTimestampMux.Lock()
defer rs.mTimestampMux.Unlock()
if rs.mTimestamp >= ts {
return true
}
if len(rs.manipulationsChan) == 0 {
return true
}
return false
}()
r2 := func() bool {
if selection&uint32(4) == 0 {
return true
}
rs.qTimestampMux.Lock()
defer rs.qTimestampMux.Unlock()
if rs.qTimestamp >= ts {
return true
}
if len(rs.queryChan) == 0 {
return true
}
return false
}()
return r1 && r2
}

View File

@ -1,477 +0,0 @@
package proxy
import (
"context"
"encoding/json"
"fmt"
"log"
"net"
"sync"
"time"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
mpb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
etcd "go.etcd.io/etcd/clientv3"
"go.uber.org/atomic"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
const (
keyCollectionPath = "collection"
keySegmentPath = "segment"
)
type proxyServer struct {
servicepb.UnimplementedMilvusServiceServer
address string
masterAddress string
rootPath string // etcd root path
pulsarAddr string // pulsar address for reader
readerTopics []string //reader topics
deleteTopic string
queryTopic string
resultTopic string
resultGroup string
numReaderNode int
proxyId int64
getTimestamp func(count uint32) ([]Timestamp, error)
client *etcd.Client
ctx context.Context
wg sync.WaitGroup
////////////////////////////////////////////////////////////////
masterConn *grpc.ClientConn
masterClient mpb.MasterClient
grpcServer *grpc.Server
reqSch *requestScheduler
///////////////////////////////////////////////////////////////
collectionList map[int64]*etcdpb.CollectionMeta
nameCollectionId map[string]int64
segmentList map[int64]*etcdpb.SegmentMeta
collectionMux sync.Mutex
queryId atomic.Int64
}
func (s *proxyServer) CreateCollection(ctx context.Context, req *schemapb.CollectionSchema) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (s *proxyServer) DropCollection(ctx context.Context, req *servicepb.CollectionName) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (s *proxyServer) HasCollection(ctx context.Context, req *servicepb.CollectionName) (*servicepb.BoolResponse, error) {
return &servicepb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
Value: true,
}, nil
}
func (s *proxyServer) DescribeCollection(ctx context.Context, req *servicepb.CollectionName) (*servicepb.CollectionDescription, error) {
return &servicepb.CollectionDescription{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}
func (s *proxyServer) ShowCollections(ctx context.Context, req *commonpb.Empty) (*servicepb.StringListResponse, error) {
return &servicepb.StringListResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}
func (s *proxyServer) CreatePartition(ctx context.Context, in *servicepb.PartitionName) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (s *proxyServer) DropPartition(ctx context.Context, in *servicepb.PartitionName) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: 0,
Reason: "",
}, nil
}
func (s *proxyServer) HasPartition(ctx context.Context, in *servicepb.PartitionName) (*servicepb.BoolResponse, error) {
return &servicepb.BoolResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
Value: true,
}, nil
}
func (s *proxyServer) DescribePartition(ctx context.Context, in *servicepb.PartitionName) (*servicepb.PartitionDescription, error) {
return &servicepb.PartitionDescription{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}
func (s *proxyServer) ShowPartitions(ctx context.Context, req *servicepb.CollectionName) (*servicepb.StringListResponse, error) {
return &servicepb.StringListResponse{
Status: &commonpb.Status{
ErrorCode: 0,
Reason: "",
},
}, nil
}
func (s *proxyServer) Insert(ctx context.Context, req *servicepb.RowBatch) (*servicepb.IntegerRangeResponse, error) {
log.Printf("Insert Entities, total = %d", len(req.RowData))
msgMap := make(map[uint32]*pb.ManipulationReqMsg)
//TODO check collection schema's auto_id
if len(req.RowData) == 0 { //primary key is empty, set primary key by server
log.Printf("Set primary key")
}
if len(req.HashKeys) != len(req.RowData) {
return &servicepb.IntegerRangeResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: fmt.Sprintf("length of EntityIdArray not equal to lenght of RowsData"),
},
}, nil
}
for i := 0; i < len(req.HashKeys); i++ {
key := int64(req.HashKeys[i])
hash, err := typeutil.Hash32Int64(key)
if err != nil {
return nil, status.Errorf(codes.Unknown, "hash failed on %d", key)
}
hash = hash % uint32(len(s.readerTopics))
ipm, ok := msgMap[hash]
if !ok {
segId, err := s.getSegmentId(int32(hash), req.CollectionName)
if err != nil {
return nil, err
}
msgMap[hash] = &pb.ManipulationReqMsg{
CollectionName: req.CollectionName,
PartitionTag: req.PartitionTag,
SegmentId: segId,
ChannelId: int64(hash),
ReqType: pb.ReqType_kInsert,
ProxyId: s.proxyId,
//ExtraParams: req.ExtraParams,
}
ipm = msgMap[hash]
}
ipm.PrimaryKeys = append(ipm.PrimaryKeys, key)
ipm.RowsData = append(ipm.RowsData, &pb.RowData{Blob: req.RowData[i].Value}) // czs_tag
}
// TODO: alloc manipulation request id
mReq := manipulationReq{
stats: make([]commonpb.Status, len(msgMap)),
msgs: make([]*pb.ManipulationReqMsg, len(msgMap)),
wg: sync.WaitGroup{},
proxy: s,
}
for _, v := range msgMap {
mReq.msgs = append(mReq.msgs, v)
}
if st := mReq.PreExecute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS { //do nothing
return &servicepb.IntegerRangeResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
}, nil
}
if st := mReq.Execute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS { // push into chan
return &servicepb.IntegerRangeResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
}, nil
}
if st := mReq.PostExecute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS { //post to pulsar
return &servicepb.IntegerRangeResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
}, nil
}
if st := mReq.WaitToFinish(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
log.Printf("Wait to finish failed, error code = %d", st.ErrorCode)
}
return &servicepb.IntegerRangeResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
}, nil
}
func (s *proxyServer) Search(ctx context.Context, req *servicepb.Query) (*servicepb.QueryResult, error) {
qm := &queryReq{
SearchRequest: internalpb.SearchRequest{
MsgType: internalpb.MsgType_kSearch,
ProxyId: s.proxyId,
ReqId: s.queryId.Add(1),
Timestamp: 0,
ResultChannelId: 0,
},
proxy: s,
}
log.Printf("search on collection %s, proxy id = %d, query id = %d", req.CollectionName, qm.ProxyId, qm.ReqId)
if st := qm.PreExecute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
return &servicepb.QueryResult{
Status: &st,
}, nil
}
if st := qm.Execute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
return &servicepb.QueryResult{
Status: &st,
}, nil
}
if st := qm.PostExecute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
return &servicepb.QueryResult{
Status: &st,
}, nil
}
if st := qm.WaitToFinish(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
return &servicepb.QueryResult{
Status: &st,
}, nil
}
return s.reduceResults(qm), nil
}
//check if proxySerer is set correct
func (s *proxyServer) check() error {
if len(s.address) == 0 {
return fmt.Errorf("proxy address is unset")
}
if len(s.masterAddress) == 0 {
return fmt.Errorf("master address is unset")
}
if len(s.rootPath) == 0 {
return fmt.Errorf("root path for etcd is unset")
}
if len(s.pulsarAddr) == 0 {
return fmt.Errorf("pulsar address is unset")
}
if len(s.readerTopics) == 0 {
return fmt.Errorf("reader topics is unset")
}
if len(s.deleteTopic) == 0 {
return fmt.Errorf("delete topic is unset")
}
if len(s.queryTopic) == 0 {
return fmt.Errorf("query topic is unset")
}
if len(s.resultTopic) == 0 {
return fmt.Errorf("result topic is unset")
}
if len(s.resultGroup) == 0 {
return fmt.Errorf("result group is unset")
}
if s.numReaderNode <= 0 {
return fmt.Errorf("number of reader nodes is unset")
}
if s.proxyId <= 0 {
return fmt.Errorf("proxyId is unset")
}
log.Printf("proxy id = %d", s.proxyId)
if s.getTimestamp == nil {
return fmt.Errorf("getTimestamp is unset")
}
if s.client == nil {
return fmt.Errorf("etcd client is unset")
}
if s.ctx == nil {
return fmt.Errorf("context is unset")
}
return nil
}
func (s *proxyServer) getSegmentId(channelId int32, colName string) (int64, error) {
s.collectionMux.Lock()
defer s.collectionMux.Unlock()
colId, ok := s.nameCollectionId[colName]
if !ok {
return 0, status.Errorf(codes.Unknown, "can't get collection id of %s", colName)
}
colInfo, ok := s.collectionList[colId]
if !ok {
return 0, status.Errorf(codes.Unknown, "can't get collection, name = %s, id = %d", colName, colId)
}
for _, segId := range colInfo.SegmentIds {
_, ok := s.segmentList[segId]
if !ok {
return 0, status.Errorf(codes.Unknown, "can't get segment of %d", segId)
}
return segId, nil
}
return 0, status.Errorf(codes.Unknown, "can't get segment id, channel id = %d", channelId)
}
func (s *proxyServer) connectMaster() error {
ctx, _ := context.WithTimeout(context.Background(), 2*time.Second)
conn, err := grpc.DialContext(ctx, s.masterAddress, grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
log.Printf("Connect to master failed, error= %v", err)
return err
}
log.Printf("Connected to master, master_addr=%s", s.masterAddress)
s.masterConn = conn
s.masterClient = mpb.NewMasterClient(conn)
return nil
}
func (s *proxyServer) Close() {
s.client.Close()
s.masterConn.Close()
s.grpcServer.Stop()
}
func (s *proxyServer) StartGrpcServer() error {
lis, err := net.Listen("tcp", s.address)
if err != nil {
return err
}
go func() {
s.wg.Add(1)
defer s.wg.Done()
server := grpc.NewServer()
servicepb.RegisterMilvusServiceServer(server, s)
err := server.Serve(lis)
if err != nil {
log.Fatalf("Proxy grpc server fatal error=%v", err)
}
}()
return nil
}
func (s *proxyServer) WatchEtcd() error {
s.collectionMux.Lock()
defer s.collectionMux.Unlock()
cos, err := s.client.Get(s.ctx, s.rootPath+"/"+keyCollectionPath, etcd.WithPrefix())
if err != nil {
return err
}
for _, cob := range cos.Kvs {
// TODO: simplify collection struct
var co etcdpb.CollectionMeta
var mco collection.Collection
if err := json.Unmarshal(cob.Value, &mco); err != nil {
return err
}
proto.UnmarshalText(mco.GrpcMarshalString, &co)
s.nameCollectionId[co.Schema.Name] = co.Id
s.collectionList[co.Id] = &co
log.Printf("watch collection, name = %s, id = %d", co.Schema.Name, co.Id)
}
segs, err := s.client.Get(s.ctx, s.rootPath+"/"+keySegmentPath, etcd.WithPrefix())
if err != nil {
return err
}
for _, segb := range segs.Kvs {
var seg etcdpb.SegmentMeta
if err := json.Unmarshal(segb.Value, &seg); err != nil {
return err
}
s.segmentList[seg.SegmentId] = &seg
log.Printf("watch segment id = %d\n", seg.SegmentId)
}
cow := s.client.Watch(s.ctx, s.rootPath+"/"+keyCollectionPath, etcd.WithPrefix(), etcd.WithRev(cos.Header.Revision+1))
segw := s.client.Watch(s.ctx, s.rootPath+"/"+keySegmentPath, etcd.WithPrefix(), etcd.WithRev(segs.Header.Revision+1))
go func() {
s.wg.Add(1)
defer s.wg.Done()
for {
select {
case <-s.ctx.Done():
return
case coe := <-cow:
func() {
s.collectionMux.Lock()
defer s.collectionMux.Unlock()
for _, e := range coe.Events {
var co etcdpb.CollectionMeta
var mco collection.Collection
if err := json.Unmarshal(e.Kv.Value, &mco); err != nil {
log.Printf("unmarshal Collection failed, error = %v", err)
} else {
proto.UnmarshalText(mco.GrpcMarshalString, &co)
s.nameCollectionId[co.Schema.Name] = co.Id
s.collectionList[co.Id] = &co
log.Printf("watch collection, name = %s, id = %d", co.Schema.Name, co.Id)
}
}
}()
case sege := <-segw:
func() {
s.collectionMux.Lock()
defer s.collectionMux.Unlock()
for _, e := range sege.Events {
var seg etcdpb.SegmentMeta
if err := json.Unmarshal(e.Kv.Value, &seg); err != nil {
log.Printf("unmarshal Segment failed, error = %v", err)
} else {
s.segmentList[seg.SegmentId] = &seg
log.Printf("watch segment id = %d\n", seg.SegmentId)
}
}
}()
}
}
}()
return nil
}
func startProxyServer(srv *proxyServer) error {
if err := srv.check(); err != nil {
return err
}
srv.reqSch = &requestScheduler{}
if err := srv.restartManipulationRoutine(1024); err != nil {
return err
}
if err := srv.restartQueryRoutine(1024); err != nil {
return err
}
srv.nameCollectionId = make(map[string]int64)
srv.collectionList = make(map[int64]*etcdpb.CollectionMeta)
srv.segmentList = make(map[int64]*etcdpb.SegmentMeta)
if err := srv.connectMaster(); err != nil {
return err
}
if err := srv.WatchEtcd(); err != nil {
return err
}
srv.queryId.Store(time.Now().UnixNano())
return srv.StartGrpcServer()
}

View File

@ -1,525 +0,0 @@
package proxy
import (
"context"
"encoding/binary"
"encoding/json"
"net"
"sort"
"testing"
"time"
"unsafe"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
mpb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
)
type testMasterServer struct {
mpb.UnimplementedMasterServer
}
func (s *testMasterServer) CreateCollection(ctx context.Context, in *internalpb.CreateCollectionRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
}, nil
}
func (*testMasterServer) CreateIndex(ctx context.Context, req *pb.IndexParam) (*pb.Status, error) {
return &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS, Reason: req.IndexName}, nil
}
func startTestMaster(master_addr string, t *testing.T) *grpc.Server {
lis, err := net.Listen("tcp", master_addr)
assert.Nil(t, err)
s := grpc.NewServer()
mpb.RegisterMasterServer(s, &testMasterServer{})
go func() {
if err := s.Serve(lis); err != nil {
t.Fatal(err)
}
}()
return s
}
func startTestProxyServer(proxy_addr string, master_addr string, t *testing.T) *proxyServer {
client, err := clientv3.New(clientv3.Config{Endpoints: []string{"127.0.0.1:2379"}})
assert.Nil(t, err)
ctx, _ := context.WithTimeout(context.Background(), 2*time.Second)
var timestamp uint64 = 1000
p := &proxyServer{
address: proxy_addr,
masterAddress: master_addr,
rootPath: "/proxy/root",
pulsarAddr: "pulsar://localhost:6650",
readerTopics: []string{"reader1", "reader2"},
deleteTopic: "deleteT",
queryTopic: "queryer",
resultTopic: "resulter",
resultGroup: "reusltG",
numReaderNode: 2,
proxyId: 1,
getTimestamp: func(count uint32) ([]Timestamp, error) {
timestamp += 100
t := make([]Timestamp, count)
for i := 0; i < int(count); i++ {
t[i] = timestamp
}
return t, nil
},
client: client,
ctx: ctx,
}
go func() {
if err := startProxyServer(p); err != nil {
t.Fatal(err)
}
}()
return p
}
func uint64ToBytes(v uint64) []byte {
b := make([]byte, unsafe.Sizeof(v))
binary.LittleEndian.PutUint64(b, v)
return b
}
func TestProxyServer_CreateCollectionAndIndex(t *testing.T) {
_ = startTestMaster("localhost:10000", t)
//defer ms.Stop()
time.Sleep(100 * time.Millisecond)
ps := startTestProxyServer("localhost:10001", "localhost:10000", t)
//defer ps.Close()
time.Sleep(100 * time.Millisecond)
ctx := ps.ctx
conn, err := grpc.DialContext(ctx, "localhost:10001", grpc.WithInsecure(), grpc.WithBlock())
assert.Nil(t, err)
defer conn.Close()
cli := pb.NewMilvusServiceClient(conn)
st, err := cli.CreateCollection(ctx, &pb.Mapping{CollectionName: "testCollectionName"})
assert.Nil(t, err)
assert.Equalf(t, st.ErrorCode, pb.ErrorCode_SUCCESS, "CreateCollection failed")
assert.Equalf(t, st.Reason, "testCollectionName", "CreateCollection failed")
st, err = cli.CreateIndex(ctx, &pb.IndexParam{IndexName: "testIndexName"})
assert.Nil(t, err)
assert.Equalf(t, st.ErrorCode, pb.ErrorCode_SUCCESS, "CreateIndex failed")
assert.Equalf(t, st.Reason, "testIndexName", "CreateIndex failed")
}
func TestProxyServer_WatchEtcd(t *testing.T) {
client, err := clientv3.New(clientv3.Config{Endpoints: []string{"127.0.0.1:2379"}})
assert.Nil(t, err)
defer client.Close()
ctx, _ := context.WithTimeout(context.Background(), 2*time.Second)
col1 := etcdpb.CollectionMeta{
Id: 1,
SegmentIds: []int64{2, 3},
}
seg2 := etcdpb.SegmentMeta{
SegmentId: 2,
NumRows: 10,
}
seg3 := etcdpb.SegmentMeta{
SegmentId: 3,
NumRows: 10,
}
if cb1, err := json.Marshal(&col1); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ctx, "/proxy/root/"+keyCollectionPath+"/1", string(cb1)); err != nil {
t.Fatal(err)
}
if sb2, err := json.Marshal(&seg2); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ctx, "/proxy/root/"+keySegmentPath+"/2", string(sb2)); err != nil {
t.Fatal(err)
}
if sb3, err := json.Marshal(&seg3); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ctx, "/proxy/root/"+keySegmentPath+"/3", string(sb3)); err != nil {
t.Fatal(err)
}
_ = startTestMaster("localhost:10002", t)
//defer ms.Stop()
time.Sleep(100 * time.Millisecond)
ps := startTestProxyServer("localhost:10003", "localhost:10002", t)
//defer ps.Close()
time.Sleep(100 * time.Millisecond)
conn, err := grpc.DialContext(ps.ctx, "localhost:10003", grpc.WithInsecure(), grpc.WithBlock())
assert.Nil(t, err)
defer conn.Close()
cli := pb.NewMilvusServiceClient(conn)
cr, err := cli.CountCollection(ps.ctx, &pb.CollectionName{CollectionName: "c1"})
assert.Nil(t, err)
assert.Equalf(t, cr.Status.ErrorCode, pb.ErrorCode_SUCCESS, "CountCollection failed : %s", cr.Status.Reason)
assert.Equalf(t, cr.CollectionRowCount, int64(20), "collection count expect to be 20, count = %d", cr.CollectionRowCount)
col4 := etcdpb.CollectionMeta{
Id: 4,
SegmentIds: []int64{5},
}
seg5 := etcdpb.SegmentMeta{
SegmentId: 5,
NumRows: 10,
}
if cb4, err := json.Marshal(&col4); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ps.ctx, "/proxy/root/"+keyCollectionPath+"/4", string(cb4)); err != nil {
t.Fatal(err)
}
if sb5, err := json.Marshal(&seg5); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ps.ctx, "/proxy/root/"+keySegmentPath+"/5", string(sb5)); err != nil {
t.Fatal(err)
}
cr, err = cli.CountCollection(ps.ctx, &pb.CollectionName{CollectionName: "c4"})
assert.Nil(t, err)
assert.Equalf(t, cr.Status.ErrorCode, pb.ErrorCode_SUCCESS, "CountCollection failed : %s", cr.Status.Reason)
assert.Equalf(t, cr.CollectionRowCount, int64(10), "collection count expect to be 10, count = %d", cr.CollectionRowCount)
}
func TestProxyServer_InsertAndDelete(t *testing.T) {
client, err := clientv3.New(clientv3.Config{Endpoints: []string{"127.0.0.1:2379"}})
assert.Nil(t, err)
defer client.Close()
ctx, _ := context.WithTimeout(context.Background(), 2*time.Second)
col10 := etcdpb.CollectionMeta{
Id: 10,
Schema: nil,
CreateTime: 0,
SegmentIds: []int64{11, 12},
PartitionTags: nil,
}
seg11 := etcdpb.SegmentMeta{
SegmentId: 11,
CollectionId: 10,
ChannelStart: 0,
ChannelEnd: 1,
}
seg12 := etcdpb.SegmentMeta{
SegmentId: 12,
CollectionId: 10,
ChannelStart: 1,
ChannelEnd: 2,
}
if cb10, err := json.Marshal(&col10); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ctx, "/proxy/root/"+keyCollectionPath+"/10", string(cb10)); err != nil {
t.Fatal(err)
}
if sb11, err := json.Marshal(&seg11); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ctx, "/proxy/root/"+keySegmentPath+"/11", string(sb11)); err != nil {
t.Fatal(err)
}
if sb12, err := json.Marshal(&seg12); err != nil {
t.Fatal(err)
} else if _, err := client.Put(ctx, "/proxy/root/"+keySegmentPath+"/12", string(sb12)); err != nil {
t.Fatal(err)
}
_ = startTestMaster("localhost:10004", t)
time.Sleep(100 * time.Millisecond)
ps := startTestProxyServer("localhost:10005", "localhost:10004", t)
//defer ps.Close()
time.Sleep(100 * time.Millisecond)
conn, err := grpc.DialContext(ps.ctx, "localhost:10005", grpc.WithInsecure(), grpc.WithBlock())
assert.Nil(t, err)
defer conn.Close()
pulsarClient, err := pulsar.NewClient(pulsar.ClientOptions{URL: ps.pulsarAddr})
assert.Nil(t, err)
defer pulsarClient.Close()
reader, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topics: ps.readerTopics,
SubscriptionName: "reader-group",
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
assert.Nil(t, err)
defer reader.Close()
deleter, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topic: ps.deleteTopic,
SubscriptionName: "delete-group",
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
pctx, _ := context.WithTimeout(ps.ctx, time.Second)
isbreak := false
for {
if isbreak {
break
}
select {
case <-pctx.Done():
isbreak = true
break
case cm, ok := <-reader.Chan():
if !ok {
t.Fatalf("reader closed")
}
reader.AckID(cm.ID())
break
case cm, ok := <-deleter.Chan():
assert.Truef(t, ok, "deleter closed")
deleter.AckID(cm.ID())
}
}
ip := pb.InsertParam{
CollectionName: "col10",
Schema: nil,
RowsData: []*pb.RowData{
{Blob: uint64ToBytes(1)},
{Blob: uint64ToBytes(2)},
{Blob: uint64ToBytes(3)},
{Blob: uint64ToBytes(4)},
{Blob: uint64ToBytes(5)},
},
EntityIdArray: []int64{1, 2, 3, 4, 5},
PartitionTag: "",
ExtraParams: nil,
}
dp := pb.DeleteByIDParam{
CollectionName: "deleteCollection",
IdArray: []int64{1, 2, 3, 4, 5},
}
serverClient := pb.NewMilvusServiceClient(conn)
ir, err := serverClient.Insert(ps.ctx, &ip)
assert.Nil(t, err)
assert.Equalf(t, ir.Status.ErrorCode, pb.ErrorCode_SUCCESS, "Insert failed, error code = %d, reason = %s", ir.Status.ErrorCode, ir.Status.Reason)
assert.Equalf(t, len(ir.EntityIdArray), 5, "insert failed, len(ir.EntityIdArray) expect to be 5")
sort.Slice(ir.EntityIdArray, func(i int, j int) bool { return ir.EntityIdArray[i] < ir.EntityIdArray[j] })
for i := 0; i < 5; i++ {
assert.Equal(t, ir.EntityIdArray[i], int64(i+1))
}
dr, err := serverClient.DeleteByID(ps.ctx, &dp)
assert.Nil(t, err)
assert.Equalf(t, dr.ErrorCode, pb.ErrorCode_SUCCESS, "delete failed, error code = %d, reason = %s", dr.ErrorCode, dr.Reason)
var primaryKey []int64
isbreak = false
for {
if isbreak {
break
}
select {
case <-ps.ctx.Done():
isbreak = true
break
case cm, ok := <-reader.Chan():
assert.Truef(t, ok, "reader closed")
msg := cm.Message
var m pb.ManipulationReqMsg
if err := proto.Unmarshal(msg.Payload(), &m); err != nil {
t.Fatal(err)
}
for i, k := range m.PrimaryKeys {
primaryKey = append(primaryKey, k)
rowValue := binary.LittleEndian.Uint64(m.RowsData[i].Blob)
t.Logf("primary key = %d, rowvalue =%d", k, rowValue)
assert.Equalf(t, k, rowValue, "key expect equal to row value")
}
reader.AckID(cm.ID())
break
case cm, ok := <-deleter.Chan():
assert.Truef(t, ok, "deleter closed")
var m pb.ManipulationReqMsg
if err := proto.Unmarshal(cm.Message.Payload(), &m); err != nil {
t.Fatal(err)
}
assert.Equalf(t, m.CollectionName, "deleteCollection", "delete failed, collection name = %s", m.CollectionName)
assert.Equalf(t, len(m.PrimaryKeys), 5, "delete failed,len(m.PrimaryKeys) = %d", len(m.PrimaryKeys))
for i, v := range m.PrimaryKeys {
assert.Equalf(t, v, uint64(i+1), "delete failed")
}
}
}
assert.Equalf(t, len(primaryKey), 5, "Receive from pulsar failed")
sort.Slice(primaryKey, func(i int, j int) bool { return primaryKey[i] < primaryKey[j] })
for i := 0; i < 5; i++ {
assert.Equalf(t, primaryKey[i], uint64(i+1), "insert failed")
}
t.Logf("m_timestamp = %d", ps.reqSch.mTimestamp)
assert.Equalf(t, ps.reqSch.mTimestamp, Timestamp(1300), "insert failed")
}
func TestProxyServer_Search(t *testing.T) {
_ = startTestMaster("localhost:10006", t)
time.Sleep(100 * time.Millisecond)
ps := startTestProxyServer("localhost:10007", "localhost:10006", t)
time.Sleep(100 * time.Millisecond)
conn, err := grpc.DialContext(ps.ctx, "localhost:10007", grpc.WithInsecure(), grpc.WithBlock())
assert.Nil(t, err)
defer conn.Close()
pulsarClient, err := pulsar.NewClient(pulsar.ClientOptions{URL: ps.pulsarAddr})
assert.Nil(t, err)
defer pulsarClient.Close()
query, err := pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topic: ps.queryTopic,
SubscriptionName: "query-group",
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
})
assert.Nil(t, err)
defer query.Close()
result, err := pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: ps.resultTopic})
assert.Nil(t, err)
defer result.Close()
pctx, _ := context.WithTimeout(ps.ctx, time.Second)
func() {
for {
select {
case <-pctx.Done():
return
case cm, ok := <-query.Chan():
if !ok {
t.Fatal("query topic is closed")
}
query.AckID(cm.ID())
}
}
}()
go func() {
cm, ok := <-query.Chan()
query.AckID(cm.ID())
assert.Truef(t, ok, "query topic is closed")
var qm pb.QueryReqMsg
if err := proto.Unmarshal(cm.Payload(), &qm); err != nil {
t.Fatal(err)
}
if qm.ProxyId != ps.proxyId {
t.Fatalf("search failed, incorrect proxy id = %d", qm.ProxyId)
}
if qm.CollectionName != "collection_search" {
t.Fatalf("search failed, incorrect collection name = %s", qm.CollectionName)
}
r1 := pb.QueryResult{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Entities: &pb.Entities{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Ids: []int64{1, 3, 5},
ValidRow: []bool{true, true, true},
RowsData: []*pb.RowData{
{Blob: uint64ToBytes(1)},
{Blob: uint64ToBytes(3)},
{Blob: uint64ToBytes(5)},
},
},
RowNum: 3,
Scores: []float32{1, 3, 5},
Distances: []float32{1, 3, 5},
ExtraParams: nil,
QueryId: qm.QueryId,
ProxyId: qm.ProxyId,
}
r2 := pb.QueryResult{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Entities: &pb.Entities{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},
Ids: []int64{2, 4, 6},
ValidRow: []bool{true, false, true},
RowsData: []*pb.RowData{
{Blob: uint64ToBytes(2)},
{Blob: uint64ToBytes(4)},
{Blob: uint64ToBytes(6)},
},
},
RowNum: 3,
Scores: []float32{2, 4, 6},
Distances: []float32{2, 4, 6},
ExtraParams: nil,
QueryId: qm.QueryId,
ProxyId: qm.ProxyId,
}
b1, err := proto.Marshal(&r1)
assert.Nil(t, err)
b2, err := proto.Marshal(&r2)
assert.Nil(t, err)
if _, err := result.Send(ps.ctx, &pulsar.ProducerMessage{Payload: b1}); err != nil {
t.Fatal(err)
}
if _, err := result.Send(ps.ctx, &pulsar.ProducerMessage{Payload: b2}); err != nil {
t.Fatal(err)
}
}()
sm := pb.SearchParam{
CollectionName: "collection_search",
VectorParam: nil,
Dsl: "",
PartitionTag: nil,
ExtraParams: nil,
}
serverClient := pb.NewMilvusServiceClient(conn)
qr, err := serverClient.Search(ps.ctx, &sm)
assert.Nil(t, err)
assert.Equalf(t, qr.Status.ErrorCode, pb.ErrorCode_SUCCESS, "query failed")
assert.Equalf(t, qr.Entities.Status.ErrorCode, pb.ErrorCode_SUCCESS, "query failed")
assert.Equalf(t, len(qr.Entities.Ids), 3, "query failed")
assert.Equalf(t, qr.Entities.Ids, []int64{6, 5, 3}, "query failed")
assert.Equalf(t, len(qr.Entities.ValidRow), 3, "query failed")
assert.Equalf(t, qr.Entities.ValidRow, []bool{true, true, true}, "query failed")
assert.Equalf(t, len(qr.Entities.RowsData), 3, "query failed")
assert.Equalf(t, qr.Entities.RowsData, []*pb.RowData{
{Blob: uint64ToBytes(6)},
{Blob: uint64ToBytes(5)},
{Blob: uint64ToBytes(3)},
}, "query failed")
assert.Equalf(t, len(qr.Scores), 3, "query failed")
assert.Equalf(t, qr.Scores, []float32{6, 5, 3}, "query failed")
assert.Equalf(t, len(qr.Distances), 3, "query failed")
assert.Equalf(t, qr.Distances, []float32{6, 5, 3}, "query failed")
}

View File

@ -1,40 +1,162 @@
package proxy
import (
"context"
"errors"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"log"
)
type task interface {
Id() UniqueID // return ReqId
Type() internalpb.MsgType
GetTs() Timestamp
BeginTs() Timestamp
EndTs() Timestamp
SetTs(ts Timestamp)
PreExecute() error
Execute() error
PostExecute() error
WaitToFinish() error
Notify() error
Notify(err error)
}
type baseTask struct {
ReqType internalpb.MsgType
ReqId UniqueID
Ts Timestamp
ProxyId UniqueID
type baseInsertTask = msgstream.InsertMsg
type InsertTask struct {
baseInsertTask
ts Timestamp
done chan error
resultChan chan *servicepb.IntegerRangeResponse
manipulationMsgStream *msgstream.PulsarMsgStream
ctx context.Context
cancel context.CancelFunc
}
func (bt *baseTask) Id() UniqueID {
return bt.ReqId
func (it *InsertTask) SetTs(ts Timestamp) {
it.ts = ts
}
func (bt *baseTask) Type() internalpb.MsgType {
return bt.ReqType
func (it *InsertTask) BeginTs() Timestamp {
return it.ts
}
func (bt *baseTask) GetTs() Timestamp {
return bt.Ts
func (it *InsertTask) EndTs() Timestamp {
return it.ts
}
func (bt *baseTask) SetTs(ts Timestamp) {
bt.Ts = ts
func (it *InsertTask) Id() UniqueID {
return it.ReqId
}
func (it *InsertTask) Type() internalpb.MsgType {
return it.MsgType
}
func (it *InsertTask) PreExecute() error {
return nil
}
func (it *InsertTask) Execute() error {
var tsMsg msgstream.TsMsg = it
msgPack := &msgstream.MsgPack{
BeginTs: it.BeginTs(),
EndTs: it.EndTs(),
Msgs: make([]*msgstream.TsMsg, 1),
}
msgPack.Msgs[0] = &tsMsg
it.manipulationMsgStream.Produce(msgPack)
return nil
}
func (it *InsertTask) PostExecute() error {
return nil
}
func (it *InsertTask) WaitToFinish() error {
defer it.cancel()
for {
select {
case err := <-it.done:
return err
case <-it.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
}
}
}
func (it *InsertTask) Notify(err error) {
it.done <- err
}
type CreateCollectionTask struct {
internalpb.CreateCollectionRequest
masterClient masterpb.MasterClient
done chan error
resultChan chan *commonpb.Status
ctx context.Context
cancel context.CancelFunc
}
func (cct *CreateCollectionTask) Id() UniqueID {
return cct.ReqId
}
func (cct *CreateCollectionTask) Type() internalpb.MsgType {
return cct.MsgType
}
func (cct *CreateCollectionTask) BeginTs() Timestamp {
return cct.Timestamp
}
func (cct *CreateCollectionTask) EndTs() Timestamp {
return cct.Timestamp
}
func (cct *CreateCollectionTask) SetTs(ts Timestamp) {
cct.Timestamp = ts
}
func (cct *CreateCollectionTask) PreExecute() error {
return nil
}
func (cct *CreateCollectionTask) Execute() error {
resp, err := cct.masterClient.CreateCollection(cct.ctx, &cct.CreateCollectionRequest)
if err != nil {
log.Printf("create collection failed, error= %v", err)
cct.resultChan <- &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: err.Error(),
}
} else {
cct.resultChan <- resp
}
return err
}
func (cct *CreateCollectionTask) PostExecute() error {
return nil
}
func (cct *CreateCollectionTask) WaitToFinish() error {
defer cct.cancel()
for {
select {
case err := <- cct.done:
return err
case <- cct.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
}
}
}
func (cct *CreateCollectionTask) Notify(err error) {
cct.done <- err
}

View File

@ -2,6 +2,7 @@ package proxy
import (
"container/list"
"context"
"log"
"sync"
)
@ -64,10 +65,10 @@ func (queue *baseTaskQueue) PopUnissuedTask() *task {
func (queue *baseTaskQueue) AddActiveTask(t *task) {
queue.atLock.Lock()
defer queue.atLock.Lock()
ts := (*t).GetTs()
ts := (*t).EndTs()
_, ok := queue.activeTasks[ts]
if ok {
log.Fatalf("task with timestamp %d already in active task list!", ts)
log.Fatalf("task with timestamp %v already in active task list!", ts)
}
queue.activeTasks[ts] = t
}
@ -88,7 +89,7 @@ func (queue *baseTaskQueue) TaskDoneTest(ts Timestamp) bool {
queue.utLock.Lock()
defer queue.utLock.Unlock()
for e := queue.unissuedTasks.Front(); e != nil; e = e.Next() {
if (*(e.Value.(*task))).GetTs() >= ts {
if (*(e.Value.(*task))).EndTs() >= ts {
return false
}
}
@ -124,100 +125,93 @@ func (queue *dqTaskQueue) Enqueue(t *task) error {
return nil
}
type taskScheduler struct {
type TaskScheduler struct {
DdQueue *ddTaskQueue
DmQueue *dmTaskQueue
DqQueue *dqTaskQueue
// tsAllocator, ReqIdAllocator
wg sync.WaitGroup
ctx context.Context
cancel context.CancelFunc
}
func (sched *taskScheduler) scheduleDdTask() *task {
func (sched *TaskScheduler) scheduleDdTask() *task {
return sched.DdQueue.PopUnissuedTask()
}
func (sched *taskScheduler) scheduleDmTask() *task {
func (sched *TaskScheduler) scheduleDmTask() *task {
return sched.DmQueue.PopUnissuedTask()
}
func (sched *taskScheduler) scheduleDqTask() *task {
func (sched *TaskScheduler) scheduleDqTask() *task {
return sched.DqQueue.PopUnissuedTask()
}
func (sched *taskScheduler) Start() error {
go func() {
for {
if sched.DdQueue.Empty() {
continue
}
t := sched.scheduleDdTask()
if err := (*t).PreExecute(); err != nil {
return
}
if err := (*t).Execute(); err != nil {
return
}
if err := (*t).PostExecute(); err != nil {
return
}
if err := (*t).WaitToFinish(); err != nil {
return
}
if err := (*t).Notify(); err != nil {
return
}
func (sched *TaskScheduler) definitionLoop() {
defer sched.wg.Done()
defer sched.cancel()
}
func (sched *TaskScheduler) manipulationLoop() {
defer sched.wg.Done()
defer sched.cancel()
for {
if sched.DmQueue.Empty() {
continue
}
}()
go func() {
for {
if sched.DdQueue.Empty() {
continue
}
t := sched.scheduleDmTask()
if err := (*t).PreExecute(); err != nil {
return
}
if err := (*t).Execute(); err != nil {
return
}
if err := (*t).PostExecute(); err != nil {
return
}
if err := (*t).WaitToFinish(); err != nil {
return
}
if err := (*t).Notify(); err != nil {
return
}
sched.DmQueue.atLock.Lock()
t := sched.scheduleDmTask()
if err := (*t).PreExecute(); err != nil {
return
}
}()
go func() {
for {
if sched.DdQueue.Empty() {
continue
go func() {
err := (*t).Execute()
if err != nil {
log.Printf("execute manipulation task failed, error = %v", err)
}
t := sched.scheduleDqTask()
if err := (*t).PreExecute(); err != nil {
return
}
if err := (*t).Execute(); err != nil {
return
}
if err := (*t).PostExecute(); err != nil {
return
}
if err := (*t).WaitToFinish(); err != nil {
return
}
if err := (*t).Notify(); err != nil {
return
}
}
}()
(*t).Notify(err)
}()
sched.DmQueue.AddActiveTask(t)
sched.DmQueue.atLock.Unlock()
go func() {
(*t).WaitToFinish()
(*t).PostExecute()
// remove from active list
sched.DmQueue.PopActiveTask((*t).EndTs())
}()
}
}
func (sched *TaskScheduler) queryLoop() {
defer sched.wg.Done()
defer sched.cancel()
}
func (sched *TaskScheduler) Start(ctx context.Context) error {
sched.ctx, sched.cancel = context.WithCancel(ctx)
sched.wg.Add(3)
go sched.definitionLoop()
go sched.manipulationLoop()
go sched.queryLoop()
return nil
}
func (sched *taskScheduler) TaskDoneTest(ts Timestamp) bool {
func (sched *TaskScheduler) Close() {
sched.cancel()
sched.wg.Wait()
}
func (sched *TaskScheduler) TaskDoneTest(ts Timestamp) bool {
ddTaskDone := sched.DdQueue.TaskDoneTest(ts)
dmTaskDone := sched.DmQueue.TaskDoneTest(ts)
dqTaskDone := sched.DqQueue.TaskDoneTest(ts)

View File

@ -62,7 +62,7 @@ func TestManipulationService_Start(t *testing.T) {
insertMessages := make([]*msgstream.TsMsg, 0)
for i := 0; i < msgLength; i++ {
var msg msgstream.TsMsg = &msgstream.InsertTask{
var msg msgstream.TsMsg = &msgstream.InsertMsg{
InsertRequest: internalPb.InsertRequest{
ReqId: int64(0),
CollectionName: "collection0",

View File

@ -14,7 +14,7 @@ type msgStreamMsg struct {
}
type dmMsg struct {
insertMessages []*msgstream.InsertTask
insertMessages []*msgstream.InsertMsg
// TODO: add delete message support
// deleteMessages []*msgstream.DeleteTask
timeRange TimeRange
@ -30,7 +30,7 @@ type schemaUpdateMsg struct {
}
type filteredDmMsg struct {
insertMessages []*msgstream.InsertTask
insertMessages []*msgstream.InsertMsg
// TODO: add delete message support
// deleteMessages []*msgstream.DeleteTask
timeRange TimeRange

View File

@ -34,14 +34,14 @@ func (msNode *msgStreamNode) Operate(in []*Msg) []*Msg {
// TODO: add time range check
var dmMsg = dmMsg{
insertMessages: make([]*msgstream.InsertTask, 0),
insertMessages: make([]*msgstream.InsertMsg, 0),
// deleteMessages: make([]*msgstream.DeleteTask, 0),
timeRange: streamMsg.timeRange,
}
for _, msg := range streamMsg.tsMessages {
switch (*msg).Type() {
case internalPb.MsgType_kInsert:
dmMsg.insertMessages = append(dmMsg.insertMessages, (*msg).(*msgstream.InsertTask))
dmMsg.insertMessages = append(dmMsg.insertMessages, (*msg).(*msgstream.InsertMsg))
// case msgstream.KDelete:
// dmMsg.deleteMessages = append(dmMsg.deleteMessages, (*msg).(*msgstream.DeleteTask))
default:

View File

@ -1,466 +0,0 @@
package timesync
import (
"context"
"encoding/json"
"fmt"
"log"
"os"
"sort"
"strconv"
"sync"
"time"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
type UniqueID = typeutil.UniqueID
type Timestamp = typeutil.Timestamp
type InsertLog struct {
MsgLength int
DurationInMilliseconds int64
InsertTime time.Time
NumSince int64
Speed float64
}
type TimeSyncRole int
const (
Reader TimeSyncRole = 0
Writer TimeSyncRole = 1
)
const ReadStopFlagEnd int64 = 0
type TimeSync interface {
Start() error
Close()
TimeSync() <-chan TimeSyncMsg
InsertOrDelete() <-chan *pb.InsertOrDeleteMsg
IsInsertDeleteChanFull() bool
}
type TimeSyncMsg struct {
Timestamp Timestamp
NumRecorders int64
}
type TimeSyncOption func(*TimeSyncCfg)
type TimeSyncCfg struct {
pulsarClient pulsar.Client
timeSyncConsumer pulsar.Consumer
readerConsumer pulsar.Consumer
readerProducer []pulsar.Producer
timesyncMsgChan chan TimeSyncMsg
insertOrDeleteChan chan *pb.InsertOrDeleteMsg //output insert or delete msg
readStopFlagClientId int64
interval int64
proxyIdList []int64
readerQueueSize int
revTimesyncFromReader map[uint64]int
ctx context.Context
InsertLogs []InsertLog
RoleType TimeSyncRole
}
/*
layout of timestamp
time ms logic number
/-------46 bit-----------\/------18bit-----\
+-------------------------+================+
*/
func toMillisecond(ts *internalpb.TimeTickMsg) int {
// get Millisecond in second
return int(ts.GetTimestamp() >> 18)
}
func NewTimeSync(
ctx context.Context,
timeSyncTopic string,
timeSyncSubName string,
readTopics []string,
readSubName string,
proxyIdList []UniqueID,
readStopFlagClientId int64,
opts ...TimeSyncOption,
) (TimeSync, error) {
pulsarAddr := "pulsar://"
pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
interval := int64(conf.Config.Timesync.Interval)
//check if proxyId has duplication
if len(proxyIdList) == 0 {
return nil, fmt.Errorf("proxy id list is empty")
}
if len(proxyIdList) > 1 {
sort.Slice(proxyIdList, func(i int, j int) bool { return proxyIdList[i] < proxyIdList[j] })
}
for i := 1; i < len(proxyIdList); i++ {
if proxyIdList[i] == proxyIdList[i-1] {
return nil, fmt.Errorf("there are two proxies have the same id = %d", proxyIdList[i])
}
}
r := &TimeSyncCfg{
interval: interval,
proxyIdList: proxyIdList,
}
for _, opt := range opts {
opt(r)
}
//check if read topic is empty
if len(readTopics) == 0 {
return nil, fmt.Errorf("read topic is empyt")
}
//set default value
if r.readerQueueSize == 0 {
r.readerQueueSize = 1024
}
if readStopFlagClientId >= ReadStopFlagEnd {
return nil, fmt.Errorf("read stop flag client id should less than %d", ReadStopFlagEnd)
}
r.readStopFlagClientId = readStopFlagClientId
r.timesyncMsgChan = make(chan TimeSyncMsg, len(readTopics)*r.readerQueueSize)
r.insertOrDeleteChan = make(chan *pb.InsertOrDeleteMsg, len(readTopics)*r.readerQueueSize)
r.revTimesyncFromReader = make(map[uint64]int)
r.ctx = ctx
var client pulsar.Client
var err error
if conf.Config.Pulsar.Authentication {
client, err = pulsar.NewClient(pulsar.ClientOptions{
URL: pulsarAddr,
Authentication: pulsar.NewAuthenticationToken(conf.Config.Pulsar.Token),
})
} else {
client, err = pulsar.NewClient(pulsar.ClientOptions{URL: pulsarAddr})
}
if err != nil {
return nil, fmt.Errorf("connect pulsar failed, %v", err)
}
r.pulsarClient = client
timeSyncChan := make(chan pulsar.ConsumerMessage, len(r.proxyIdList))
if r.timeSyncConsumer, err = r.pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topic: timeSyncTopic,
SubscriptionName: timeSyncSubName,
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
MessageChannel: timeSyncChan,
}); err != nil {
return nil, fmt.Errorf("failed to subscribe topic %s, error = %v", timeSyncTopic, err)
}
readerChan := make(chan pulsar.ConsumerMessage, len(readTopics)*r.readerQueueSize)
if r.readerConsumer, err = r.pulsarClient.Subscribe(pulsar.ConsumerOptions{
Topics: readTopics,
SubscriptionName: readSubName,
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
MessageChannel: readerChan,
}); err != nil {
return nil, fmt.Errorf("failed to subscrive reader topics : %v, error = %v", readTopics, err)
}
r.readerProducer = make([]pulsar.Producer, 0, len(readTopics))
for i := 0; i < len(readTopics); i++ {
rp, err := r.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readTopics[i]})
if err != nil {
return nil, fmt.Errorf("failed to create reader producer %s, error = %v", readTopics[i], err)
}
r.readerProducer = append(r.readerProducer, rp)
}
return r, nil
}
func (r *TimeSyncCfg) Close() {
if r.timeSyncConsumer != nil {
r.timeSyncConsumer.Close()
}
if r.readerConsumer != nil {
r.readerConsumer.Close()
}
for i := 0; i < len(r.readerProducer); i++ {
if r.readerProducer[i] != nil {
r.readerProducer[i].Close()
}
}
if r.pulsarClient != nil {
r.pulsarClient.Close()
}
}
func (r *TimeSyncCfg) Start() error {
go r.startReadTopics()
go r.startTimeSync()
return r.ctx.Err()
}
func (r *TimeSyncCfg) InsertOrDelete() <-chan *pb.InsertOrDeleteMsg {
return r.insertOrDeleteChan
}
func (r *TimeSyncCfg) TimeSync() <-chan TimeSyncMsg {
return r.timesyncMsgChan
}
func (r *TimeSyncCfg) TimeSyncChanLen() int {
return len(r.timesyncMsgChan)
}
func (r *TimeSyncCfg) IsInsertDeleteChanFull() bool {
return len(r.insertOrDeleteChan) == len(r.readerProducer)*r.readerQueueSize
}
func (r *TimeSyncCfg) alignTimeSync(ts []*internalpb.TimeTickMsg) []*internalpb.TimeTickMsg {
if len(r.proxyIdList) > 1 {
if len(ts) > 1 {
for i := 1; i < len(r.proxyIdList); i++ {
curIdx := len(ts) - 1 - i
preIdx := len(ts) - i
timeGap := toMillisecond(ts[curIdx]) - toMillisecond(ts[preIdx])
if int64(timeGap) >= (r.interval/2) || int64(timeGap) <= (-r.interval/2) {
ts = ts[preIdx:]
return ts
}
}
ts = ts[len(ts)-len(r.proxyIdList):]
sort.Slice(ts, func(i int, j int) bool { return ts[i].PeerId < ts[j].PeerId })
for i := 0; i < len(r.proxyIdList); i++ {
if ts[i].PeerId != r.proxyIdList[i] {
ts = ts[:0]
return ts
}
}
}
} else {
if len(ts) > 1 {
ts = ts[len(ts)-1:]
}
}
return ts
}
func (r *TimeSyncCfg) readTimeSync(ctx context.Context, ts []*internalpb.TimeTickMsg, n int) ([]*internalpb.TimeTickMsg, error) {
for i := 0; i < n; i++ {
select {
case <-ctx.Done():
return nil, ctx.Err()
case cm, ok := <-r.timeSyncConsumer.Chan():
if ok == false {
return nil, fmt.Errorf("timesync consumer closed")
}
msg := cm.Message
var tsm internalpb.TimeTickMsg
if err := proto.Unmarshal(msg.Payload(), &tsm); err != nil {
return nil, err
}
ts = append(ts, &tsm)
r.timeSyncConsumer.AckID(msg.ID())
}
}
return ts, nil
}
func (r *TimeSyncCfg) sendEOFMsg(ctx context.Context, msg *pulsar.ProducerMessage, index int, wg *sync.WaitGroup) {
if _, err := r.readerProducer[index].Send(ctx, msg); err != nil {
//TODO, log error
log.Printf("Send timesync flag error %v", err)
}
wg.Done()
}
func (r *TimeSyncCfg) startTimeSync() {
ctx := r.ctx
tsm := make([]*internalpb.TimeTickMsg, 0, len(r.proxyIdList)*2)
var err error
for {
select {
case <-ctx.Done():
return
default:
//var start time.Time
for len(tsm) != len(r.proxyIdList) {
tsm = r.alignTimeSync(tsm)
tsm, err = r.readTimeSync(ctx, tsm, len(r.proxyIdList)-len(tsm))
if err != nil {
if ctx.Err() != nil {
return
} else {
//TODO, log error msg
log.Printf("read time sync error %v", err)
}
}
}
ts := tsm[0].Timestamp
for i := 1; i < len(tsm); i++ {
if tsm[i].Timestamp < ts {
ts = tsm[i].Timestamp
}
}
tsm = tsm[:0]
//send timestamp flag to reader channel
msg := pb.InsertOrDeleteMsg{Timestamp: ts, ClientId: r.readStopFlagClientId}
payload, err := proto.Marshal(&msg)
if err != nil {
//TODO log error
log.Printf("Marshal timesync flag error %v", err)
} else {
wg := sync.WaitGroup{}
wg.Add(len(r.readerProducer))
for index := range r.readerProducer {
go r.sendEOFMsg(ctx, &pulsar.ProducerMessage{Payload: payload}, index, &wg)
}
wg.Wait()
}
}
}
}
func (r *TimeSyncCfg) isReadStopFlag(imsg *pb.InsertOrDeleteMsg) bool {
return imsg.ClientId < ReadStopFlagEnd
}
func (r *TimeSyncCfg) WriteInsertLog() {
fileName := "/tmp/reader_get_pulsar.txt"
if r.RoleType == Writer {
fileName = "/tmp/writer_get_pulsar.txt"
}
f, err := os.OpenFile(fileName, os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0644)
if err != nil {
log.Fatal(err)
}
// write logs
for _, insertLog := range r.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
r.InsertLogs = make([]InsertLog, 0)
err = f.Close()
if err != nil {
log.Fatal(err)
}
fmt.Println("write get pulsar log done")
}
func (r *TimeSyncCfg) startReadTopics() {
ctx := r.ctx
tsm := TimeSyncMsg{Timestamp: 0, NumRecorders: 0}
const Debug = true
const WriterBaseline = 1000 * 1000
const LogBaseline = 100000
var Counter int64 = 0
var LastCounter int64 = 0
r.InsertLogs = make([]InsertLog, 0)
InsertTime := time.Now()
var BaselineCounter int64 = 0
for {
select {
case <-ctx.Done():
return
case cm, ok := <-r.readerConsumer.Chan():
if ok == false {
//TODO,log error
log.Printf("reader consumer closed")
}
msg := cm.Message
var imsg pb.InsertOrDeleteMsg
if err := proto.Unmarshal(msg.Payload(), &imsg); err != nil {
//TODO, log error
log.Printf("unmarshal InsertOrDeleteMsg error %v", err)
break
}
if r.isReadStopFlag(&imsg) { //timestamp flag
if imsg.ClientId == r.readStopFlagClientId {
gval := r.revTimesyncFromReader[imsg.Timestamp]
gval++
if gval >= len(r.readerProducer) {
if imsg.Timestamp >= tsm.Timestamp {
tsm.Timestamp = imsg.Timestamp
r.timesyncMsgChan <- tsm
tsm.NumRecorders = 0
}
delete(r.revTimesyncFromReader, imsg.Timestamp)
} else {
r.revTimesyncFromReader[imsg.Timestamp] = gval
}
}
} else {
if r.IsInsertDeleteChanFull() {
log.Printf("WARN : Insert or delete chan is full ...")
}
tsm.NumRecorders++
if Debug {
r.insertOrDeleteChan <- &imsg
Counter++
if Counter%LogBaseline == 0 {
timeNow := time.Now()
duration := timeNow.Sub(InsertTime)
speed := float64(Counter-LastCounter) / duration.Seconds()
insertLog := InsertLog{
MsgLength: int(Counter - LastCounter),
DurationInMilliseconds: duration.Milliseconds(),
InsertTime: timeNow,
NumSince: Counter,
Speed: speed,
}
r.InsertLogs = append(r.InsertLogs, insertLog)
LastCounter = Counter
InsertTime = timeNow
}
if Counter/WriterBaseline != BaselineCounter {
r.WriteInsertLog()
BaselineCounter = Counter / WriterBaseline
}
} else {
r.insertOrDeleteChan <- &imsg
}
}
r.readerConsumer.AckID(msg.ID())
}
}
}
func WithReaderQueueSize(size int) TimeSyncOption {
return func(r *TimeSyncCfg) {
r.readerQueueSize = size
}
}

View File

@ -1,628 +0,0 @@
package timesync
import (
"context"
"log"
"sync"
"testing"
"time"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
const (
pulsarAddr = "pulsar://localhost:6650"
timeSyncTopic = "timesync"
timeSyncTopic2 = "timesync2"
timeSyncTopic3 = "timesync3"
timeSyncSubName = "timesync-g"
timeSyncSubName1 = "timesync-g1"
timeSyncSubName2 = "timesync-g2"
timeSyncSubName3 = "timesync-g3"
readerTopic1 = "reader1"
readerTopic12 = "reader12"
readerTopic13 = "reader13"
readerTopic2 = "reader2"
readerTopic22 = "reader22"
readerTopic23 = "reader23"
readerTopic3 = "reader3"
readerTopic32 = "reader32"
readerTopic33 = "reader33"
readerTopic4 = "reader4"
readerTopic42 = "reader42"
readerTopic43 = "reader43"
readerSubName = "reader-g"
readerSubName1 = "reader-g1"
readerSubName2 = "reader-g2"
readerSubName3 = "reader-g3"
interval = 200
readStopFlag int64 = -1
readStopFlag1 int64 = -1
readStopFlag2 int64 = -2
readStopFlag3 int64 = -3
)
func TestAlignTimeSync(t *testing.T) {
r := &TimeSyncCfg{
proxyIdList: []UniqueID{1, 2, 3},
interval: 200,
}
ts := []*internalpb.TimeTickMsg{
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 3,
Timestamp: toTimestamp(15),
},
{
PeerId: 2,
Timestamp: toTimestamp(20),
},
}
r.alignTimeSync(ts)
if len(r.proxyIdList) != 3 {
t.Fatalf("proxyIdList should be : 1 2 3")
}
for i := 0; i < len(r.proxyIdList); i++ {
if r.proxyIdList[i] != ts[i].PeerId {
t.Fatalf("Align falied")
}
}
}
func TestAlignTimeSync2(t *testing.T) {
r := &TimeSyncCfg{
proxyIdList: []UniqueID{1, 2, 3},
interval: 200,
}
ts := []*internalpb.TimeTickMsg{
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 3,
Timestamp: toTimestamp(150),
},
{
PeerId: 2,
Timestamp: toTimestamp(20),
},
}
ts = r.alignTimeSync(ts)
if len(r.proxyIdList) != 3 {
t.Fatalf("proxyIdList should be : 1 2 3")
}
if len(ts) != 1 || ts[0].PeerId != 2 {
t.Fatalf("align failed")
}
}
func TestAlignTimeSync3(t *testing.T) {
r := &TimeSyncCfg{
proxyIdList: []UniqueID{1, 2, 3},
interval: 200,
}
ts := []*internalpb.TimeTickMsg{
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 3,
Timestamp: toTimestamp(15),
},
{
PeerId: 2,
Timestamp: toTimestamp(20),
},
}
ts = r.alignTimeSync(ts)
if len(r.proxyIdList) != 3 {
t.Fatalf("proxyIdList should be : 1 2 3")
}
for i := 0; i < len(r.proxyIdList); i++ {
if r.proxyIdList[i] != ts[i].PeerId {
t.Fatalf("Align falied")
}
}
}
func TestAlignTimeSync4(t *testing.T) {
r := &TimeSyncCfg{
proxyIdList: []UniqueID{1},
interval: 200,
}
ts := []*internalpb.TimeTickMsg{
{
PeerId: 1,
Timestamp: toTimestamp(15),
},
{
PeerId: 1,
Timestamp: toTimestamp(25),
},
{
PeerId: 1,
Timestamp: toTimestamp(35),
},
}
ts = r.alignTimeSync(ts)
if len(r.proxyIdList) != 1 {
t.Fatalf("proxyIdList should be : 1")
}
if len(ts) != 1 {
t.Fatalf("aligned failed")
}
if getMillisecond(ts[0].Timestamp) != 35 {
t.Fatalf("aligned failed")
}
}
func TestAlignTimeSync5(t *testing.T) {
r := &TimeSyncCfg{
proxyIdList: []UniqueID{1, 2, 3},
interval: 200,
}
ts := []*internalpb.TimeTickMsg{
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 1,
Timestamp: toTimestamp(5),
},
{
PeerId: 3,
Timestamp: toTimestamp(15),
},
{
PeerId: 3,
Timestamp: toTimestamp(20),
},
}
ts = r.alignTimeSync(ts)
if len(ts) != 0 {
t.Fatalf("aligned failed")
}
}
func TestNewTimeSync(t *testing.T) {
r, err := NewTimeSync(context.Background(),
timeSyncTopic,
timeSyncSubName,
[]string{readerTopic1, readerTopic2, readerTopic3, readerTopic4},
readerSubName,
[]UniqueID{2, 1},
interval,
WithReaderQueueSize(8),
)
if err != nil {
t.Fatal(err)
}
rr := r.(*TimeSyncCfg)
if rr.pulsarClient == nil {
t.Fatalf("create pulsar client failed")
}
if rr.timeSyncConsumer == nil {
t.Fatalf("create time sync consumer failed")
}
if rr.readerConsumer == nil {
t.Fatalf("create reader consumer failed")
}
if len(rr.readerProducer) != 4 {
t.Fatalf("create reader producer failed")
}
if rr.interval != interval {
t.Fatalf("interval shoudl be %d", interval)
}
if rr.readStopFlagClientId != readStopFlag {
t.Fatalf("raed stop flag client id should be %d", rr.readStopFlagClientId)
}
if rr.readerQueueSize != 8 {
t.Fatalf("set read queue size failed")
}
if len(rr.proxyIdList) != 2 {
t.Fatalf("set proxy id failed")
}
if rr.proxyIdList[0] != 1 || rr.proxyIdList[1] != 2 {
t.Fatalf("set proxy id failed")
}
r.Close()
}
func TestPulsarClient(t *testing.T) {
t.Skip("skip pulsar client")
client, err := pulsar.NewClient(pulsar.ClientOptions{URL: pulsarAddr})
if err != nil {
t.Fatal(err)
}
ctx, _ := context.WithTimeout(context.Background(), 3*time.Second)
go startWriteTimeSync(1, timeSyncTopic, client, 2*time.Second, t)
go startWriteTimeSync(2, timeSyncTopic, client, 2*time.Second, t)
timeSyncChan := make(chan pulsar.ConsumerMessage)
consumer, err := client.Subscribe(pulsar.ConsumerOptions{
Topic: timeSyncTopic,
SubscriptionName: timeSyncSubName,
Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
MessageChannel: timeSyncChan,
})
if err != nil {
log.Fatal(err)
}
for {
select {
case cm := <-timeSyncChan:
msg := cm.Message
var tsm pb.TimeSyncMsg
if err := proto.Unmarshal(msg.Payload(), &tsm); err != nil {
log.Fatal(err)
}
consumer.AckID(msg.ID())
log.Printf("read time stamp, id = %d, time stamp = %d\n", tsm.Peer_Id, tsm.Timestamp)
case <-ctx.Done():
break
}
if ctx.Err() != nil {
break
}
}
}
func TestTimeSync(t *testing.T) {
r, err := NewTimeSync(context.Background(),
timeSyncTopic,
timeSyncSubName,
[]string{readerTopic1, readerTopic2, readerTopic3, readerTopic4},
readerSubName,
[]UniqueID{2, 1},
interval,
WithReaderQueueSize(1024),
)
if err != nil {
t.Fatal(err)
}
rr := r.(*TimeSyncCfg)
pt1, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic})
if err != nil {
t.Fatalf("create time sync producer 1 error %v", err)
}
pt2, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic})
if err != nil {
t.Fatalf("create time sync producer 2 error %v", err)
}
pr1, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic1})
if err != nil {
t.Fatalf("create reader 1 error %v", err)
}
pr2, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic2})
if err != nil {
t.Fatalf("create reader 2 error %v", err)
}
pr3, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic3})
if err != nil {
t.Fatalf("create reader 3 error %v", err)
}
pr4, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic4})
if err != nil {
t.Fatalf("create reader 4 error %v", err)
}
go startProxy(pt1, 1, pr1, 1, pr2, 2, 2*time.Second, t)
go startProxy(pt2, 2, pr3, 3, pr4, 4, 2*time.Second, t)
ctx, _ := context.WithTimeout(context.Background(), 3*time.Second)
r.Start()
var tsm1, tsm2 TimeSyncMsg
var totalRecordes int64 = 0
for {
if ctx.Err() != nil {
break
}
select {
case <-ctx.Done():
tsm1.NumRecorders = 0
break
case tsm1 = <-r.TimeSync():
}
if tsm1.NumRecorders > 0 {
log.Printf("timestamp %d, num records = %d", getMillisecond(tsm1.Timestamp), tsm1.NumRecorders)
totalRecordes += tsm1.NumRecorders
for i := int64(0); i < tsm1.NumRecorders; i++ {
im := <-r.InsertOrDelete()
//log.Printf("%d - %d", getMillisecond(im.Timestamp), getMillisecond(tsm2.Timestamp))
if im.Timestamp < tsm2.Timestamp {
t.Fatalf("time sync error , im.Timestamp = %d, tsm2.Timestamp = %d", im.Timestamp, tsm2.Timestamp)
}
}
tsm2 = tsm1
}
}
log.Printf("total recordes = %d", totalRecordes)
if totalRecordes != 800 {
t.Fatalf("total records should be 800")
}
r.Close()
pt1.Close()
pt2.Close()
pr1.Close()
pr2.Close()
pr3.Close()
pr4.Close()
}
func TestTimeSync2(t *testing.T) {
client, _ := pulsar.NewClient(pulsar.ClientOptions{URL: pulsarAddr})
pt1, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic2})
pt2, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic2})
pr1, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic12})
pr2, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic22})
pr3, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic32})
pr4, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic42})
go startProxy(pt1, 1, pr1, 1, pr2, 2, 2*time.Second, t)
go startProxy(pt2, 2, pr3, 3, pr4, 4, 2*time.Second, t)
r1, _ := NewTimeSync(context.Background(),
timeSyncTopic2,
timeSyncSubName1,
[]string{readerTopic12, readerTopic22, readerTopic32, readerTopic42},
readerSubName1,
[]int64{2, 1},
interval,
WithReaderQueueSize(1024),
)
r2, _ := NewTimeSync(context.Background(),
timeSyncTopic2,
timeSyncSubName2,
[]string{readerTopic12, readerTopic22, readerTopic32, readerTopic42},
readerSubName2,
[]int64{2, 1},
interval,
WithReaderQueueSize(1024),
)
ctx, _ := context.WithTimeout(context.Background(), 3*time.Second)
rt := []TimeSync{r1, r2}
var wg sync.WaitGroup
for _, r := range rt {
r := r
_ = r.Start()
wg.Add(1)
go func() {
var tsm1, tsm2 TimeSyncMsg
var totalRecordes int64 = 0
work := false
defer wg.Done()
for {
if ctx.Err() != nil {
break
}
select {
case tsm1 = <-r.TimeSync():
work = true
default:
work = false
}
if work {
if tsm1.NumRecorders > 0 {
//log.Printf("timestamp %d, num records = %d", getMillisecond(tsm1.Timestamp), tsm1.NumRecorders)
totalRecordes += tsm1.NumRecorders
for i := int64(0); i < tsm1.NumRecorders; i++ {
im := <-r.InsertOrDelete()
//log.Printf("%d - %d", getMillisecond(im.Timestamp), getMillisecond(tsm2.Timestamp))
if im.Timestamp < tsm2.Timestamp {
t.Fatalf("time sync error , im.Timestamp = %d, tsm2.Timestamp = %d", getMillisecond(im.Timestamp), getMillisecond(tsm2.Timestamp))
}
}
tsm2 = tsm1
}
}
}
log.Printf("total recordes = %d", totalRecordes)
if totalRecordes != 800 {
t.Fatalf("total records should be 800")
}
}()
}
wg.Wait()
r1.Close()
r2.Close()
pt1.Close()
pt2.Close()
pr1.Close()
pr2.Close()
pr3.Close()
pr4.Close()
}
func TestTimeSync3(t *testing.T) {
client, _ := pulsar.NewClient(pulsar.ClientOptions{URL: pulsarAddr})
pt, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic3})
pr1, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic13})
pr2, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic23})
pr3, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic33})
pr4, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic43})
defer func() {
pr1.Close()
pr2.Close()
pr3.Close()
pr4.Close()
pt.Close()
client.Close()
}()
go func() {
total := 2 * 1000 / 10
ticker := time.Tick(10 * time.Millisecond)
var timestamp uint64 = 0
prlist := []pulsar.Producer{pr1, pr2, pr3, pr4}
for i := 1; i <= total; i++ {
<-ticker
timestamp += 10
for idx, pr := range prlist {
msg := pb.InsertOrDeleteMsg{ClientId: int64(idx + 1), Timestamp: toTimestamp(timestamp)}
mb, err := proto.Marshal(&msg)
if err != nil {
t.Fatal(err)
}
if _, err := pr.Send(context.Background(), &pulsar.ProducerMessage{Payload: mb}); err != nil {
t.Fatal(err)
}
}
if i%20 == 0 {
tm := pb.TimeSyncMsg{Peer_Id: 1, Timestamp: toTimestamp(timestamp)}
tb, err := proto.Marshal(&tm)
if err != nil {
t.Fatal(err)
}
if _, err := pt.Send(context.Background(), &pulsar.ProducerMessage{Payload: tb}); err != nil {
t.Fatal(err)
}
}
}
}()
r, err := NewTimeSync(context.Background(),
timeSyncTopic3,
timeSyncSubName3,
[]string{readerTopic13, readerTopic23, readerTopic33, readerTopic43},
readerSubName3,
[]int64{1},
interval,
WithReaderQueueSize(1024))
if err != nil {
t.Fatal(err)
}
defer r.Close()
ctx, _ := context.WithTimeout(context.Background(), 3*time.Second)
if err := r.Start(); err != nil {
t.Fatal(err)
}
var tsm1, tsm2 TimeSyncMsg
var totalRecords int64 = 0
for {
if ctx.Err() != nil {
break
}
select {
case <-ctx.Done():
tsm1.NumRecorders = 0
break
case tsm1 = <-r.TimeSync():
}
if tsm1.NumRecorders > 0 {
totalRecords += tsm1.NumRecorders
for i := int64(0); i < tsm1.NumRecorders; i++ {
im := <-r.InsertOrDelete()
if im.Timestamp < tsm2.Timestamp {
t.Fatalf("time sync error , im.Timestamp = %d, tsm2.Timestamp = %d", im.Timestamp, tsm2.Timestamp)
}
}
tsm2 = tsm1
}
}
log.Printf("total records = %d", totalRecords)
if totalRecords != 800 {
t.Fatalf("total records should be 800")
}
}
func getMillisecond(ts uint64) uint64 {
return ts >> 18
}
func toTimestamp(ts uint64) uint64 {
return ts << 18
}
func startWriteTimeSync(id int64, topic string, client pulsar.Client, duration time.Duration, t *testing.T) {
p, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: topic})
ticker := time.Tick(interval * time.Millisecond)
numSteps := int(duration / (interval * time.Millisecond))
var tm uint64 = 0
for i := 0; i < numSteps; i++ {
<-ticker
tm += interval
tsm := pb.TimeSyncMsg{Timestamp: toTimestamp(tm), Peer_Id: id}
tb, _ := proto.Marshal(&tsm)
if _, err := p.Send(context.Background(), &pulsar.ProducerMessage{Payload: tb}); err != nil {
t.Fatalf("send failed tsm id=%d, timestamp=%d, err=%v", tsm.Peer_Id, tsm.Timestamp, err)
} else {
//log.Printf("send tsm id=%d, timestamp=%d", tsm.Peer_Id, tsm.Timestamp)
}
}
}
func startProxy(pt pulsar.Producer, ptid int64, pr1 pulsar.Producer, prid1 int64, pr2 pulsar.Producer, prid2 int64, duration time.Duration, t *testing.T) {
total := int(duration / (10 * time.Millisecond))
ticker := time.Tick(10 * time.Millisecond)
var timestamp Timestamp = 0
for i := 1; i <= total; i++ {
<-ticker
timestamp += 10
msg := pb.InsertOrDeleteMsg{ClientId: prid1, Timestamp: toTimestamp(timestamp)}
mb, err := proto.Marshal(&msg)
if err != nil {
t.Fatalf("marshal error %v", err)
}
if _, err := pr1.Send(context.Background(), &pulsar.ProducerMessage{Payload: mb}); err != nil {
t.Fatalf("send msg error %v", err)
}
msg.ClientId = prid2
mb, err = proto.Marshal(&msg)
if err != nil {
t.Fatalf("marshal error %v", err)
}
if _, err := pr2.Send(context.Background(), &pulsar.ProducerMessage{Payload: mb}); err != nil {
t.Fatalf("send msg error %v", err)
}
//log.Printf("send msg id = [ %d %d ], timestamp = %d", prid1, prid2, timestamp)
if i%20 == 0 {
tm := pb.TimeSyncMsg{Peer_Id: ptid, Timestamp: toTimestamp(timestamp)}
tb, err := proto.Marshal(&tm)
if err != nil {
t.Fatalf("marshal error %v", err)
}
if _, err := pt.Send(context.Background(), &pulsar.ProducerMessage{Payload: tb}); err != nil {
t.Fatalf("send msg error %v", err)
}
//log.Printf("send timestamp id = %d, timestamp = %d", ptid, timestamp)
}
}
}