mirror of https://github.com/milvus-io/milvus.git
parent
8a4c6a039f
commit
432b53149b
|
@ -0,0 +1,18 @@
|
|||
package client
|
||||
|
||||
type Client interface {
|
||||
// Create a producer instance
|
||||
CreateProducer(options ProducerOptions) (Producer, error)
|
||||
|
||||
// Create a consumer instance and subscribe a topic
|
||||
Subscribe(options ConsumerOptions) (Consumer, error)
|
||||
|
||||
// Get the earliest MessageID
|
||||
EarliestMessageID() MessageID
|
||||
|
||||
// String to msg ID
|
||||
StringToMsgID(string) (MessageID, error)
|
||||
|
||||
// Close the client and free associated resources
|
||||
Close()
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
package client
|
||||
|
||||
type SubscriptionInitialPosition int
|
||||
|
||||
const (
|
||||
// Latest position which means the start consuming position will be the last message
|
||||
SubscriptionPositionLatest SubscriptionInitialPosition = iota
|
||||
|
||||
// Earliest position which means the start consuming position will be the first message
|
||||
SubscriptionPositionEarliest
|
||||
)
|
||||
|
||||
type SubscriptionType int
|
||||
|
||||
const (
|
||||
// Exclusive there can be only 1 consumer on the same topic with the same subscription name
|
||||
Exclusive SubscriptionType = iota
|
||||
|
||||
// Shared subscription mode, multiple consumer will be able to use the same subscription name
|
||||
// and the messages will be dispatched according to
|
||||
// a round-robin rotation between the connected consumers
|
||||
Shared
|
||||
|
||||
// Failover subscription mode, multiple consumer will be able to use the same subscription name
|
||||
// but only 1 consumer will receive the messages.
|
||||
// If that consumer disconnects, one of the other connected consumers will start receiving messages.
|
||||
Failover
|
||||
|
||||
// KeyShared subscription mode, multiple consumer will be able to use the same
|
||||
// subscription and all messages with the same key will be dispatched to only one consumer
|
||||
KeyShared
|
||||
)
|
||||
|
||||
type UniqueID = int64
|
||||
|
||||
type ConsumerOptions struct {
|
||||
// The topic that this consumer will subscribe on
|
||||
Topic string
|
||||
|
||||
// The subscription name for this consumer
|
||||
SubscriptionName string
|
||||
|
||||
// InitialPosition at which the cursor will be set when subscribe
|
||||
// Default is `Latest`
|
||||
SubscriptionInitialPosition
|
||||
|
||||
// Message for this consumer
|
||||
// When a message is received, it will be pushed to this channel for consumption
|
||||
MessageChannel chan ConsumerMessage
|
||||
|
||||
// Set receive channel size
|
||||
BufSize int64
|
||||
|
||||
// Select the subscription type to be used when subscribing to the topic.
|
||||
// Default is `Exclusive`
|
||||
Type SubscriptionType
|
||||
}
|
||||
|
||||
type Consumer interface {
|
||||
// returns the subscription for the consumer
|
||||
Subscription() string
|
||||
|
||||
// Message channel
|
||||
Chan() <-chan ConsumerMessage
|
||||
|
||||
// Seek to the uniqueID position
|
||||
Seek(MessageID) error //nolint:govet
|
||||
|
||||
// Make sure that msg is received. Only used in pulsar
|
||||
Ack(ConsumerMessage)
|
||||
|
||||
// Close consumer
|
||||
Close()
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
package client
|
||||
|
||||
type MessageID interface {
|
||||
// Serialize the message id into a sequence of bytes that can be stored somewhere else
|
||||
Serialize() []byte
|
||||
}
|
|
@ -0,0 +1,17 @@
|
|||
package client
|
||||
|
||||
type ConsumerMessage interface {
|
||||
// Topic get the topic from which this message originated from
|
||||
Topic() string
|
||||
|
||||
// Properties are application defined key/value pairs that will be attached to the message.
|
||||
// Return the properties attached to the message.
|
||||
Properties() map[string]string
|
||||
|
||||
// Payload get the payload of the message
|
||||
Payload() []byte
|
||||
|
||||
// ID get the unique message ID associated with this message.
|
||||
// The message id can be used to univocally refer to a message without having the keep the entire payload in memory.
|
||||
ID() MessageID
|
||||
}
|
|
@ -0,0 +1,22 @@
|
|||
package client
|
||||
|
||||
import "context"
|
||||
|
||||
type ProducerOptions struct {
|
||||
Topic string
|
||||
}
|
||||
|
||||
type ProducerMessage struct {
|
||||
Payload []byte
|
||||
Properties map[string]string
|
||||
}
|
||||
|
||||
type Producer interface {
|
||||
// return the topic which producer is publishing to
|
||||
//Topic() string
|
||||
|
||||
// publish a message
|
||||
Send(ctx context.Context, message *ProducerMessage) error
|
||||
|
||||
Close()
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
package pulsar
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type pulsarClient struct {
|
||||
client pulsar.Client
|
||||
}
|
||||
|
||||
func NewPulsarClient(opts pulsar.ClientOptions) (*pulsarClient, error) {
|
||||
c, err := pulsar.NewClient(opts)
|
||||
if err != nil {
|
||||
log.Error("Set pulsar client failed, error", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
cli := &pulsarClient{client: c}
|
||||
return cli, nil
|
||||
}
|
||||
|
||||
func (pc *pulsarClient) CreateProducer(options client.ProducerOptions) (client.Producer, error) {
|
||||
opts := pulsar.ProducerOptions{Topic: options.Topic}
|
||||
pp, err := pc.client.CreateProducer(opts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if pp == nil {
|
||||
return nil, errors.New("pulsar is not ready, producer is nil")
|
||||
}
|
||||
producer := &pulsarProducer{p: pp}
|
||||
return producer, nil
|
||||
}
|
||||
|
||||
func (pc *pulsarClient) Subscribe(options client.ConsumerOptions) (client.Consumer, error) {
|
||||
receiveChannel := make(chan pulsar.ConsumerMessage, options.BufSize)
|
||||
consumer, err := pc.client.Subscribe(pulsar.ConsumerOptions{
|
||||
Topic: options.Topic,
|
||||
SubscriptionName: options.SubscriptionName,
|
||||
Type: pulsar.SubscriptionType(options.Type),
|
||||
SubscriptionInitialPosition: pulsar.SubscriptionInitialPosition(options.SubscriptionInitialPosition),
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
msgChannel := make(chan client.ConsumerMessage, 1)
|
||||
pConsumer := &pulsarConsumer{c: consumer, msgChannel: msgChannel}
|
||||
|
||||
go func() {
|
||||
for { //nolint:gosimple
|
||||
select {
|
||||
case msg, ok := <-pConsumer.c.Chan():
|
||||
if !ok {
|
||||
close(msgChannel)
|
||||
return
|
||||
}
|
||||
msgChannel <- &pulsarMessage{msg: msg}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return pConsumer, nil
|
||||
}
|
||||
|
||||
func (pc *pulsarClient) EarliestMessageID() client.MessageID {
|
||||
msgID := pulsar.EarliestMessageID()
|
||||
return &pulsarID{messageID: msgID}
|
||||
}
|
||||
|
||||
func (pc *pulsarClient) StringToMsgID(id string) (client.MessageID, error) {
|
||||
pID, err := typeutil.StringToPulsarMsgID(id)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &pulsarID{messageID: pID}, nil
|
||||
}
|
||||
|
||||
func (pc *pulsarClient) Close() {
|
||||
pc.client.Close()
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
package pulsar
|
||||
|
||||
import (
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
)
|
||||
|
||||
type pulsarConsumer struct {
|
||||
c pulsar.Consumer
|
||||
msgChannel chan client.ConsumerMessage
|
||||
}
|
||||
|
||||
func (pc *pulsarConsumer) Subscription() string {
|
||||
return pc.c.Subscription()
|
||||
}
|
||||
|
||||
func (pc *pulsarConsumer) Chan() <-chan client.ConsumerMessage {
|
||||
return pc.msgChannel
|
||||
}
|
||||
|
||||
func (pc *pulsarConsumer) Seek(id client.MessageID) error {
|
||||
messageID, err := typeutil.StringToPulsarMsgID(string(id.Serialize()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return pc.c.Seek(messageID)
|
||||
}
|
||||
|
||||
func (pc *pulsarConsumer) Ack(message client.ConsumerMessage) {
|
||||
pm := message.(*pulsarMessage)
|
||||
pc.c.Ack(pm.msg)
|
||||
}
|
||||
|
||||
func (pc *pulsarConsumer) Close() {
|
||||
pc.c.Close()
|
||||
}
|
|
@ -0,0 +1,11 @@
|
|||
package pulsar
|
||||
|
||||
import "github.com/apache/pulsar-client-go/pulsar"
|
||||
|
||||
type pulsarID struct {
|
||||
messageID pulsar.MessageID
|
||||
}
|
||||
|
||||
func (pid *pulsarID) Serialize() []byte {
|
||||
return pid.messageID.Serialize()
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
package pulsar
|
||||
|
||||
import (
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
)
|
||||
|
||||
type pulsarMessage struct {
|
||||
msg pulsar.ConsumerMessage
|
||||
}
|
||||
|
||||
func (pm *pulsarMessage) Topic() string {
|
||||
return pm.msg.Topic()
|
||||
}
|
||||
|
||||
func (pm *pulsarMessage) Properties() map[string]string {
|
||||
return pm.msg.Properties()
|
||||
}
|
||||
|
||||
func (pm *pulsarMessage) Payload() []byte {
|
||||
return pm.msg.Payload()
|
||||
}
|
||||
|
||||
func (pm *pulsarMessage) ID() client.MessageID {
|
||||
id := pm.msg.ID()
|
||||
pid := &pulsarID{messageID: id}
|
||||
return pid
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
package pulsar
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
)
|
||||
|
||||
type pulsarProducer struct {
|
||||
p pulsar.Producer
|
||||
}
|
||||
|
||||
func (pp *pulsarProducer) Topic() string {
|
||||
return pp.p.Topic()
|
||||
}
|
||||
|
||||
func (pp *pulsarProducer) Send(ctx context.Context, message *client.ProducerMessage) error {
|
||||
ppm := &pulsar.ProducerMessage{Payload: message.Payload}
|
||||
_, err := pp.p.Send(ctx, ppm)
|
||||
return err
|
||||
}
|
||||
|
||||
func (pp *pulsarProducer) Close() {
|
||||
pp.p.Close()
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
package rocksmq
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type rmqClient struct {
|
||||
client rocksmq.Client
|
||||
}
|
||||
|
||||
func NewRmqClient(opts rocksmq.ClientOptions) (*rmqClient, error) {
|
||||
c, err := rocksmq.NewClient(opts)
|
||||
if err != nil {
|
||||
log.Error("Set rmq client failed, error", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
return &rmqClient{client: c}, nil
|
||||
}
|
||||
|
||||
func (rc *rmqClient) CreateProducer(options client.ProducerOptions) (client.Producer, error) {
|
||||
rmqOpts := rocksmq.ProducerOptions{Topic: options.Topic}
|
||||
pp, err := rc.client.CreateProducer(rmqOpts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
rp := rmqProducer{p: pp}
|
||||
return &rp, nil
|
||||
}
|
||||
|
||||
func (rc *rmqClient) Subscribe(options client.ConsumerOptions) (client.Consumer, error) {
|
||||
receiveChannel := make(chan rocksmq.ConsumerMessage, options.BufSize)
|
||||
|
||||
cli, err := rc.client.Subscribe(rocksmq.ConsumerOptions{
|
||||
Topic: options.Topic,
|
||||
SubscriptionName: options.SubscriptionName,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
msgChannel := make(chan client.ConsumerMessage, 1)
|
||||
rConsumer := &rmqConsumer{c: cli, msgChannel: msgChannel}
|
||||
|
||||
go func() {
|
||||
for { //nolint:gosimple
|
||||
select {
|
||||
case msg, ok := <-rConsumer.c.Chan():
|
||||
if !ok {
|
||||
close(msgChannel)
|
||||
return
|
||||
}
|
||||
msgChannel <- &rmqMessage{msg: msg}
|
||||
}
|
||||
}
|
||||
}()
|
||||
return rConsumer, nil
|
||||
}
|
||||
|
||||
func (rc *rmqClient) EarliestMessageID() client.MessageID {
|
||||
rID := rocksmq.EarliestMessageID()
|
||||
return &rmqID{messageID: rID}
|
||||
}
|
||||
|
||||
func (rc *rmqClient) StringToMsgID(id string) (client.MessageID, error) {
|
||||
rID, err := strconv.ParseInt(id, 10, 64)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &rmqID{messageID: rID}, nil
|
||||
}
|
||||
|
||||
func (rc *rmqClient) Close() {
|
||||
rc.client.Close()
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
package rocksmq
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
)
|
||||
|
||||
type rmqConsumer struct {
|
||||
c rocksmq.Consumer
|
||||
msgChannel chan client.ConsumerMessage
|
||||
}
|
||||
|
||||
func (rc *rmqConsumer) Subscription() string {
|
||||
return rc.c.Subscription()
|
||||
}
|
||||
|
||||
func (rc *rmqConsumer) Chan() <-chan client.ConsumerMessage {
|
||||
return rc.msgChannel
|
||||
}
|
||||
|
||||
func (rc *rmqConsumer) Seek(id client.MessageID) error {
|
||||
msgID, err := strconv.ParseInt(string(id.Serialize()), 10, 64)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return rc.c.Seek(msgID)
|
||||
}
|
||||
|
||||
func (rc *rmqConsumer) Ack(message client.ConsumerMessage) {
|
||||
}
|
||||
|
||||
func (rc *rmqConsumer) Close() {
|
||||
}
|
|
@ -0,0 +1,15 @@
|
|||
package rocksmq
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/server/rocksmq"
|
||||
)
|
||||
|
||||
type rmqID struct {
|
||||
messageID rocksmq.UniqueID
|
||||
}
|
||||
|
||||
func (rid *rmqID) Serialize() []byte {
|
||||
return []byte(strconv.Itoa((int(rid.messageID))))
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
package rocksmq
|
||||
|
||||
import (
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
)
|
||||
|
||||
type rmqMessage struct {
|
||||
msg rocksmq.ConsumerMessage
|
||||
}
|
||||
|
||||
func (rm *rmqMessage) Topic() string {
|
||||
return rm.msg.Topic
|
||||
}
|
||||
|
||||
func (rm *rmqMessage) Properties() map[string]string {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rm *rmqMessage) Payload() []byte {
|
||||
return rm.msg.Payload
|
||||
}
|
||||
|
||||
func (rm *rmqMessage) ID() client.MessageID {
|
||||
return &rmqID{messageID: rm.msg.MsgID}
|
||||
}
|
|
@ -0,0 +1,24 @@
|
|||
package rocksmq
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
)
|
||||
|
||||
type rmqProducer struct {
|
||||
p rocksmq.Producer
|
||||
}
|
||||
|
||||
func (rp *rmqProducer) Topic() string {
|
||||
return rp.p.Topic()
|
||||
}
|
||||
|
||||
func (rp *rmqProducer) Send(ctx context.Context, message *client.ProducerMessage) error {
|
||||
pm := &rocksmq.ProducerMessage{Payload: message.Payload}
|
||||
return rp.p.Send(pm)
|
||||
}
|
||||
|
||||
func (rp *rmqProducer) Close() {
|
||||
}
|
|
@ -0,0 +1,787 @@
|
|||
package ms
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/util"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/trace"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type MessageID = client.MessageID
|
||||
type Client = client.Client
|
||||
type Producer = client.Producer
|
||||
type Consumer = client.Consumer
|
||||
type TsMsg = msgstream.TsMsg
|
||||
type MsgPack = msgstream.MsgPack
|
||||
type MsgType = msgstream.MsgType
|
||||
type UniqueID = msgstream.UniqueID
|
||||
type BaseMsg = msgstream.BaseMsg
|
||||
type Timestamp = msgstream.Timestamp
|
||||
type IntPrimaryKey = msgstream.IntPrimaryKey
|
||||
type TimeTickMsg = msgstream.TimeTickMsg
|
||||
type QueryNodeStatsMsg = msgstream.QueryNodeStatsMsg
|
||||
type RepackFunc = msgstream.RepackFunc
|
||||
type UnmarshalDispatcher = msgstream.UnmarshalDispatcher
|
||||
|
||||
type msgStream struct {
|
||||
ctx context.Context
|
||||
client Client
|
||||
producers map[string]Producer
|
||||
producerChannels []string
|
||||
consumers map[string]Consumer
|
||||
consumerChannels []string
|
||||
repackFunc RepackFunc
|
||||
unmarshal UnmarshalDispatcher
|
||||
receiveBuf chan *MsgPack
|
||||
wait *sync.WaitGroup
|
||||
streamCancel func()
|
||||
bufSize int64
|
||||
producerLock *sync.Mutex
|
||||
consumerLock *sync.Mutex
|
||||
|
||||
scMap *sync.Map
|
||||
}
|
||||
|
||||
func NewMsgStream(ctx context.Context,
|
||||
receiveBufSize int64,
|
||||
bufSize int64,
|
||||
client Client,
|
||||
unmarshal UnmarshalDispatcher) (*msgStream, error) {
|
||||
|
||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
||||
producers := make(map[string]Producer)
|
||||
consumers := make(map[string]Consumer)
|
||||
producerChannels := make([]string, 0)
|
||||
consumerChannels := make([]string, 0)
|
||||
receiveBuf := make(chan *MsgPack, receiveBufSize)
|
||||
|
||||
var err error
|
||||
if err != nil {
|
||||
defer streamCancel()
|
||||
log.Error("Set client failed, error", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
stream := &msgStream{
|
||||
ctx: streamCtx,
|
||||
client: client,
|
||||
producers: producers,
|
||||
producerChannels: producerChannels,
|
||||
consumers: consumers,
|
||||
consumerChannels: consumerChannels,
|
||||
unmarshal: unmarshal,
|
||||
bufSize: bufSize,
|
||||
receiveBuf: receiveBuf,
|
||||
streamCancel: streamCancel,
|
||||
producerLock: &sync.Mutex{},
|
||||
consumerLock: &sync.Mutex{},
|
||||
wait: &sync.WaitGroup{},
|
||||
scMap: &sync.Map{},
|
||||
}
|
||||
|
||||
return stream, nil
|
||||
}
|
||||
|
||||
func (ms *msgStream) AsProducer(channels []string) {
|
||||
for _, channel := range channels {
|
||||
fn := func() error {
|
||||
pp, err := ms.client.CreateProducer(client.ProducerOptions{Topic: channel})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pp == nil {
|
||||
return errors.New("Producer is nil")
|
||||
}
|
||||
|
||||
ms.producerLock.Lock()
|
||||
ms.producers[channel] = pp
|
||||
ms.producerChannels = append(ms.producerChannels, channel)
|
||||
ms.producerLock.Unlock()
|
||||
return nil
|
||||
}
|
||||
err := util.Retry(20, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
errMsg := "Failed to create producer " + channel + ", error = " + err.Error()
|
||||
panic(errMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) AsConsumer(channels []string,
|
||||
subName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := ms.consumers[channel]; ok {
|
||||
continue
|
||||
}
|
||||
fn := func() error {
|
||||
receiveChannel := make(chan client.ConsumerMessage, ms.bufSize)
|
||||
pc, err := ms.client.Subscribe(client.ConsumerOptions{
|
||||
Topic: channel,
|
||||
SubscriptionName: subName,
|
||||
Type: client.KeyShared,
|
||||
SubscriptionInitialPosition: client.SubscriptionPositionEarliest,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pc == nil {
|
||||
return errors.New("Consumer is nil")
|
||||
}
|
||||
|
||||
ms.consumers[channel] = pc
|
||||
ms.consumerChannels = append(ms.consumerChannels, channel)
|
||||
ms.wait.Add(1)
|
||||
go ms.receiveMsg(pc)
|
||||
return nil
|
||||
}
|
||||
err := util.Retry(20, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
errMsg := "Failed to create consumer " + channel + ", error = " + err.Error()
|
||||
panic(errMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) SetRepackFunc(repackFunc RepackFunc) {
|
||||
ms.repackFunc = repackFunc
|
||||
}
|
||||
|
||||
func (ms *msgStream) Start() {
|
||||
}
|
||||
|
||||
func (ms *msgStream) Close() {
|
||||
ms.streamCancel()
|
||||
ms.wait.Wait()
|
||||
|
||||
for _, producer := range ms.producers {
|
||||
if producer != nil {
|
||||
producer.Close()
|
||||
}
|
||||
}
|
||||
for _, consumer := range ms.consumers {
|
||||
if consumer != nil {
|
||||
consumer.Close()
|
||||
}
|
||||
}
|
||||
if ms.client != nil {
|
||||
ms.client.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) Produce(msgPack *MsgPack) error {
|
||||
tsMsgs := msgPack.Msgs
|
||||
if len(tsMsgs) <= 0 {
|
||||
log.Debug("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 idx, tsMsg := range tsMsgs {
|
||||
hashValues := tsMsg.HashKeys()
|
||||
bucketValues := make([]int32, len(hashValues))
|
||||
for index, hashValue := range hashValues {
|
||||
if tsMsg.Type() == commonpb.MsgType_SearchResult {
|
||||
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 msg to unKnow channel")
|
||||
}
|
||||
bucketValues[index] = int32(hashValue % uint32(len(ms.producers)))
|
||||
continue
|
||||
}
|
||||
bucketValues[index] = int32(hashValue % uint32(len(ms.producers)))
|
||||
}
|
||||
reBucketValues[idx] = bucketValues
|
||||
}
|
||||
|
||||
var result map[int32]*MsgPack
|
||||
var err error
|
||||
if ms.repackFunc != nil {
|
||||
result, err = ms.repackFunc(tsMsgs, reBucketValues)
|
||||
} else {
|
||||
msgType := (tsMsgs[0]).Type()
|
||||
switch msgType {
|
||||
case commonpb.MsgType_Insert:
|
||||
result, err = util.InsertRepackFunc(tsMsgs, reBucketValues)
|
||||
case commonpb.MsgType_Delete:
|
||||
result, err = util.DeleteRepackFunc(tsMsgs, reBucketValues)
|
||||
default:
|
||||
result, err = util.DefaultRepackFunc(tsMsgs, reBucketValues)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for k, v := range result {
|
||||
channel := ms.producerChannels[k]
|
||||
for i := 0; i < len(v.Msgs); i++ {
|
||||
mb, err := v.Msgs[i].Marshal(v.Msgs[i])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
m, err := msgstream.ConvertToByteArray(mb)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
msg := &client.ProducerMessage{Payload: m, Properties: map[string]string{}}
|
||||
|
||||
sp, spanCtx := trace.MsgSpanFromCtx(v.Msgs[i].TraceCtx(), v.Msgs[i])
|
||||
trace.InjectContextToPulsarMsgProperties(sp.Context(), msg.Properties)
|
||||
|
||||
if err := ms.producers[channel].Send(
|
||||
spanCtx,
|
||||
msg,
|
||||
); err != nil {
|
||||
trace.LogError(sp, err)
|
||||
sp.Finish()
|
||||
return err
|
||||
}
|
||||
sp.Finish()
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *msgStream) Broadcast(msgPack *MsgPack) error {
|
||||
for _, v := range msgPack.Msgs {
|
||||
mb, err := v.Marshal(v)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
m, err := msgstream.ConvertToByteArray(mb)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
msg := &client.ProducerMessage{Payload: m, Properties: map[string]string{}}
|
||||
|
||||
sp, spanCtx := trace.MsgSpanFromCtx(v.TraceCtx(), v)
|
||||
trace.InjectContextToPulsarMsgProperties(sp.Context(), msg.Properties)
|
||||
|
||||
ms.producerLock.Lock()
|
||||
for _, producer := range ms.producers {
|
||||
if err := producer.Send(
|
||||
spanCtx,
|
||||
msg,
|
||||
); err != nil {
|
||||
trace.LogError(sp, err)
|
||||
sp.Finish()
|
||||
return err
|
||||
}
|
||||
}
|
||||
ms.producerLock.Unlock()
|
||||
sp.Finish()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *msgStream) Consume() *MsgPack {
|
||||
for {
|
||||
select {
|
||||
case cm, ok := <-ms.receiveBuf:
|
||||
if !ok {
|
||||
log.Debug("buf chan closed")
|
||||
return nil
|
||||
}
|
||||
return cm
|
||||
case <-ms.ctx.Done():
|
||||
//log.Debug("context closed")
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) receiveMsg(consumer Consumer) {
|
||||
defer ms.wait.Done()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
return
|
||||
case msg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
consumer.Ack(msg)
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(msg.Payload(), &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(msg.Payload(), headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(msg.Topic()),
|
||||
//FIXME
|
||||
MsgID: typeutil.MsgIDToString(msg.ID()),
|
||||
})
|
||||
|
||||
sp, ok := trace.ExtractFromPulsarMsgProperties(tsMsg, msg.Properties())
|
||||
if ok {
|
||||
ms.scMap.Store(tsMsg.ID(), sp.Context())
|
||||
}
|
||||
|
||||
msgPack := MsgPack{Msgs: []TsMsg{tsMsg}}
|
||||
ms.receiveBuf <- &msgPack
|
||||
|
||||
sp.Finish()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) Chan() <-chan *MsgPack {
|
||||
return ms.receiveBuf
|
||||
}
|
||||
|
||||
func (ms *msgStream) Seek(mp *internalpb.MsgPosition) error {
|
||||
if _, ok := ms.consumers[mp.ChannelName]; ok {
|
||||
consumer := ms.consumers[mp.ChannelName]
|
||||
messageID, err := ms.client.StringToMsgID(mp.MsgID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = consumer.Seek(messageID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
return errors.New("msgStream seek fail")
|
||||
}
|
||||
|
||||
type TtMsgStream struct {
|
||||
msgStream
|
||||
unsolvedBuf map[Consumer][]TsMsg
|
||||
msgPositions map[Consumer]*internalpb.MsgPosition
|
||||
unsolvedMutex *sync.Mutex
|
||||
lastTimeStamp Timestamp
|
||||
syncConsumer chan int
|
||||
stopConsumeChan map[Consumer]chan bool
|
||||
}
|
||||
|
||||
func NewTtMsgStream(ctx context.Context,
|
||||
receiveBufSize int64,
|
||||
bufSize int64,
|
||||
client Client,
|
||||
unmarshal UnmarshalDispatcher) (*TtMsgStream, error) {
|
||||
msgStream, err := NewMsgStream(ctx, receiveBufSize, bufSize, client, unmarshal)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
unsolvedBuf := make(map[Consumer][]TsMsg)
|
||||
stopChannel := make(map[Consumer]chan bool)
|
||||
msgPositions := make(map[Consumer]*internalpb.MsgPosition)
|
||||
syncConsumer := make(chan int, 1)
|
||||
|
||||
return &TtMsgStream{
|
||||
msgStream: *msgStream,
|
||||
unsolvedBuf: unsolvedBuf,
|
||||
msgPositions: msgPositions,
|
||||
unsolvedMutex: &sync.Mutex{},
|
||||
syncConsumer: syncConsumer,
|
||||
stopConsumeChan: stopChannel,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) addConsumer(consumer Consumer, channel string) {
|
||||
if len(ms.consumers) == 0 {
|
||||
ms.syncConsumer <- 1
|
||||
}
|
||||
ms.consumers[channel] = consumer
|
||||
ms.unsolvedBuf[consumer] = make([]TsMsg, 0)
|
||||
ms.consumerChannels = append(ms.consumerChannels, channel)
|
||||
ms.msgPositions[consumer] = &internalpb.MsgPosition{
|
||||
ChannelName: channel,
|
||||
MsgID: "",
|
||||
Timestamp: ms.lastTimeStamp,
|
||||
}
|
||||
stopConsumeChan := make(chan bool)
|
||||
ms.stopConsumeChan[consumer] = stopConsumeChan
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) AsConsumer(channels []string,
|
||||
subName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := ms.consumers[channel]; ok {
|
||||
continue
|
||||
}
|
||||
fn := func() error {
|
||||
receiveChannel := make(chan client.ConsumerMessage, ms.bufSize)
|
||||
pc, err := ms.client.Subscribe(client.ConsumerOptions{
|
||||
Topic: channel,
|
||||
SubscriptionName: subName,
|
||||
Type: client.KeyShared,
|
||||
SubscriptionInitialPosition: client.SubscriptionPositionEarliest,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pc == nil {
|
||||
return errors.New("Consumer is nil")
|
||||
}
|
||||
|
||||
ms.consumerLock.Lock()
|
||||
ms.addConsumer(pc, channel)
|
||||
ms.consumerLock.Unlock()
|
||||
return nil
|
||||
}
|
||||
err := util.Retry(10, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
errMsg := "Failed to create consumer " + channel + ", error = " + err.Error()
|
||||
panic(errMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) Start() {
|
||||
if ms.consumers != nil {
|
||||
ms.wait.Add(1)
|
||||
go ms.bufMsgPackToChannel()
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) Close() {
|
||||
ms.streamCancel()
|
||||
close(ms.syncConsumer)
|
||||
ms.wait.Wait()
|
||||
|
||||
for _, producer := range ms.producers {
|
||||
if producer != nil {
|
||||
producer.Close()
|
||||
}
|
||||
}
|
||||
for _, consumer := range ms.consumers {
|
||||
if consumer != nil {
|
||||
consumer.Close()
|
||||
}
|
||||
}
|
||||
if ms.client != nil {
|
||||
ms.client.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) bufMsgPackToChannel() {
|
||||
defer ms.wait.Done()
|
||||
ms.unsolvedBuf = make(map[Consumer][]TsMsg)
|
||||
isChannelReady := make(map[Consumer]bool)
|
||||
eofMsgTimeStamp := make(map[Consumer]Timestamp)
|
||||
|
||||
if _, ok := <-ms.syncConsumer; !ok {
|
||||
log.Debug("consumer closed!")
|
||||
return
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
return
|
||||
default:
|
||||
wg := sync.WaitGroup{}
|
||||
findMapMutex := sync.RWMutex{}
|
||||
ms.consumerLock.Lock()
|
||||
for _, consumer := range ms.consumers {
|
||||
if isChannelReady[consumer] {
|
||||
continue
|
||||
}
|
||||
wg.Add(1)
|
||||
go ms.findTimeTick(consumer, eofMsgTimeStamp, &wg, &findMapMutex)
|
||||
}
|
||||
ms.consumerLock.Unlock()
|
||||
wg.Wait()
|
||||
timeStamp, ok := checkTimeTickMsg(eofMsgTimeStamp, isChannelReady, &findMapMutex)
|
||||
if !ok || timeStamp <= ms.lastTimeStamp {
|
||||
//log.Printf("All timeTick's timestamps are inconsistent")
|
||||
continue
|
||||
}
|
||||
timeTickBuf := make([]TsMsg, 0)
|
||||
startMsgPosition := make([]*internalpb.MsgPosition, 0)
|
||||
endMsgPositions := make([]*internalpb.MsgPosition, 0)
|
||||
ms.unsolvedMutex.Lock()
|
||||
for consumer, msgs := range ms.unsolvedBuf {
|
||||
if len(msgs) == 0 {
|
||||
continue
|
||||
}
|
||||
tempBuffer := make([]TsMsg, 0)
|
||||
var timeTickMsg TsMsg
|
||||
for _, v := range msgs {
|
||||
if v.Type() == commonpb.MsgType_TimeTick {
|
||||
timeTickMsg = v
|
||||
continue
|
||||
}
|
||||
if v.EndTs() <= timeStamp {
|
||||
timeTickBuf = append(timeTickBuf, v)
|
||||
} else {
|
||||
tempBuffer = append(tempBuffer, v)
|
||||
}
|
||||
}
|
||||
ms.unsolvedBuf[consumer] = tempBuffer
|
||||
|
||||
startMsgPosition = append(startMsgPosition, ms.msgPositions[consumer])
|
||||
var newPos *internalpb.MsgPosition
|
||||
if len(tempBuffer) > 0 {
|
||||
newPos = &internalpb.MsgPosition{
|
||||
ChannelName: tempBuffer[0].Position().ChannelName,
|
||||
MsgID: tempBuffer[0].Position().MsgID,
|
||||
Timestamp: timeStamp,
|
||||
}
|
||||
endMsgPositions = append(endMsgPositions, newPos)
|
||||
} else {
|
||||
newPos = &internalpb.MsgPosition{
|
||||
ChannelName: timeTickMsg.Position().ChannelName,
|
||||
MsgID: timeTickMsg.Position().MsgID,
|
||||
Timestamp: timeStamp,
|
||||
}
|
||||
endMsgPositions = append(endMsgPositions, newPos)
|
||||
}
|
||||
ms.msgPositions[consumer] = newPos
|
||||
}
|
||||
ms.unsolvedMutex.Unlock()
|
||||
|
||||
msgPack := MsgPack{
|
||||
BeginTs: ms.lastTimeStamp,
|
||||
EndTs: timeStamp,
|
||||
Msgs: timeTickBuf,
|
||||
StartPositions: startMsgPosition,
|
||||
EndPositions: endMsgPositions,
|
||||
}
|
||||
|
||||
ms.receiveBuf <- &msgPack
|
||||
ms.lastTimeStamp = timeStamp
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) findTimeTick(consumer Consumer,
|
||||
eofMsgMap map[Consumer]Timestamp,
|
||||
wg *sync.WaitGroup,
|
||||
findMapMutex *sync.RWMutex) {
|
||||
defer wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
return
|
||||
case msg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
log.Debug("consumer closed!")
|
||||
return
|
||||
}
|
||||
consumer.Ack(msg)
|
||||
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(msg.Payload(), &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(msg.Payload(), headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
// set msg info to tsMsg
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(msg.Topic()),
|
||||
MsgID: typeutil.MsgIDToString(msg.ID()),
|
||||
})
|
||||
|
||||
sp, ok := trace.ExtractFromPulsarMsgProperties(tsMsg, msg.Properties())
|
||||
if ok {
|
||||
ms.scMap.Store(tsMsg.ID(), sp.Context())
|
||||
}
|
||||
|
||||
ms.unsolvedMutex.Lock()
|
||||
ms.unsolvedBuf[consumer] = append(ms.unsolvedBuf[consumer], tsMsg)
|
||||
ms.unsolvedMutex.Unlock()
|
||||
|
||||
if headerMsg.Base.MsgType == commonpb.MsgType_TimeTick {
|
||||
findMapMutex.Lock()
|
||||
eofMsgMap[consumer] = tsMsg.(*TimeTickMsg).Base.Timestamp
|
||||
findMapMutex.Unlock()
|
||||
sp.Finish()
|
||||
return
|
||||
}
|
||||
sp.Finish()
|
||||
case <-ms.stopConsumeChan[consumer]:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func checkTimeTickMsg(msg map[Consumer]Timestamp,
|
||||
isChannelReady map[Consumer]bool,
|
||||
mu *sync.RWMutex) (Timestamp, bool) {
|
||||
checkMap := make(map[Timestamp]int)
|
||||
var maxTime Timestamp = 0
|
||||
for _, v := range msg {
|
||||
checkMap[v]++
|
||||
if v > maxTime {
|
||||
maxTime = v
|
||||
}
|
||||
}
|
||||
if len(checkMap) <= 1 {
|
||||
for consumer := range msg {
|
||||
isChannelReady[consumer] = false
|
||||
}
|
||||
return maxTime, true
|
||||
}
|
||||
for consumer := range msg {
|
||||
mu.RLock()
|
||||
v := msg[consumer]
|
||||
mu.RUnlock()
|
||||
if v != maxTime {
|
||||
isChannelReady[consumer] = false
|
||||
} else {
|
||||
isChannelReady[consumer] = true
|
||||
}
|
||||
}
|
||||
|
||||
return 0, false
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) Seek(mp *internalpb.MsgPosition) error {
|
||||
if len(mp.MsgID) == 0 {
|
||||
return errors.New("when msgID's length equal to 0, please use AsConsumer interface")
|
||||
}
|
||||
var consumer Consumer
|
||||
var err error
|
||||
var hasWatched bool
|
||||
seekChannel := mp.ChannelName
|
||||
subName := mp.MsgGroup
|
||||
ms.consumerLock.Lock()
|
||||
defer ms.consumerLock.Unlock()
|
||||
consumer, hasWatched = ms.consumers[seekChannel]
|
||||
|
||||
if hasWatched {
|
||||
return errors.New("the channel should has been subscribed")
|
||||
}
|
||||
|
||||
receiveChannel := make(chan client.ConsumerMessage, ms.bufSize)
|
||||
consumer, err = ms.client.Subscribe(client.ConsumerOptions{
|
||||
Topic: seekChannel,
|
||||
SubscriptionName: subName,
|
||||
SubscriptionInitialPosition: client.SubscriptionPositionEarliest,
|
||||
Type: client.KeyShared,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if consumer == nil {
|
||||
return errors.New("Consumer is nil")
|
||||
}
|
||||
|
||||
seekMsgID, err := ms.client.StringToMsgID(mp.MsgID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
consumer.Seek(seekMsgID)
|
||||
ms.addConsumer(consumer, seekChannel)
|
||||
|
||||
//TODO: May cause problem
|
||||
if len(consumer.Chan()) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
return nil
|
||||
case msg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
return errors.New("consumer closed")
|
||||
}
|
||||
consumer.Ack(msg)
|
||||
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(msg.Payload(), &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(msg.Payload(), headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
}
|
||||
if tsMsg.Type() == commonpb.MsgType_TimeTick {
|
||||
if tsMsg.BeginTs() >= mp.Timestamp {
|
||||
ms.unsolvedMutex.Unlock()
|
||||
return nil
|
||||
}
|
||||
continue
|
||||
}
|
||||
if tsMsg.BeginTs() > mp.Timestamp {
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(msg.Topic()),
|
||||
MsgID: typeutil.MsgIDToString(msg.ID()),
|
||||
})
|
||||
ms.unsolvedBuf[consumer] = append(ms.unsolvedBuf[consumer], tsMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//TODO test InMemMsgStream
|
||||
/*
|
||||
type InMemMsgStream struct {
|
||||
buffer chan *MsgPack
|
||||
}
|
||||
|
||||
func (ms *InMemMsgStream) Start() {}
|
||||
func (ms *InMemMsgStream) Close() {}
|
||||
|
||||
func (ms *InMemMsgStream) ProduceOne(msg TsMsg) error {
|
||||
msgPack := MsgPack{}
|
||||
msgPack.BeginTs = msg.BeginTs()
|
||||
msgPack.EndTs = msg.EndTs()
|
||||
msgPack.Msgs = append(msgPack.Msgs, msg)
|
||||
buffer <- &msgPack
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *InMemMsgStream) Produce(msgPack *MsgPack) error {
|
||||
buffer <- msgPack
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ms *InMemMsgStream) Broadcast(msgPack *MsgPack) error {
|
||||
return ms.Produce(msgPack)
|
||||
}
|
||||
|
||||
func (ms *InMemMsgStream) Consume() *MsgPack {
|
||||
select {
|
||||
case msgPack := <-ms.buffer:
|
||||
return msgPack
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *InMemMsgStream) Chan() <- chan *MsgPack {
|
||||
return buffer
|
||||
}
|
||||
*/
|
|
@ -3,6 +3,10 @@ package pulsarms
|
|||
import (
|
||||
"context"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
pulsar2 "github.com/zilliztech/milvus-distributed/internal/msgstream/client/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/ms"
|
||||
|
||||
"github.com/mitchellh/mapstructure"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
@ -24,11 +28,19 @@ func (f *Factory) SetParams(params map[string]interface{}) error {
|
|||
}
|
||||
|
||||
func (f *Factory) NewMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return newPulsarMsgStream(ctx, f.PulsarAddress, f.ReceiveBufSize, f.PulsarBufSize, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
pulsarClient, err := pulsar2.NewPulsarClient(pulsar.ClientOptions{URL: f.PulsarAddress})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return ms.NewMsgStream(ctx, f.ReceiveBufSize, f.PulsarBufSize, pulsarClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *Factory) NewTtMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return newPulsarTtMsgStream(ctx, f.PulsarAddress, f.ReceiveBufSize, f.PulsarBufSize, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
pulsarClient, err := pulsar2.NewPulsarClient(pulsar.ClientOptions{URL: f.PulsarAddress})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return ms.NewTtMsgStream(ctx, f.ReceiveBufSize, f.PulsarBufSize, pulsarClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *Factory) NewQueryMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
|
|
|
@ -3,6 +3,10 @@ package rmqms
|
|||
import (
|
||||
"context"
|
||||
|
||||
rocksmq2 "github.com/zilliztech/milvus-distributed/internal/msgstream/client/rocksmq"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/ms"
|
||||
client "github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
|
||||
"github.com/mitchellh/mapstructure"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
|
@ -26,11 +30,19 @@ func (f *Factory) SetParams(params map[string]interface{}) error {
|
|||
}
|
||||
|
||||
func (f *Factory) NewMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return newRmqMsgStream(ctx, f.ReceiveBufSize, f.RmqBufSize, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
rmqClient, err := rocksmq2.NewRmqClient(client.ClientOptions{Server: rocksmq.Rmq})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return ms.NewMsgStream(ctx, f.ReceiveBufSize, f.RmqBufSize, rmqClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *Factory) NewTtMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
return newRmqTtMsgStream(ctx, f.ReceiveBufSize, f.RmqBufSize, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
rmqClient, err := rocksmq2.NewRmqClient(client.ClientOptions{Server: rocksmq.Rmq})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return ms.NewTtMsgStream(ctx, f.ReceiveBufSize, f.RmqBufSize, rmqClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *Factory) NewQueryMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
|
|
|
@ -27,8 +27,8 @@ import (
|
|||
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/ms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/rmqms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
|
@ -407,8 +407,7 @@ func (node *QueryNode) WatchDmChannels(ctx context.Context, in *queryPb.WatchDmC
|
|||
}
|
||||
|
||||
switch t := ds.dmStream.(type) {
|
||||
case *pulsarms.PulsarTtMsgStream:
|
||||
case *rmqms.RmqTtMsgStream:
|
||||
case *ms.TtMsgStream:
|
||||
default:
|
||||
_ = t
|
||||
errMsg := "type assertion failed for dm message stream"
|
||||
|
|
|
@ -10,6 +10,10 @@ const (
|
|||
SubscriptionPositionEarliest
|
||||
)
|
||||
|
||||
func EarliestMessageID() UniqueID {
|
||||
return -1
|
||||
}
|
||||
|
||||
type ConsumerOptions struct {
|
||||
// The topic that this consumer will subscribe on
|
||||
Topic string
|
||||
|
@ -27,7 +31,9 @@ type ConsumerOptions struct {
|
|||
}
|
||||
|
||||
type ConsumerMessage struct {
|
||||
Consumer
|
||||
MsgID UniqueID
|
||||
Topic string
|
||||
Payload []byte
|
||||
}
|
||||
|
||||
|
|
|
@ -6,6 +6,8 @@ import (
|
|||
"reflect"
|
||||
"strings"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/client"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
)
|
||||
|
||||
|
@ -45,6 +47,10 @@ func PulsarMsgIDToString(messageID pulsar.MessageID) string {
|
|||
return strings.ToValidUTF8(string(messageID.Serialize()), "")
|
||||
}
|
||||
|
||||
func MsgIDToString(messageID client.MessageID) string {
|
||||
return strings.ToValidUTF8(string(messageID.Serialize()), "")
|
||||
}
|
||||
|
||||
func StringToPulsarMsgID(msgString string) (pulsar.MessageID, error) {
|
||||
return pulsar.DeserializeMessageID([]byte(msgString))
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue