mirror of https://github.com/milvus-io/milvus.git
Refactor proxy framework and add CreateCollection Task
Signed-off-by: dragondriver <jiquan.long@zilliz.com>pull/4973/head^2
parent
e45df02874
commit
c732c267e1
|
@ -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)
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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}
|
||||
//}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
//}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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 := ×tampOracle{
|
||||
// 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
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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,
|
||||
},
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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()
|
||||
}
|
|
@ -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")
|
||||
}
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue