mirror of https://github.com/milvus-io/milvus.git
Add consume in rmq_msgstream
Signed-off-by: yukun <kun.yu@zilliz.com>pull/4973/head^2
parent
73a8e7de84
commit
35f6f4b7c2
3
go.mod
3
go.mod
|
@ -18,6 +18,7 @@ require (
|
|||
github.com/fsnotify/fsnotify v1.4.9 // indirect
|
||||
github.com/go-basic/ipv4 v1.0.0
|
||||
github.com/go-kit/kit v0.9.0
|
||||
github.com/gogo/protobuf v1.2.1
|
||||
github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e // indirect
|
||||
github.com/golang/mock v1.3.1
|
||||
github.com/golang/protobuf v1.3.2
|
||||
|
@ -26,6 +27,7 @@ require (
|
|||
github.com/klauspost/compress v1.10.11 // indirect
|
||||
github.com/kr/text v0.2.0 // indirect
|
||||
github.com/minio/minio-go/v7 v7.0.5
|
||||
github.com/modern-go/reflect2 v1.0.1
|
||||
github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e // indirect
|
||||
github.com/oklog/run v1.1.0
|
||||
github.com/onsi/ginkgo v1.12.1 // indirect
|
||||
|
@ -57,6 +59,7 @@ require (
|
|||
golang.org/x/lint v0.0.0-20200302205851-738671d3881b // indirect
|
||||
golang.org/x/net v0.0.0-20200904194848-62affa334b73
|
||||
golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f // indirect
|
||||
golang.org/x/text v0.3.3
|
||||
golang.org/x/time v0.0.0-20191024005414-555d28b269f0 // indirect
|
||||
golang.org/x/tools v0.0.0-20200825202427-b303f430e36d // indirect
|
||||
golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect
|
||||
|
|
|
@ -2,103 +2,258 @@ package rmqms
|
|||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"log"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/masterservice"
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/util"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
rocksmq "github.com/zilliztech/milvus-distributed/internal/util/rocksmq"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
||||
type RmqMsgStream struct {
|
||||
isServing int64
|
||||
idAllocator *masterservice.GlobalIDAllocator
|
||||
ctx context.Context
|
||||
serverLoopWg sync.WaitGroup
|
||||
serverLoopCtx context.Context
|
||||
serverLoopCancel func()
|
||||
|
||||
rmq *rocksmq.RocksMQ
|
||||
repackFunc msgstream.RepackFunc
|
||||
consumers []rocksmq.Consumer
|
||||
producers []string
|
||||
|
||||
unmarshal *util.UnmarshalDispatcher
|
||||
receiveBuf chan *msgstream.MsgPack
|
||||
wait *sync.WaitGroup
|
||||
// tso ticker
|
||||
tsoTicker *time.Ticker
|
||||
streamCancel func()
|
||||
}
|
||||
|
||||
func NewRmqMsgStream() *RmqMsgStream {
|
||||
//idAllocator := master.NewGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase([]string{""}, "singleNode/rocksmq", "gid"))
|
||||
//if err := idAllocator.Initialize(); err != nil {
|
||||
// return nil
|
||||
//}
|
||||
//
|
||||
//return &RmqMsgStream{
|
||||
// idAllocator: idAllocator,
|
||||
//}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) startServerLoop(ctx context.Context) error {
|
||||
ms.serverLoopCtx, ms.serverLoopCancel = context.WithCancel(ctx)
|
||||
|
||||
ms.serverLoopWg.Add(1)
|
||||
go ms.tsLoop()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) stopServerLoop() {
|
||||
ms.serverLoopCancel()
|
||||
ms.serverLoopWg.Wait()
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) tsLoop() {
|
||||
defer ms.serverLoopWg.Done()
|
||||
|
||||
ms.tsoTicker = time.NewTicker(masterservice.UpdateTimestampStep)
|
||||
defer ms.tsoTicker.Stop()
|
||||
|
||||
ctx, cancel := context.WithCancel(ms.serverLoopCtx)
|
||||
defer cancel()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ms.tsoTicker.C:
|
||||
if err := ms.idAllocator.UpdateID(); err != nil {
|
||||
log.Println("failed to update id", err)
|
||||
return
|
||||
}
|
||||
case <-ctx.Done():
|
||||
// Server is closed and it should return nil.
|
||||
log.Println("tsLoop is closed")
|
||||
return
|
||||
}
|
||||
func NewRmqMsgStream(ctx context.Context, rmq *rocksmq.RocksMQ, receiveBufSize int64) *RmqMsgStream {
|
||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
||||
receiveBuf := make(chan *msgstream.MsgPack, receiveBufSize)
|
||||
stream := &RmqMsgStream{
|
||||
ctx: streamCtx,
|
||||
rmq: nil,
|
||||
receiveBuf: receiveBuf,
|
||||
streamCancel: streamCancel,
|
||||
}
|
||||
|
||||
return stream
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) Start() {
|
||||
if err := ms.startServerLoop(ms.ctx); err != nil {
|
||||
return
|
||||
ms.wait = &sync.WaitGroup{}
|
||||
if ms.consumers != nil {
|
||||
ms.wait.Add(1)
|
||||
go ms.bufMsgPackToChannel()
|
||||
}
|
||||
|
||||
atomic.StoreInt64(&ms.isServing, 1)
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) Close() {
|
||||
if !atomic.CompareAndSwapInt64(&ms.isServing, 1, 0) {
|
||||
// server is already closed
|
||||
return
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) CreateProducers(channels []string) error {
|
||||
for _, channel := range channels {
|
||||
// TODO(yhz): Here may allow to create an existing channel
|
||||
if err := ms.rmq.CreateChannel(channel); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
log.Print("closing server")
|
||||
|
||||
ms.stopServerLoop()
|
||||
func (ms *RmqMsgStream) CreateConsumers(channels []string, groupName string) error {
|
||||
for _, channelName := range channels {
|
||||
if err := ms.rmq.CreateConsumerGroup(groupName, channelName); err != nil {
|
||||
return err
|
||||
}
|
||||
msgNum := make(chan int)
|
||||
ms.consumers = append(ms.consumers, rocksmq.Consumer{GroupName: groupName, ChannelName: channelName, MsgNum: msgNum})
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) Produce(pack *msgstream.MsgPack) error {
|
||||
tsMsgs := pack.Msgs
|
||||
if len(tsMsgs) <= 0 {
|
||||
log.Printf("Warning: Receive empty msgPack")
|
||||
return nil
|
||||
}
|
||||
if len(ms.producers) <= 0 {
|
||||
return errors.New("nil producer in msg stream")
|
||||
}
|
||||
reBucketValues := make([][]int32, len(tsMsgs))
|
||||
for channelID, tsMsg := range tsMsgs {
|
||||
hashValues := tsMsg.HashKeys()
|
||||
bucketValues := make([]int32, len(hashValues))
|
||||
for index, hashValue := range hashValues {
|
||||
if tsMsg.Type() == commonpb.MsgType_kSearchResult {
|
||||
searchResult := tsMsg.(*msgstream.SearchResultMsg)
|
||||
channelID := searchResult.ResultChannelID
|
||||
channelIDInt, _ := strconv.ParseInt(channelID, 10, 64)
|
||||
if channelIDInt >= int64(len(ms.producers)) {
|
||||
return errors.New("Failed to produce pulsar msg to unKnow channel")
|
||||
}
|
||||
bucketValues[index] = int32(channelIDInt)
|
||||
continue
|
||||
}
|
||||
bucketValues[index] = int32(hashValue % uint32(len(ms.producers)))
|
||||
}
|
||||
reBucketValues[channelID] = bucketValues
|
||||
}
|
||||
|
||||
var result map[int32]*msgstream.MsgPack
|
||||
var err error
|
||||
if ms.repackFunc != nil {
|
||||
result, err = ms.repackFunc(tsMsgs, reBucketValues)
|
||||
} else {
|
||||
msgType := (tsMsgs[0]).Type()
|
||||
switch msgType {
|
||||
case commonpb.MsgType_kInsert:
|
||||
result, err = util.InsertRepackFunc(tsMsgs, reBucketValues)
|
||||
case commonpb.MsgType_kDelete:
|
||||
result, err = util.DeleteRepackFunc(tsMsgs, reBucketValues)
|
||||
default:
|
||||
result, err = util.DefaultRepackFunc(tsMsgs, reBucketValues)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for k, v := range result {
|
||||
for i := 0; i < len(v.Msgs); i++ {
|
||||
mb, err := v.Msgs[i].Marshal(v.Msgs[i])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
//
|
||||
//msg := &pulsar.ProducerMessage{Payload: mb}
|
||||
|
||||
//var child opentracing.Span
|
||||
if v.Msgs[i].Type() == commonpb.MsgType_kInsert ||
|
||||
v.Msgs[i].Type() == commonpb.MsgType_kSearch ||
|
||||
v.Msgs[i].Type() == commonpb.MsgType_kSearchResult {
|
||||
//tracer := opentracing.GlobalTracer()
|
||||
//ctx := v.Msgs[i].GetMsgContext()
|
||||
//if ctx == nil {
|
||||
// ctx = context.Background()
|
||||
//}
|
||||
//
|
||||
//if parent := opentracing.SpanFromContext(ctx); parent != nil {
|
||||
// child = tracer.StartSpan("start send pulsar msg",
|
||||
// opentracing.FollowsFrom(parent.Context()))
|
||||
//} else {
|
||||
// child = tracer.StartSpan("start send pulsar msg")
|
||||
//}
|
||||
//child.SetTag("hash keys", v.Msgs[i].HashKeys())
|
||||
//child.SetTag("start time", v.Msgs[i].BeginTs())
|
||||
//child.SetTag("end time", v.Msgs[i].EndTs())
|
||||
//child.SetTag("msg type", v.Msgs[i].Type())
|
||||
//msg.Properties = make(map[string]string)
|
||||
//err = tracer.Inject(child.Context(), opentracing.TextMap, &propertiesReaderWriter{msg.Properties})
|
||||
//if err != nil {
|
||||
// child.LogFields(oplog.Error(err))
|
||||
// child.Finish()
|
||||
// return err
|
||||
//}
|
||||
//child.LogFields(oplog.String("inject success", "inject success"))
|
||||
}
|
||||
|
||||
msg := make([]rocksmq.ProducerMessage, 0)
|
||||
msg = append(msg, *rocksmq.NewProducerMessage(mb))
|
||||
if err := ms.rmq.Produce(ms.producers[k], msg); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) Consume() *msgstream.MsgPack {
|
||||
return nil
|
||||
for {
|
||||
select {
|
||||
case cm, ok := <-ms.receiveBuf:
|
||||
if !ok {
|
||||
log.Println("buf chan closed")
|
||||
return nil
|
||||
}
|
||||
return cm
|
||||
case <-ms.ctx.Done():
|
||||
log.Printf("context closed")
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) bufMsgPackToChannel() {
|
||||
defer ms.wait.Done()
|
||||
|
||||
cases := make([]reflect.SelectCase, len(ms.consumers))
|
||||
for i := 0; i < len(ms.consumers); i++ {
|
||||
ch := ms.consumers[i].MsgNum
|
||||
cases[i] = reflect.SelectCase{Dir: reflect.SelectRecv, Chan: reflect.ValueOf(ch)}
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
log.Println("done")
|
||||
return
|
||||
default:
|
||||
tsMsgList := make([]msgstream.TsMsg, 0)
|
||||
|
||||
for {
|
||||
chosen, value, ok := reflect.Select(cases)
|
||||
if !ok {
|
||||
log.Printf("channel closed")
|
||||
return
|
||||
}
|
||||
|
||||
msgNum := value.Interface().(int)
|
||||
rmqMsg, err := ms.rmq.Consume(ms.consumers[chosen].GroupName, ms.consumers[chosen].ChannelName, msgNum)
|
||||
if err != nil {
|
||||
log.Printf("Failed to consume message in rocksmq, error = %v", err)
|
||||
continue
|
||||
}
|
||||
for j := 0; j < len(rmqMsg); j++ {
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(rmqMsg[j].Payload, &headerMsg)
|
||||
if err != nil {
|
||||
log.Printf("Failed to unmarshal message header, error = %v", err)
|
||||
continue
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(rmqMsg[j].Payload, headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Printf("Failed to unmarshal tsMsg, error = %v", err)
|
||||
continue
|
||||
}
|
||||
tsMsgList = append(tsMsgList, tsMsg)
|
||||
}
|
||||
noMoreMessage := true
|
||||
for k := 0; k < len(ms.consumers); k++ {
|
||||
if len(ms.consumers[k].MsgNum) > 0 {
|
||||
noMoreMessage = false
|
||||
}
|
||||
}
|
||||
|
||||
if noMoreMessage {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if len(tsMsgList) > 0 {
|
||||
msgPack := util.MsgPack{Msgs: tsMsgList}
|
||||
ms.receiveBuf <- &msgPack
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *RmqMsgStream) Chan() <-chan *msgstream.MsgPack {
|
||||
|
|
|
@ -1,13 +1,19 @@
|
|||
package rocksmq
|
||||
|
||||
var rmq *RocksMQ
|
||||
var Rmq *RocksMQ
|
||||
|
||||
type Consumer struct {
|
||||
GroupName string
|
||||
ChannelName string
|
||||
MsgNum chan int
|
||||
}
|
||||
|
||||
func InitRmq(rocksdbName string, idAllocator IDAllocator) error {
|
||||
var err error
|
||||
rmq, err = NewRocksMQ(rocksdbName, idAllocator)
|
||||
Rmq, err = NewRocksMQ(rocksdbName, idAllocator)
|
||||
return err
|
||||
}
|
||||
|
||||
func GetRmq() *RocksMQ {
|
||||
return rmq
|
||||
return Rmq
|
||||
}
|
||||
|
|
|
@ -55,7 +55,7 @@ type ProducerMessage struct {
|
|||
|
||||
type ConsumerMessage struct {
|
||||
msgID UniqueID
|
||||
payload []byte
|
||||
Payload []byte
|
||||
}
|
||||
|
||||
type Channel struct {
|
||||
|
@ -75,6 +75,8 @@ type RocksMQ struct {
|
|||
idAllocator IDAllocator
|
||||
produceMu sync.Mutex
|
||||
consumeMu sync.Mutex
|
||||
|
||||
notify map[string][]Consumer
|
||||
//ctx context.Context
|
||||
//serverLoopWg sync.WaitGroup
|
||||
//serverLoopCtx context.Context
|
||||
|
@ -105,9 +107,16 @@ func NewRocksMQ(name string, idAllocator IDAllocator) (*RocksMQ, error) {
|
|||
idAllocator: idAllocator,
|
||||
}
|
||||
rmq.channels = make(map[string]*Channel)
|
||||
rmq.notify = make(map[string][]Consumer)
|
||||
return rmq, nil
|
||||
}
|
||||
|
||||
func NewProducerMessage(data []byte) *ProducerMessage {
|
||||
return &ProducerMessage{
|
||||
payload: data,
|
||||
}
|
||||
}
|
||||
|
||||
func (rmq *RocksMQ) checkKeyExist(key string) bool {
|
||||
val, _ := rmq.kv.Load(key)
|
||||
return val != ""
|
||||
|
@ -228,7 +237,15 @@ func (rmq *RocksMQ) Produce(channelName string, messages []ProducerMessage) erro
|
|||
kvChannelEndID := channelName + "/end_id"
|
||||
kvValues[kvChannelEndID] = strconv.FormatInt(idEnd, 10)
|
||||
|
||||
return rmq.kv.MultiSave(kvValues)
|
||||
err = rmq.kv.MultiSave(kvValues)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, consumer := range rmq.notify[channelName] {
|
||||
consumer.MsgNum <- msgLen
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rmq *RocksMQ) Consume(groupName string, channelName string, n int) ([]ConsumerMessage, error) {
|
||||
|
@ -274,7 +291,7 @@ func (rmq *RocksMQ) Consume(groupName string, channelName string, n int) ([]Cons
|
|||
}
|
||||
msg := ConsumerMessage{
|
||||
msgID: msgID,
|
||||
payload: val.Data(),
|
||||
Payload: val.Data(),
|
||||
}
|
||||
consumerMessage = append(consumerMessage, msg)
|
||||
key.Free()
|
||||
|
|
|
@ -66,13 +66,13 @@ func TestRocksMQ(t *testing.T) {
|
|||
cMsgs, err := rmq.Consume(groupName, channelName, 1)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(cMsgs), 1)
|
||||
assert.Equal(t, string(cMsgs[0].payload), "a_message")
|
||||
assert.Equal(t, string(cMsgs[0].Payload), "a_message")
|
||||
|
||||
cMsgs, err = rmq.Consume(groupName, channelName, 2)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(cMsgs), 2)
|
||||
assert.Equal(t, string(cMsgs[0].payload), "b_message")
|
||||
assert.Equal(t, string(cMsgs[1].payload), "c_message")
|
||||
assert.Equal(t, string(cMsgs[0].Payload), "b_message")
|
||||
assert.Equal(t, string(cMsgs[1].Payload), "c_message")
|
||||
}
|
||||
|
||||
func TestRocksMQ_Loop(t *testing.T) {
|
||||
|
@ -127,15 +127,15 @@ func TestRocksMQ_Loop(t *testing.T) {
|
|||
cMsgs, err := rmq.Consume(groupName, channelName, loopNum)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(cMsgs), loopNum)
|
||||
assert.Equal(t, string(cMsgs[0].payload), "message_"+strconv.Itoa(0))
|
||||
assert.Equal(t, string(cMsgs[loopNum-1].payload), "message_"+strconv.Itoa(loopNum-1))
|
||||
assert.Equal(t, string(cMsgs[0].Payload), "message_"+strconv.Itoa(0))
|
||||
assert.Equal(t, string(cMsgs[loopNum-1].Payload), "message_"+strconv.Itoa(loopNum-1))
|
||||
|
||||
// Consume one message once
|
||||
for i := 0; i < loopNum; i++ {
|
||||
oneMsgs, err := rmq.Consume(groupName, channelName, 1)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(oneMsgs), 1)
|
||||
assert.Equal(t, string(oneMsgs[0].payload), "message_"+strconv.Itoa(i+loopNum))
|
||||
assert.Equal(t, string(oneMsgs[0].Payload), "message_"+strconv.Itoa(i+loopNum))
|
||||
}
|
||||
|
||||
cMsgs, err = rmq.Consume(groupName, channelName, 1)
|
||||
|
|
Loading…
Reference in New Issue