mirror of https://github.com/milvus-io/milvus.git
Reorganize msgstream
Signed-off-by: Xiangyu Wang <xiangyu.wang@zilliz.com>pull/4973/head^2
parent
39ac47d16c
commit
87a1e0b662
|
@ -14,7 +14,7 @@ import (
|
|||
|
||||
distributed "github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
||||
func main() {
|
||||
|
@ -22,7 +22,7 @@ func main() {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
dn.Params.Init()
|
||||
logutil.SetupLogger(&dn.Params.Log)
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ import (
|
|||
|
||||
"github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
||||
func main() {
|
||||
|
@ -22,7 +22,7 @@ func main() {
|
|||
dataservice.Params.Init()
|
||||
logutil.SetupLogger(&dataservice.Params.Log)
|
||||
defer log.Sync()
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
|
||||
svr, err := components.NewDataService(ctx, msFactory)
|
||||
if err != nil {
|
||||
|
|
|
@ -12,17 +12,15 @@ import (
|
|||
ds "github.com/zilliztech/milvus-distributed/internal/dataservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/rmqms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/server/rocksmq"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/trace"
|
||||
)
|
||||
|
||||
func newMsgFactory(localMsg bool) msgstream.Factory {
|
||||
if localMsg {
|
||||
return rmqms.NewFactory()
|
||||
return msgstream.NewRmsFactory()
|
||||
}
|
||||
return pulsarms.NewFactory()
|
||||
return msgstream.NewPmsFactory()
|
||||
}
|
||||
|
||||
type MilvusRoles struct {
|
||||
|
|
|
@ -11,7 +11,7 @@ import (
|
|||
distributed "github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/masterservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
@ -27,7 +27,7 @@ func main() {
|
|||
}
|
||||
}()
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
ms, err := distributed.NewMasterService(ctx, msFactory)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -11,14 +11,14 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proxynode"
|
||||
)
|
||||
|
||||
func main() {
|
||||
os.Setenv("DEPLOY_MODE", "DISTRIBUTED")
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
proxynode.Params.Init()
|
||||
logutil.SetupLogger(&proxynode.Params.Log)
|
||||
n, err := components.NewProxyNode(ctx, msFactory)
|
||||
|
|
|
@ -11,13 +11,13 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proxyservice"
|
||||
)
|
||||
|
||||
func main() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
proxyservice.Params.Init()
|
||||
logutil.SetupLogger(&proxyservice.Params.Log)
|
||||
s, err := components.NewProxyService(ctx, msFactory)
|
||||
|
|
|
@ -11,7 +11,7 @@ import (
|
|||
distributed "github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/logutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/querynode"
|
||||
)
|
||||
|
||||
|
@ -27,7 +27,7 @@ func main() {
|
|||
}
|
||||
}()
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
svr, err := distributed.NewQueryNode(ctx, msFactory)
|
||||
|
||||
if err != nil {
|
||||
|
|
|
@ -12,7 +12,7 @@ import (
|
|||
|
||||
distributed "github.com/zilliztech/milvus-distributed/cmd/distributed/components"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/queryservice"
|
||||
)
|
||||
|
||||
|
@ -28,7 +28,7 @@ func main() {
|
|||
}
|
||||
}()
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
|
||||
svr, err := distributed.NewQueryService(ctx, msFactory)
|
||||
if err != nil {
|
||||
|
|
|
@ -9,7 +9,6 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
@ -39,7 +38,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
flushChan := make(chan *flushMsg, chanSize)
|
||||
replica := newReplica()
|
||||
allocFactory := AllocatorFactory{}
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"pulsarAddress": pulsarURL,
|
||||
"receiveBufSize": 1024,
|
||||
|
|
|
@ -10,7 +10,6 @@ import (
|
|||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
|
||||
)
|
||||
|
@ -43,7 +42,7 @@ func TestFlowGraphInsertBufferNode_Operate(t *testing.T) {
|
|||
err = replica.addSegment(1, collMeta.ID, 0, Params.InsertChannelNames[0])
|
||||
require.NoError(t, err)
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": 1024,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
|
|
|
@ -16,7 +16,7 @@ import (
|
|||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
cms "github.com/zilliztech/milvus-distributed/internal/masterservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
|
@ -38,7 +38,7 @@ func TestGrpcService(t *testing.T) {
|
|||
}
|
||||
|
||||
ctx := context.Background()
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
svr, err := NewServer(ctx, msFactory)
|
||||
assert.Nil(t, err)
|
||||
svr.connectQueryService = false
|
||||
|
|
|
@ -10,8 +10,7 @@ import (
|
|||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
|
||||
|
@ -167,7 +166,7 @@ func (idx *indexMock) getFileArray() []string {
|
|||
return ret
|
||||
}
|
||||
|
||||
func consumeMsgChan(timeout time.Duration, targetChan <-chan *ms.MsgPack) {
|
||||
func consumeMsgChan(timeout time.Duration, targetChan <-chan *msgstream.MsgPack) {
|
||||
for {
|
||||
select {
|
||||
case <-time.After(timeout):
|
||||
|
@ -182,7 +181,7 @@ func TestMasterService(t *testing.T) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
Params.Init()
|
||||
core, err := NewCore(ctx, msFactory)
|
||||
assert.Nil(t, err)
|
||||
|
@ -255,8 +254,8 @@ func TestMasterService(t *testing.T) {
|
|||
|
||||
t.Run("time tick", func(t *testing.T) {
|
||||
var timeTick typeutil.Timestamp = 100
|
||||
msgPack := ms.MsgPack{}
|
||||
baseMsg := ms.BaseMsg{
|
||||
msgPack := msgstream.MsgPack{}
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
BeginTimestamp: timeTick,
|
||||
EndTimestamp: timeTick,
|
||||
HashValues: []uint32{0},
|
||||
|
@ -269,7 +268,7 @@ func TestMasterService(t *testing.T) {
|
|||
SourceID: 0,
|
||||
},
|
||||
}
|
||||
timeTickMsg := &ms.TimeTickMsg{
|
||||
timeTickMsg := &msgstream.TimeTickMsg{
|
||||
BaseMsg: baseMsg,
|
||||
TimeTickMsg: timeTickResult,
|
||||
}
|
||||
|
@ -280,14 +279,14 @@ func TestMasterService(t *testing.T) {
|
|||
ttmsg, ok := <-timeTickStream.Chan()
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(ttmsg.Msgs), 1)
|
||||
ttm, ok := (ttmsg.Msgs[0]).(*ms.TimeTickMsg)
|
||||
ttm, ok := (ttmsg.Msgs[0]).(*msgstream.TimeTickMsg)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, ttm.Base.Timestamp, timeTick)
|
||||
|
||||
ddmsg, ok := <-ddStream.Chan()
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(ddmsg.Msgs), 1)
|
||||
ddm, ok := (ddmsg.Msgs[0]).(*ms.TimeTickMsg)
|
||||
ddm, ok := (ddmsg.Msgs[0]).(*msgstream.TimeTickMsg)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, ddm.Base.Timestamp, timeTick)
|
||||
})
|
||||
|
@ -336,7 +335,7 @@ func TestMasterService(t *testing.T) {
|
|||
assert.True(t, ok)
|
||||
assert.True(t, len(msg.Msgs) == 2 || len(msg.Msgs) == 1)
|
||||
|
||||
createMsg, ok := (msg.Msgs[0]).(*ms.CreateCollectionMsg)
|
||||
createMsg, ok := (msg.Msgs[0]).(*msgstream.CreateCollectionMsg)
|
||||
assert.True(t, ok)
|
||||
createMeta, err := core.MetaTable.GetCollectionByName("testColl")
|
||||
assert.Nil(t, err)
|
||||
|
@ -344,14 +343,14 @@ func TestMasterService(t *testing.T) {
|
|||
assert.Equal(t, len(createMeta.PartitionIDs), 1)
|
||||
|
||||
if len(msg.Msgs) == 2 {
|
||||
createPart, ok := (msg.Msgs[1]).(*ms.CreatePartitionMsg)
|
||||
createPart, ok := (msg.Msgs[1]).(*msgstream.CreatePartitionMsg)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, createPart.CollectionName, "testColl")
|
||||
assert.Equal(t, createPart.PartitionID, createMeta.PartitionIDs[0])
|
||||
} else {
|
||||
msg, ok = <-ddStream.Chan()
|
||||
assert.True(t, ok)
|
||||
createPart, ok := (msg.Msgs[0]).(*ms.CreatePartitionMsg)
|
||||
createPart, ok := (msg.Msgs[0]).(*msgstream.CreatePartitionMsg)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, createPart.CollectionName, "testColl")
|
||||
assert.Equal(t, createPart.PartitionID, createMeta.PartitionIDs[0])
|
||||
|
@ -385,7 +384,7 @@ func TestMasterService(t *testing.T) {
|
|||
|
||||
msg, ok = <-ddStream.Chan()
|
||||
assert.True(t, ok)
|
||||
createMsg, ok = (msg.Msgs[0]).(*ms.CreateCollectionMsg)
|
||||
createMsg, ok = (msg.Msgs[0]).(*msgstream.CreateCollectionMsg)
|
||||
assert.True(t, ok)
|
||||
createMeta, err = core.MetaTable.GetCollectionByName("testColl-again")
|
||||
assert.Nil(t, err)
|
||||
|
@ -504,7 +503,7 @@ func TestMasterService(t *testing.T) {
|
|||
msg, ok := <-ddStream.Chan()
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(msg.Msgs), 1)
|
||||
partMsg, ok := (msg.Msgs[0]).(*ms.CreatePartitionMsg)
|
||||
partMsg, ok := (msg.Msgs[0]).(*msgstream.CreatePartitionMsg)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, partMsg.CollectionID, collMeta.ID)
|
||||
assert.Equal(t, partMsg.PartitionID, partMeta.PartitionID)
|
||||
|
@ -566,13 +565,13 @@ func TestMasterService(t *testing.T) {
|
|||
PartitionID: part.PartitionID,
|
||||
}
|
||||
|
||||
msgPack := ms.MsgPack{}
|
||||
baseMsg := ms.BaseMsg{
|
||||
msgPack := msgstream.MsgPack{}
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{0},
|
||||
}
|
||||
segMsg := &ms.SegmentInfoMsg{
|
||||
segMsg := &msgstream.SegmentInfoMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SegmentMsg: datapb.SegmentMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
|
@ -725,13 +724,13 @@ func TestMasterService(t *testing.T) {
|
|||
PartitionID: part.PartitionID,
|
||||
}
|
||||
|
||||
msgPack := ms.MsgPack{}
|
||||
baseMsg := ms.BaseMsg{
|
||||
msgPack := msgstream.MsgPack{}
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{0},
|
||||
}
|
||||
segMsg := &ms.SegmentInfoMsg{
|
||||
segMsg := &msgstream.SegmentInfoMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SegmentMsg: datapb.SegmentMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
|
@ -752,7 +751,7 @@ func TestMasterService(t *testing.T) {
|
|||
assert.Nil(t, err)
|
||||
assert.Equal(t, len(part.SegmentIDs), 2)
|
||||
|
||||
flushMsg := &ms.FlushCompletedMsg{
|
||||
flushMsg := &msgstream.FlushCompletedMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SegmentFlushCompletedMsg: internalpb.SegmentFlushCompletedMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
|
@ -764,7 +763,7 @@ func TestMasterService(t *testing.T) {
|
|||
SegmentID: 1001,
|
||||
},
|
||||
}
|
||||
msgPack.Msgs = []ms.TsMsg{flushMsg}
|
||||
msgPack.Msgs = []msgstream.TsMsg{flushMsg}
|
||||
err = dataServiceSegmentStream.Broadcast(&msgPack)
|
||||
assert.Nil(t, err)
|
||||
time.Sleep(time.Second)
|
||||
|
@ -892,7 +891,7 @@ func TestMasterService(t *testing.T) {
|
|||
msg, ok := <-ddStream.Chan()
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(msg.Msgs), 1)
|
||||
dmsg, ok := (msg.Msgs[0]).(*ms.DropPartitionMsg)
|
||||
dmsg, ok := (msg.Msgs[0]).(*msgstream.DropPartitionMsg)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, dmsg.CollectionID, collMeta.ID)
|
||||
assert.Equal(t, dmsg.PartitionID, dropPartID)
|
||||
|
@ -921,7 +920,7 @@ func TestMasterService(t *testing.T) {
|
|||
msg, ok := <-ddStream.Chan()
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(msg.Msgs), 1)
|
||||
dmsg, ok := (msg.Msgs[0]).(*ms.DropCollectionMsg)
|
||||
dmsg, ok := (msg.Msgs[0]).(*msgstream.DropCollectionMsg)
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, dmsg.CollectionID, collMeta.ID)
|
||||
collArray := pm.GetCollArray()
|
||||
|
|
|
@ -7,7 +7,7 @@ import (
|
|||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
|
||||
|
@ -80,7 +80,7 @@ func BenchmarkAllocTimestamp(b *testing.B) {
|
|||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
Params.Init()
|
||||
core, err := NewCore(ctx, msFactory)
|
||||
|
||||
|
|
|
@ -1,23 +1,21 @@
|
|||
package memms
|
||||
package msgstream
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
||||
var Mmq *MemMQ
|
||||
var once sync.Once
|
||||
|
||||
type Consumer struct {
|
||||
type MemConsumer struct {
|
||||
GroupName string
|
||||
ChannelName string
|
||||
MsgChan chan *msgstream.MsgPack
|
||||
MsgChan chan *MsgPack
|
||||
}
|
||||
|
||||
type MemMQ struct {
|
||||
consumers map[string][]*Consumer
|
||||
consumers map[string][]*MemConsumer
|
||||
consumerMu sync.Mutex
|
||||
}
|
||||
|
||||
|
@ -26,7 +24,7 @@ func (mmq *MemMQ) CreateChannel(channelName string) error {
|
|||
defer mmq.consumerMu.Unlock()
|
||||
|
||||
if _, ok := mmq.consumers[channelName]; !ok {
|
||||
consumers := make([]*Consumer, 0)
|
||||
consumers := make([]*MemConsumer, 0)
|
||||
mmq.consumers[channelName] = consumers
|
||||
}
|
||||
|
||||
|
@ -49,13 +47,13 @@ func (mmq *MemMQ) DestroyChannel(channelName string) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (mmq *MemMQ) CreateConsumerGroup(groupName string, channelName string) (*Consumer, error) {
|
||||
func (mmq *MemMQ) CreateConsumerGroup(groupName string, channelName string) (*MemConsumer, error) {
|
||||
mmq.consumerMu.Lock()
|
||||
defer mmq.consumerMu.Unlock()
|
||||
|
||||
consumers, ok := mmq.consumers[channelName]
|
||||
if !ok {
|
||||
consumers = make([]*Consumer, 0)
|
||||
consumers = make([]*MemConsumer, 0)
|
||||
mmq.consumers[channelName] = consumers
|
||||
}
|
||||
|
||||
|
@ -67,10 +65,10 @@ func (mmq *MemMQ) CreateConsumerGroup(groupName string, channelName string) (*Co
|
|||
}
|
||||
|
||||
// append new
|
||||
consumer := Consumer{
|
||||
consumer := MemConsumer{
|
||||
GroupName: groupName,
|
||||
ChannelName: channelName,
|
||||
MsgChan: make(chan *msgstream.MsgPack, 1024),
|
||||
MsgChan: make(chan *MsgPack, 1024),
|
||||
}
|
||||
|
||||
mmq.consumers[channelName] = append(mmq.consumers[channelName], &consumer)
|
||||
|
@ -86,7 +84,7 @@ func (mmq *MemMQ) DestroyConsumerGroup(groupName string, channelName string) err
|
|||
return nil
|
||||
}
|
||||
|
||||
tempConsumers := make([]*Consumer, 0)
|
||||
tempConsumers := make([]*MemConsumer, 0)
|
||||
for _, consumer := range consumers {
|
||||
if consumer.GroupName == groupName {
|
||||
// send nil to consumer so that client can close it self
|
||||
|
@ -137,7 +135,7 @@ func (mmq *MemMQ) Broadcast(msgPack *MsgPack) error {
|
|||
}
|
||||
|
||||
func (mmq *MemMQ) Consume(groupName string, channelName string) (*MsgPack, error) {
|
||||
var consumer *Consumer = nil
|
||||
var consumer *MemConsumer = nil
|
||||
mmq.consumerMu.Lock()
|
||||
consumers := mmq.consumers[channelName]
|
||||
for _, c := range consumers {
|
||||
|
@ -162,7 +160,7 @@ func InitMmq() error {
|
|||
Mmq = &MemMQ{
|
||||
consumerMu: sync.Mutex{},
|
||||
}
|
||||
Mmq.consumers = make(map[string][]*Consumer)
|
||||
Mmq.consumers = make(map[string][]*MemConsumer)
|
||||
})
|
||||
return err
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package memms
|
||||
package msgstream
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -7,41 +7,28 @@ import (
|
|||
"strconv"
|
||||
"sync"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/util"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
)
|
||||
|
||||
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 MemMsgStream struct {
|
||||
ctx context.Context
|
||||
streamCancel func()
|
||||
|
||||
repackFunc msgstream.RepackFunc
|
||||
repackFunc RepackFunc
|
||||
|
||||
consumers []*Consumer
|
||||
consumers []*MemConsumer
|
||||
producers []string
|
||||
|
||||
receiveBuf chan *msgstream.MsgPack
|
||||
receiveBuf chan *MsgPack
|
||||
|
||||
wait sync.WaitGroup
|
||||
}
|
||||
|
||||
func NewMemMsgStream(ctx context.Context, receiveBufSize int64) (*MemMsgStream, error) {
|
||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
||||
receiveBuf := make(chan *msgstream.MsgPack, receiveBufSize)
|
||||
receiveBuf := make(chan *MsgPack, receiveBufSize)
|
||||
channels := make([]string, 0)
|
||||
consumers := make([]*Consumer, 0)
|
||||
consumers := make([]*MemConsumer, 0)
|
||||
|
||||
stream := &MemMsgStream{
|
||||
ctx: streamCtx,
|
||||
|
@ -94,7 +81,7 @@ func (mms *MemMsgStream) AsConsumer(channels []string, groupName string) {
|
|||
}
|
||||
}
|
||||
|
||||
func (mms *MemMsgStream) Produce(pack *msgstream.MsgPack) error {
|
||||
func (mms *MemMsgStream) Produce(pack *MsgPack) error {
|
||||
tsMsgs := pack.Msgs
|
||||
if len(tsMsgs) <= 0 {
|
||||
log.Printf("Warning: Receive empty msgPack")
|
||||
|
@ -109,7 +96,7 @@ func (mms *MemMsgStream) Produce(pack *msgstream.MsgPack) error {
|
|||
bucketValues := make([]int32, len(hashValues))
|
||||
for index, hashValue := range hashValues {
|
||||
if tsMsg.Type() == commonpb.MsgType_SearchResult {
|
||||
searchResult := tsMsg.(*msgstream.SearchResultMsg)
|
||||
searchResult := tsMsg.(*SearchResultMsg)
|
||||
channelID := searchResult.ResultChannelID
|
||||
channelIDInt, _ := strconv.ParseInt(channelID, 10, 64)
|
||||
if channelIDInt >= int64(len(mms.producers)) {
|
||||
|
@ -123,7 +110,7 @@ func (mms *MemMsgStream) Produce(pack *msgstream.MsgPack) error {
|
|||
reBucketValues[channelID] = bucketValues
|
||||
}
|
||||
|
||||
var result map[int32]*msgstream.MsgPack
|
||||
var result map[int32]*MsgPack
|
||||
var err error
|
||||
if mms.repackFunc != nil {
|
||||
result, err = mms.repackFunc(tsMsgs, reBucketValues)
|
||||
|
@ -131,11 +118,11 @@ func (mms *MemMsgStream) Produce(pack *msgstream.MsgPack) error {
|
|||
msgType := (tsMsgs[0]).Type()
|
||||
switch msgType {
|
||||
case commonpb.MsgType_Insert:
|
||||
result, err = util.InsertRepackFunc(tsMsgs, reBucketValues)
|
||||
result, err = InsertRepackFunc(tsMsgs, reBucketValues)
|
||||
case commonpb.MsgType_Delete:
|
||||
result, err = util.DeleteRepackFunc(tsMsgs, reBucketValues)
|
||||
result, err = DeleteRepackFunc(tsMsgs, reBucketValues)
|
||||
default:
|
||||
result, err = util.DefaultRepackFunc(tsMsgs, reBucketValues)
|
||||
result, err = DefaultRepackFunc(tsMsgs, reBucketValues)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
|
@ -150,7 +137,7 @@ func (mms *MemMsgStream) Produce(pack *msgstream.MsgPack) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (mms *MemMsgStream) Broadcast(msgPack *msgstream.MsgPack) error {
|
||||
func (mms *MemMsgStream) Broadcast(msgPack *MsgPack) error {
|
||||
for _, channelName := range mms.producers {
|
||||
err := Mmq.Produce(channelName, msgPack)
|
||||
if err != nil {
|
||||
|
@ -161,7 +148,7 @@ func (mms *MemMsgStream) Broadcast(msgPack *msgstream.MsgPack) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (mms *MemMsgStream) Consume() *msgstream.MsgPack {
|
||||
func (mms *MemMsgStream) Consume() *MsgPack {
|
||||
for {
|
||||
select {
|
||||
case cm, ok := <-mms.receiveBuf:
|
||||
|
@ -181,7 +168,7 @@ func (mms *MemMsgStream) Consume() *msgstream.MsgPack {
|
|||
receiveMsg func is used to solve search timeout problem
|
||||
which is caused by selectcase
|
||||
*/
|
||||
func (mms *MemMsgStream) receiveMsg(consumer Consumer) {
|
||||
func (mms *MemMsgStream) receiveMsg(consumer MemConsumer) {
|
||||
defer mms.wait.Done()
|
||||
for {
|
||||
select {
|
||||
|
@ -197,10 +184,10 @@ func (mms *MemMsgStream) receiveMsg(consumer Consumer) {
|
|||
}
|
||||
}
|
||||
|
||||
func (mms *MemMsgStream) Chan() <-chan *msgstream.MsgPack {
|
||||
func (mms *MemMsgStream) Chan() <-chan *MsgPack {
|
||||
return mms.receiveBuf
|
||||
}
|
||||
|
||||
func (mms *MemMsgStream) Seek(offset *msgstream.MsgPosition) error {
|
||||
func (mms *MemMsgStream) Seek(offset *MsgPosition) error {
|
||||
return errors.New("MemMsgStream seek not implemented")
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package memms
|
||||
package msgstream
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -6,12 +6,11 @@ import (
|
|||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
||||
func mGetTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
||||
baseMsg := BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
|
@ -29,7 +28,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
|||
Query: nil,
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchMsg := &msgstream.SearchMsg{
|
||||
searchMsg := &SearchMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchRequest: searchRequest,
|
||||
}
|
||||
|
@ -45,7 +44,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
|||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchResultMsg := &msgstream.SearchResultMsg{
|
||||
searchResultMsg := &SearchResultMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchResults: searchResult,
|
||||
}
|
||||
|
@ -96,7 +95,7 @@ func TestStream_GlobalMmq_Func(t *testing.T) {
|
|||
}
|
||||
|
||||
// validate msg produce/consume
|
||||
msg := msgstream.MsgPack{}
|
||||
msg := MsgPack{}
|
||||
err := Mmq.Produce(channels[0], &msg)
|
||||
if err != nil {
|
||||
log.Fatalf("global mmq produce error = %v", err)
|
||||
|
@ -139,9 +138,9 @@ func TestStream_MemMsgStream_Produce(t *testing.T) {
|
|||
defer cs.Close()
|
||||
}
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
var hashValue uint32 = 2
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 1, hashValue))
|
||||
msgPack.Msgs = append(msgPack.Msgs, mGetTsMsg(commonpb.MsgType_Search, 1, hashValue))
|
||||
err := produceStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("new msgstream error = %v", err)
|
||||
|
@ -165,8 +164,8 @@ func TestStream_MemMsgStream_BroadCast(t *testing.T) {
|
|||
defer cs.Close()
|
||||
}
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 1, 100))
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, mGetTsMsg(commonpb.MsgType_Search, 1, 100))
|
||||
err := produceStream.Broadcast(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("new msgstream error = %v", err)
|
|
@ -0,0 +1,103 @@
|
|||
package msgstream
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/mitchellh/mapstructure"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/mqclient"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
rocksmqserver "github.com/zilliztech/milvus-distributed/internal/util/rocksmq/server/rocksmq"
|
||||
)
|
||||
|
||||
type PmsFactory struct {
|
||||
dispatcherFactory ProtoUDFactory
|
||||
// the following members must be public, so that mapstructure.Decode() can access them
|
||||
PulsarAddress string
|
||||
ReceiveBufSize int64
|
||||
PulsarBufSize int64
|
||||
}
|
||||
|
||||
func (f *PmsFactory) SetParams(params map[string]interface{}) error {
|
||||
err := mapstructure.Decode(params, f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *PmsFactory) NewMsgStream(ctx context.Context) (MsgStream, error) {
|
||||
pulsarClient, err := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: f.PulsarAddress})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewMqMsgStream(ctx, f.ReceiveBufSize, f.PulsarBufSize, pulsarClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *PmsFactory) NewTtMsgStream(ctx context.Context) (MsgStream, error) {
|
||||
pulsarClient, err := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: f.PulsarAddress})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewMqTtMsgStream(ctx, f.ReceiveBufSize, f.PulsarBufSize, pulsarClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *PmsFactory) NewQueryMsgStream(ctx context.Context) (MsgStream, error) {
|
||||
return f.NewMsgStream(ctx)
|
||||
}
|
||||
|
||||
func NewPmsFactory() Factory {
|
||||
f := &PmsFactory{
|
||||
dispatcherFactory: ProtoUDFactory{},
|
||||
ReceiveBufSize: 64,
|
||||
PulsarBufSize: 64,
|
||||
}
|
||||
return f
|
||||
}
|
||||
|
||||
type RmsFactory struct {
|
||||
dispatcherFactory ProtoUDFactory
|
||||
// the following members must be public, so that mapstructure.Decode() can access them
|
||||
ReceiveBufSize int64
|
||||
RmqBufSize int64
|
||||
}
|
||||
|
||||
func (f *RmsFactory) SetParams(params map[string]interface{}) error {
|
||||
err := mapstructure.Decode(params, f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *RmsFactory) NewMsgStream(ctx context.Context) (MsgStream, error) {
|
||||
rmqClient, err := mqclient.NewRmqClient(rocksmq.ClientOptions{Server: rocksmqserver.Rmq})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewMqMsgStream(ctx, f.ReceiveBufSize, f.RmqBufSize, rmqClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *RmsFactory) NewTtMsgStream(ctx context.Context) (MsgStream, error) {
|
||||
rmqClient, err := mqclient.NewRmqClient(rocksmq.ClientOptions{Server: rocksmqserver.Rmq})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewMqTtMsgStream(ctx, f.ReceiveBufSize, f.RmqBufSize, rmqClient, f.dispatcherFactory.NewUnmarshalDispatcher())
|
||||
}
|
||||
|
||||
func (f *RmsFactory) NewQueryMsgStream(ctx context.Context) (MsgStream, error) {
|
||||
InitMmq()
|
||||
return NewMemMsgStream(ctx, f.ReceiveBufSize)
|
||||
}
|
||||
|
||||
func NewRmsFactory() Factory {
|
||||
f := &RmsFactory{
|
||||
dispatcherFactory: ProtoUDFactory{},
|
||||
ReceiveBufSize: 1024,
|
||||
RmqBufSize: 1024,
|
||||
}
|
||||
|
||||
rocksmqserver.InitRocksMQ("/tmp/milvus_rdb")
|
||||
return f
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package ms
|
||||
package msgstream
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -11,8 +11,6 @@ import (
|
|||
"github.com/golang/protobuf/proto"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"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/mqclient"
|
||||
|
@ -20,28 +18,12 @@ import (
|
|||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type MessageID = mqclient.MessageID
|
||||
type Client = mqclient.Client
|
||||
type Producer = mqclient.Producer
|
||||
type Consumer = mqclient.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 {
|
||||
type mqMsgStream struct {
|
||||
ctx context.Context
|
||||
client Client
|
||||
producers map[string]Producer
|
||||
client mqclient.Client
|
||||
producers map[string]mqclient.Producer
|
||||
producerChannels []string
|
||||
consumers map[string]Consumer
|
||||
consumers map[string]mqclient.Consumer
|
||||
consumerChannels []string
|
||||
repackFunc RepackFunc
|
||||
unmarshal UnmarshalDispatcher
|
||||
|
@ -53,20 +35,20 @@ type msgStream struct {
|
|||
consumerLock *sync.Mutex
|
||||
}
|
||||
|
||||
func NewMsgStream(ctx context.Context,
|
||||
func NewMqMsgStream(ctx context.Context,
|
||||
receiveBufSize int64,
|
||||
bufSize int64,
|
||||
client Client,
|
||||
unmarshal UnmarshalDispatcher) (*msgStream, error) {
|
||||
client mqclient.Client,
|
||||
unmarshal UnmarshalDispatcher) (*mqMsgStream, error) {
|
||||
|
||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
||||
producers := make(map[string]Producer)
|
||||
consumers := make(map[string]Consumer)
|
||||
producers := make(map[string]mqclient.Producer)
|
||||
consumers := make(map[string]mqclient.Consumer)
|
||||
producerChannels := make([]string, 0)
|
||||
consumerChannels := make([]string, 0)
|
||||
receiveBuf := make(chan *MsgPack, receiveBufSize)
|
||||
|
||||
stream := &msgStream{
|
||||
stream := &mqMsgStream{
|
||||
ctx: streamCtx,
|
||||
client: client,
|
||||
producers: producers,
|
||||
|
@ -85,7 +67,7 @@ func NewMsgStream(ctx context.Context,
|
|||
return stream, nil
|
||||
}
|
||||
|
||||
func (ms *msgStream) AsProducer(channels []string) {
|
||||
func (ms *mqMsgStream) AsProducer(channels []string) {
|
||||
for _, channel := range channels {
|
||||
fn := func() error {
|
||||
pp, err := ms.client.CreateProducer(mqclient.ProducerOptions{Topic: channel})
|
||||
|
@ -102,7 +84,7 @@ func (ms *msgStream) AsProducer(channels []string) {
|
|||
ms.producerLock.Unlock()
|
||||
return nil
|
||||
}
|
||||
err := util.Retry(20, time.Millisecond*200, fn)
|
||||
err := Retry(20, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
errMsg := "Failed to create producer " + channel + ", error = " + err.Error()
|
||||
panic(errMsg)
|
||||
|
@ -110,7 +92,7 @@ func (ms *msgStream) AsProducer(channels []string) {
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) AsConsumer(channels []string,
|
||||
func (ms *mqMsgStream) AsConsumer(channels []string,
|
||||
subName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := ms.consumers[channel]; ok {
|
||||
|
@ -138,7 +120,7 @@ func (ms *msgStream) AsConsumer(channels []string,
|
|||
go ms.receiveMsg(pc)
|
||||
return nil
|
||||
}
|
||||
err := util.Retry(20, time.Millisecond*200, fn)
|
||||
err := Retry(20, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
errMsg := "Failed to create consumer " + channel + ", error = " + err.Error()
|
||||
panic(errMsg)
|
||||
|
@ -146,14 +128,14 @@ func (ms *msgStream) AsConsumer(channels []string,
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) SetRepackFunc(repackFunc RepackFunc) {
|
||||
func (ms *mqMsgStream) SetRepackFunc(repackFunc RepackFunc) {
|
||||
ms.repackFunc = repackFunc
|
||||
}
|
||||
|
||||
func (ms *msgStream) Start() {
|
||||
func (ms *mqMsgStream) Start() {
|
||||
}
|
||||
|
||||
func (ms *msgStream) Close() {
|
||||
func (ms *mqMsgStream) Close() {
|
||||
ms.streamCancel()
|
||||
ms.wait.Wait()
|
||||
|
||||
|
@ -172,7 +154,7 @@ func (ms *msgStream) Close() {
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) Produce(msgPack *MsgPack) error {
|
||||
func (ms *mqMsgStream) Produce(msgPack *MsgPack) error {
|
||||
tsMsgs := msgPack.Msgs
|
||||
if len(tsMsgs) <= 0 {
|
||||
log.Debug("Warning: Receive empty msgPack")
|
||||
|
@ -187,7 +169,7 @@ func (ms *msgStream) Produce(msgPack *MsgPack) error {
|
|||
bucketValues := make([]int32, len(hashValues))
|
||||
for index, hashValue := range hashValues {
|
||||
if tsMsg.Type() == commonpb.MsgType_SearchResult {
|
||||
searchResult := tsMsg.(*msgstream.SearchResultMsg)
|
||||
searchResult := tsMsg.(*SearchResultMsg)
|
||||
channelID := searchResult.ResultChannelID
|
||||
channelIDInt, _ := strconv.ParseInt(channelID, 10, 64)
|
||||
if channelIDInt >= int64(len(ms.producers)) {
|
||||
|
@ -209,11 +191,11 @@ func (ms *msgStream) Produce(msgPack *MsgPack) error {
|
|||
msgType := (tsMsgs[0]).Type()
|
||||
switch msgType {
|
||||
case commonpb.MsgType_Insert:
|
||||
result, err = util.InsertRepackFunc(tsMsgs, reBucketValues)
|
||||
result, err = InsertRepackFunc(tsMsgs, reBucketValues)
|
||||
case commonpb.MsgType_Delete:
|
||||
result, err = util.DeleteRepackFunc(tsMsgs, reBucketValues)
|
||||
result, err = DeleteRepackFunc(tsMsgs, reBucketValues)
|
||||
default:
|
||||
result, err = util.DefaultRepackFunc(tsMsgs, reBucketValues)
|
||||
result, err = DefaultRepackFunc(tsMsgs, reBucketValues)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
|
@ -222,14 +204,14 @@ func (ms *msgStream) Produce(msgPack *MsgPack) error {
|
|||
for k, v := range result {
|
||||
channel := ms.producerChannels[k]
|
||||
for i := 0; i < len(v.Msgs); i++ {
|
||||
sp, spanCtx := trace.MsgSpanFromCtx(v.Msgs[i].TraceCtx(), v.Msgs[i])
|
||||
sp, spanCtx := MsgSpanFromCtx(v.Msgs[i].TraceCtx(), v.Msgs[i])
|
||||
|
||||
mb, err := v.Msgs[i].Marshal(v.Msgs[i])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
m, err := msgstream.ConvertToByteArray(mb)
|
||||
m, err := ConvertToByteArray(mb)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -252,16 +234,16 @@ func (ms *msgStream) Produce(msgPack *MsgPack) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (ms *msgStream) Broadcast(msgPack *MsgPack) error {
|
||||
func (ms *mqMsgStream) Broadcast(msgPack *MsgPack) error {
|
||||
for _, v := range msgPack.Msgs {
|
||||
sp, spanCtx := trace.MsgSpanFromCtx(v.TraceCtx(), v)
|
||||
sp, spanCtx := MsgSpanFromCtx(v.TraceCtx(), v)
|
||||
|
||||
mb, err := v.Marshal(v)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
m, err := msgstream.ConvertToByteArray(mb)
|
||||
m, err := ConvertToByteArray(mb)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -287,7 +269,7 @@ func (ms *msgStream) Broadcast(msgPack *MsgPack) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (ms *msgStream) Consume() *MsgPack {
|
||||
func (ms *mqMsgStream) Consume() *MsgPack {
|
||||
for {
|
||||
select {
|
||||
case cm, ok := <-ms.receiveBuf:
|
||||
|
@ -303,7 +285,7 @@ func (ms *msgStream) Consume() *MsgPack {
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) receiveMsg(consumer Consumer) {
|
||||
func (ms *mqMsgStream) receiveMsg(consumer mqclient.Consumer) {
|
||||
defer ms.wait.Done()
|
||||
|
||||
for {
|
||||
|
@ -327,12 +309,12 @@ func (ms *msgStream) receiveMsg(consumer Consumer) {
|
|||
continue
|
||||
}
|
||||
|
||||
sp, ok := trace.ExtractFromPulsarMsgProperties(tsMsg, msg.Properties())
|
||||
sp, ok := ExtractFromPulsarMsgProperties(tsMsg, msg.Properties())
|
||||
if ok {
|
||||
tsMsg.SetTraceCtx(opentracing.ContextWithSpan(context.Background(), sp))
|
||||
}
|
||||
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
tsMsg.SetPosition(&MsgPosition{
|
||||
ChannelName: filepath.Base(msg.Topic()),
|
||||
//FIXME
|
||||
MsgID: msg.ID().Serialize(),
|
||||
|
@ -346,11 +328,11 @@ func (ms *msgStream) receiveMsg(consumer Consumer) {
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *msgStream) Chan() <-chan *MsgPack {
|
||||
func (ms *mqMsgStream) Chan() <-chan *MsgPack {
|
||||
return ms.receiveBuf
|
||||
}
|
||||
|
||||
func (ms *msgStream) Seek(mp *internalpb.MsgPosition) error {
|
||||
func (ms *mqMsgStream) Seek(mp *internalpb.MsgPosition) error {
|
||||
if _, ok := ms.consumers[mp.ChannelName]; ok {
|
||||
consumer := ms.consumers[mp.ChannelName]
|
||||
messageID, err := ms.client.BytesToMsgID(mp.MsgID)
|
||||
|
@ -367,32 +349,32 @@ func (ms *msgStream) Seek(mp *internalpb.MsgPosition) error {
|
|||
return errors.New("msgStream seek fail")
|
||||
}
|
||||
|
||||
type TtMsgStream struct {
|
||||
msgStream
|
||||
unsolvedBuf map[Consumer][]TsMsg
|
||||
msgPositions map[Consumer]*internalpb.MsgPosition
|
||||
type MqTtMsgStream struct {
|
||||
mqMsgStream
|
||||
unsolvedBuf map[mqclient.Consumer][]TsMsg
|
||||
msgPositions map[mqclient.Consumer]*internalpb.MsgPosition
|
||||
unsolvedMutex *sync.Mutex
|
||||
lastTimeStamp Timestamp
|
||||
syncConsumer chan int
|
||||
stopConsumeChan map[Consumer]chan bool
|
||||
stopConsumeChan map[mqclient.Consumer]chan bool
|
||||
}
|
||||
|
||||
func NewTtMsgStream(ctx context.Context,
|
||||
func NewMqTtMsgStream(ctx context.Context,
|
||||
receiveBufSize int64,
|
||||
bufSize int64,
|
||||
client Client,
|
||||
unmarshal UnmarshalDispatcher) (*TtMsgStream, error) {
|
||||
msgStream, err := NewMsgStream(ctx, receiveBufSize, bufSize, client, unmarshal)
|
||||
client mqclient.Client,
|
||||
unmarshal UnmarshalDispatcher) (*MqTtMsgStream, error) {
|
||||
msgStream, err := NewMqMsgStream(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)
|
||||
unsolvedBuf := make(map[mqclient.Consumer][]TsMsg)
|
||||
stopChannel := make(map[mqclient.Consumer]chan bool)
|
||||
msgPositions := make(map[mqclient.Consumer]*internalpb.MsgPosition)
|
||||
syncConsumer := make(chan int, 1)
|
||||
|
||||
return &TtMsgStream{
|
||||
msgStream: *msgStream,
|
||||
return &MqTtMsgStream{
|
||||
mqMsgStream: *msgStream,
|
||||
unsolvedBuf: unsolvedBuf,
|
||||
msgPositions: msgPositions,
|
||||
unsolvedMutex: &sync.Mutex{},
|
||||
|
@ -401,7 +383,7 @@ func NewTtMsgStream(ctx context.Context,
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) addConsumer(consumer Consumer, channel string) {
|
||||
func (ms *MqTtMsgStream) addConsumer(consumer mqclient.Consumer, channel string) {
|
||||
if len(ms.consumers) == 0 {
|
||||
ms.syncConsumer <- 1
|
||||
}
|
||||
|
@ -417,7 +399,7 @@ func (ms *TtMsgStream) addConsumer(consumer Consumer, channel string) {
|
|||
ms.stopConsumeChan[consumer] = stopConsumeChan
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) AsConsumer(channels []string,
|
||||
func (ms *MqTtMsgStream) AsConsumer(channels []string,
|
||||
subName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := ms.consumers[channel]; ok {
|
||||
|
@ -444,7 +426,7 @@ func (ms *TtMsgStream) AsConsumer(channels []string,
|
|||
ms.consumerLock.Unlock()
|
||||
return nil
|
||||
}
|
||||
err := util.Retry(10, time.Millisecond*200, fn)
|
||||
err := Retry(10, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
errMsg := "Failed to create consumer " + channel + ", error = " + err.Error()
|
||||
panic(errMsg)
|
||||
|
@ -452,14 +434,14 @@ func (ms *TtMsgStream) AsConsumer(channels []string,
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) Start() {
|
||||
func (ms *MqTtMsgStream) Start() {
|
||||
if ms.consumers != nil {
|
||||
ms.wait.Add(1)
|
||||
go ms.bufMsgPackToChannel()
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) Close() {
|
||||
func (ms *MqTtMsgStream) Close() {
|
||||
ms.streamCancel()
|
||||
close(ms.syncConsumer)
|
||||
ms.wait.Wait()
|
||||
|
@ -479,11 +461,11 @@ func (ms *TtMsgStream) Close() {
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) bufMsgPackToChannel() {
|
||||
func (ms *MqTtMsgStream) bufMsgPackToChannel() {
|
||||
defer ms.wait.Done()
|
||||
ms.unsolvedBuf = make(map[Consumer][]TsMsg)
|
||||
isChannelReady := make(map[Consumer]bool)
|
||||
eofMsgTimeStamp := make(map[Consumer]Timestamp)
|
||||
ms.unsolvedBuf = make(map[mqclient.Consumer][]TsMsg)
|
||||
isChannelReady := make(map[mqclient.Consumer]bool)
|
||||
eofMsgTimeStamp := make(map[mqclient.Consumer]Timestamp)
|
||||
|
||||
if _, ok := <-ms.syncConsumer; !ok {
|
||||
log.Debug("consumer closed!")
|
||||
|
@ -570,8 +552,8 @@ func (ms *TtMsgStream) bufMsgPackToChannel() {
|
|||
}
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) findTimeTick(consumer Consumer,
|
||||
eofMsgMap map[Consumer]Timestamp,
|
||||
func (ms *MqTtMsgStream) findTimeTick(consumer mqclient.Consumer,
|
||||
eofMsgMap map[mqclient.Consumer]Timestamp,
|
||||
wg *sync.WaitGroup,
|
||||
findMapMutex *sync.RWMutex) {
|
||||
defer wg.Done()
|
||||
|
@ -599,12 +581,12 @@ func (ms *TtMsgStream) findTimeTick(consumer Consumer,
|
|||
}
|
||||
|
||||
// set msg info to tsMsg
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
tsMsg.SetPosition(&MsgPosition{
|
||||
ChannelName: filepath.Base(msg.Topic()),
|
||||
MsgID: msg.ID().Serialize(),
|
||||
})
|
||||
|
||||
sp, ok := trace.ExtractFromPulsarMsgProperties(tsMsg, msg.Properties())
|
||||
sp, ok := ExtractFromPulsarMsgProperties(tsMsg, msg.Properties())
|
||||
if ok {
|
||||
tsMsg.SetTraceCtx(opentracing.ContextWithSpan(context.Background(), sp))
|
||||
}
|
||||
|
@ -627,8 +609,8 @@ func (ms *TtMsgStream) findTimeTick(consumer Consumer,
|
|||
}
|
||||
}
|
||||
|
||||
func checkTimeTickMsg(msg map[Consumer]Timestamp,
|
||||
isChannelReady map[Consumer]bool,
|
||||
func checkTimeTickMsg(msg map[mqclient.Consumer]Timestamp,
|
||||
isChannelReady map[mqclient.Consumer]bool,
|
||||
mu *sync.RWMutex) (Timestamp, bool) {
|
||||
checkMap := make(map[Timestamp]int)
|
||||
var maxTime Timestamp = 0
|
||||
|
@ -658,11 +640,11 @@ func checkTimeTickMsg(msg map[Consumer]Timestamp,
|
|||
return 0, false
|
||||
}
|
||||
|
||||
func (ms *TtMsgStream) Seek(mp *internalpb.MsgPosition) error {
|
||||
func (ms *MqTtMsgStream) 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 consumer mqclient.Consumer
|
||||
var err error
|
||||
var hasWatched bool
|
||||
seekChannel := mp.ChannelName
|
||||
|
@ -706,7 +688,7 @@ func (ms *TtMsgStream) Seek(mp *internalpb.MsgPosition) error {
|
|||
|
||||
return nil
|
||||
}
|
||||
err = util.Retry(20, time.Millisecond*200, fn)
|
||||
err = Retry(20, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
errMsg := "Failed to seek, error = " + err.Error()
|
||||
panic(errMsg)
|
||||
|
@ -744,7 +726,7 @@ func (ms *TtMsgStream) Seek(mp *internalpb.MsgPosition) error {
|
|||
continue
|
||||
}
|
||||
if tsMsg.BeginTs() > mp.Timestamp {
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
tsMsg.SetPosition(&MsgPosition{
|
||||
ChannelName: filepath.Base(msg.Topic()),
|
||||
MsgID: msg.ID().Serialize(),
|
||||
})
|
|
@ -1,4 +1,4 @@
|
|||
package ms
|
||||
package msgstream
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
@ -7,23 +7,19 @@ import (
|
|||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
|
||||
client "github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/server/rocksmq"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/mqclient"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/mqclient"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
|
||||
client "github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/server/rocksmq"
|
||||
)
|
||||
|
||||
var Params paramtable.BaseTable
|
||||
|
@ -73,7 +69,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
|||
RowIDs: []int64{1},
|
||||
RowData: []*commonpb.Blob{{}},
|
||||
}
|
||||
insertMsg := &msgstream.InsertMsg{
|
||||
insertMsg := &InsertMsg{
|
||||
BaseMsg: baseMsg,
|
||||
InsertRequest: insertRequest,
|
||||
}
|
||||
|
@ -91,7 +87,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
|||
Timestamps: []Timestamp{1},
|
||||
PrimaryKeys: []IntPrimaryKey{1},
|
||||
}
|
||||
deleteMsg := &msgstream.DeleteMsg{
|
||||
deleteMsg := &DeleteMsg{
|
||||
BaseMsg: baseMsg,
|
||||
DeleteRequest: deleteRequest,
|
||||
}
|
||||
|
@ -107,7 +103,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
|||
Query: nil,
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchMsg := &msgstream.SearchMsg{
|
||||
searchMsg := &SearchMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchRequest: searchRequest,
|
||||
}
|
||||
|
@ -123,7 +119,7 @@ func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
|||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchResultMsg := &msgstream.SearchResultMsg{
|
||||
searchResultMsg := &SearchResultMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchResults: searchResult,
|
||||
}
|
||||
|
@ -183,25 +179,25 @@ func initPulsarStream(pulsarAddress string,
|
|||
producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerSubName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
opts ...RepackFunc) (MsgStream, MsgStream) {
|
||||
factory := ProtoUDFactory{}
|
||||
|
||||
// set input stream
|
||||
pulsarClient, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
inputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
var input MsgStream = inputStream
|
||||
|
||||
// set output stream
|
||||
pulsarClient2, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
outputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
var output MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
@ -210,30 +206,30 @@ func initPulsarTtStream(pulsarAddress string,
|
|||
producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerSubName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
opts ...RepackFunc) (MsgStream, MsgStream) {
|
||||
factory := ProtoUDFactory{}
|
||||
|
||||
// set input stream
|
||||
pulsarClient, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
inputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
var input MsgStream = inputStream
|
||||
|
||||
// set output stream
|
||||
pulsarClient2, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
outputStream, _ := NewTtMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream, _ := NewMqTtMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
var output MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
||||
func receiveMsg(outputStream msgstream.MsgStream, msgCount int) {
|
||||
func receiveMsg(outputStream MsgStream, msgCount int) {
|
||||
receiveCount := 0
|
||||
for {
|
||||
result := outputStream.Consume()
|
||||
|
@ -257,7 +253,7 @@ func TestStream_PulsarMsgStream_Insert(t *testing.T) {
|
|||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
|
@ -279,7 +275,7 @@ func TestStream_PulsarMsgStream_Delete(t *testing.T) {
|
|||
producerChannels := []string{c}
|
||||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Delete, 1, 1))
|
||||
//msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Delete, 3, 3))
|
||||
|
||||
|
@ -300,7 +296,7 @@ func TestStream_PulsarMsgStream_Search(t *testing.T) {
|
|||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 3, 3))
|
||||
|
||||
|
@ -320,7 +316,7 @@ func TestStream_PulsarMsgStream_SearchResult(t *testing.T) {
|
|||
producerChannels := []string{c}
|
||||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 3, 3))
|
||||
|
||||
|
@ -340,7 +336,7 @@ func TestStream_PulsarMsgStream_TimeTick(t *testing.T) {
|
|||
producerChannels := []string{c}
|
||||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 3, 3))
|
||||
|
||||
|
@ -361,7 +357,7 @@ func TestStream_PulsarMsgStream_BroadCast(t *testing.T) {
|
|||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 3, 3))
|
||||
|
||||
|
@ -382,7 +378,7 @@ func TestStream_PulsarMsgStream_RepackFunc(t *testing.T) {
|
|||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
|
@ -402,7 +398,7 @@ func TestStream_PulsarMsgStream_InsertRepackFunc(t *testing.T) {
|
|||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
baseMsg := BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{1, 3},
|
||||
|
@ -419,30 +415,30 @@ func TestStream_PulsarMsgStream_InsertRepackFunc(t *testing.T) {
|
|||
PartitionName: "Partition",
|
||||
SegmentID: 1,
|
||||
ChannelID: "1",
|
||||
Timestamps: []msgstream.Timestamp{1, 1},
|
||||
Timestamps: []Timestamp{1, 1},
|
||||
RowIDs: []int64{1, 3},
|
||||
RowData: []*commonpb.Blob{{}, {}},
|
||||
}
|
||||
insertMsg := &msgstream.InsertMsg{
|
||||
insertMsg := &InsertMsg{
|
||||
BaseMsg: baseMsg,
|
||||
InsertRequest: insertRequest,
|
||||
}
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, insertMsg)
|
||||
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
factory := ProtoUDFactory{}
|
||||
|
||||
pulsarClient, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
inputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
inputStream.Start()
|
||||
|
||||
pulsarClient2, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
outputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
var output MsgStream = outputStream
|
||||
|
||||
err := (*inputStream).Produce(&msgPack)
|
||||
if err != nil {
|
||||
|
@ -460,7 +456,7 @@ func TestStream_PulsarMsgStream_DeleteRepackFunc(t *testing.T) {
|
|||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
baseMsg := BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{1, 3},
|
||||
|
@ -475,28 +471,28 @@ func TestStream_PulsarMsgStream_DeleteRepackFunc(t *testing.T) {
|
|||
},
|
||||
CollectionName: "Collection",
|
||||
ChannelID: "1",
|
||||
Timestamps: []msgstream.Timestamp{1, 1},
|
||||
Timestamps: []Timestamp{1, 1},
|
||||
PrimaryKeys: []int64{1, 3},
|
||||
}
|
||||
deleteMsg := &msgstream.DeleteMsg{
|
||||
deleteMsg := &DeleteMsg{
|
||||
BaseMsg: baseMsg,
|
||||
DeleteRequest: deleteRequest,
|
||||
}
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, deleteMsg)
|
||||
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
factory := ProtoUDFactory{}
|
||||
pulsarClient, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
inputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
inputStream.Start()
|
||||
|
||||
pulsarClient2, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
outputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
var output MsgStream = outputStream
|
||||
|
||||
err := (*inputStream).Produce(&msgPack)
|
||||
if err != nil {
|
||||
|
@ -514,23 +510,23 @@ func TestStream_PulsarMsgStream_DefaultRepackFunc(t *testing.T) {
|
|||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 2, 2))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 3, 3))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_QueryNodeStats, 4, 4))
|
||||
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
factory := ProtoUDFactory{}
|
||||
pulsarClient, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
inputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
inputStream.Start()
|
||||
|
||||
pulsarClient2, _ := mqclient.NewPulsarClient(pulsar.ClientOptions{URL: pulsarAddress})
|
||||
outputStream, _ := NewMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream, _ := NewMqMsgStream(context.Background(), 100, 100, pulsarClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
var output MsgStream = outputStream
|
||||
|
||||
err := (*inputStream).Produce(&msgPack)
|
||||
if err != nil {
|
||||
|
@ -547,14 +543,14 @@ func TestStream_PulsarTtMsgStream_Insert(t *testing.T) {
|
|||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack0 := msgstream.MsgPack{}
|
||||
msgPack0 := MsgPack{}
|
||||
msgPack0.Msgs = append(msgPack0.Msgs, getTimeTickMsg(0, 0, 0))
|
||||
|
||||
msgPack1 := msgstream.MsgPack{}
|
||||
msgPack1 := MsgPack{}
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
msgPack2 := msgstream.MsgPack{}
|
||||
msgPack2 := MsgPack{}
|
||||
msgPack2.Msgs = append(msgPack2.Msgs, getTimeTickMsg(5, 5, 5))
|
||||
|
||||
inputStream, outputStream := initPulsarTtStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
|
@ -636,14 +632,14 @@ func TestStream_PulsarTtMsgStream_UnMarshalHeader(t *testing.T) {
|
|||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack0 := msgstream.MsgPack{}
|
||||
msgPack0 := MsgPack{}
|
||||
msgPack0.Msgs = append(msgPack0.Msgs, getTimeTickMsg(0, 0, 0))
|
||||
|
||||
msgPack1 := msgstream.MsgPack{}
|
||||
msgPack1 := MsgPack{}
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
msgPack2 := msgstream.MsgPack{}
|
||||
msgPack2 := MsgPack{}
|
||||
msgPack2.Msgs = append(msgPack2.Msgs, getTimeTickMsg(5, 5, 5))
|
||||
|
||||
inputStream, outputStream := initPulsarTtStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
|
@ -687,7 +683,7 @@ func initRmq(name string) *etcdkv.EtcdKV {
|
|||
return etcdKV
|
||||
}
|
||||
|
||||
func Close(rocksdbName string, intputStream, outputStream msgstream.MsgStream, etcdKV *etcdkv.EtcdKV) {
|
||||
func Close(rocksdbName string, intputStream, outputStream MsgStream, etcdKV *etcdkv.EtcdKV) {
|
||||
intputStream.Close()
|
||||
outputStream.Close()
|
||||
etcdKV.Close()
|
||||
|
@ -698,23 +694,23 @@ func Close(rocksdbName string, intputStream, outputStream msgstream.MsgStream, e
|
|||
func initRmqStream(producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerGroupName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
opts ...RepackFunc) (MsgStream, MsgStream) {
|
||||
factory := ProtoUDFactory{}
|
||||
|
||||
rmqClient, _ := mqclient.NewRmqClient(client.ClientOptions{Server: rocksmq.Rmq})
|
||||
inputStream, _ := NewMsgStream(context.Background(), 100, 100, rmqClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream, _ := NewMqMsgStream(context.Background(), 100, 100, rmqClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
var input MsgStream = inputStream
|
||||
|
||||
rmqClient2, _ := mqclient.NewRmqClient(client.ClientOptions{Server: rocksmq.Rmq})
|
||||
outputStream, _ := NewMsgStream(context.Background(), 100, 100, rmqClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream, _ := NewMqMsgStream(context.Background(), 100, 100, rmqClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerGroupName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
var output MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
@ -722,23 +718,23 @@ func initRmqStream(producerChannels []string,
|
|||
func initRmqTtStream(producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerGroupName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
opts ...RepackFunc) (MsgStream, MsgStream) {
|
||||
factory := ProtoUDFactory{}
|
||||
|
||||
rmqClient, _ := mqclient.NewRmqClient(client.ClientOptions{Server: rocksmq.Rmq})
|
||||
inputStream, _ := NewMsgStream(context.Background(), 100, 100, rmqClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream, _ := NewMqMsgStream(context.Background(), 100, 100, rmqClient, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
var input MsgStream = inputStream
|
||||
|
||||
rmqClient2, _ := mqclient.NewRmqClient(client.ClientOptions{Server: rocksmq.Rmq})
|
||||
outputStream, _ := NewTtMsgStream(context.Background(), 100, 100, rmqClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream, _ := NewMqTtMsgStream(context.Background(), 100, 100, rmqClient2, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerGroupName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
var output MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
@ -748,7 +744,7 @@ func TestStream_RmqMsgStream_Insert(t *testing.T) {
|
|||
consumerChannels := []string{"insert1", "insert2"}
|
||||
consumerGroupName := "InsertGroup"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack := MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
|
@ -769,14 +765,14 @@ func TestStream_RmqTtMsgStream_Insert(t *testing.T) {
|
|||
consumerChannels := []string{"insert1", "insert2"}
|
||||
consumerSubName := "subInsert"
|
||||
|
||||
msgPack0 := msgstream.MsgPack{}
|
||||
msgPack0 := MsgPack{}
|
||||
msgPack0.Msgs = append(msgPack0.Msgs, getTimeTickMsg(0, 0, 0))
|
||||
|
||||
msgPack1 := msgstream.MsgPack{}
|
||||
msgPack1 := MsgPack{}
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
msgPack2 := msgstream.MsgPack{}
|
||||
msgPack2 := MsgPack{}
|
||||
msgPack2.Msgs = append(msgPack2.Msgs, getTimeTickMsg(5, 5, 5))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_insert_tt"
|
|
@ -1,57 +0,0 @@
|
|||
package pulsarms
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/ms"
|
||||
pulsar2 "github.com/zilliztech/milvus-distributed/internal/util/mqclient"
|
||||
|
||||
"github.com/mitchellh/mapstructure"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
)
|
||||
|
||||
type Factory struct {
|
||||
dispatcherFactory msgstream.ProtoUDFactory
|
||||
// the following members must be public, so that mapstructure.Decode() can access them
|
||||
PulsarAddress string
|
||||
ReceiveBufSize int64
|
||||
PulsarBufSize int64
|
||||
}
|
||||
|
||||
func (f *Factory) SetParams(params map[string]interface{}) error {
|
||||
err := mapstructure.Decode(params, f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *Factory) NewMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
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) {
|
||||
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) {
|
||||
return f.NewMsgStream(ctx)
|
||||
}
|
||||
|
||||
func NewFactory() msgstream.Factory {
|
||||
f := &Factory{
|
||||
dispatcherFactory: msgstream.ProtoUDFactory{},
|
||||
ReceiveBufSize: 64,
|
||||
PulsarBufSize: 64,
|
||||
}
|
||||
return f
|
||||
}
|
|
@ -1,181 +0,0 @@
|
|||
package pulsarms
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"testing"
|
||||
|
||||
"errors"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
type MarshalType = msgstream.MarshalType
|
||||
|
||||
type InsertTask struct {
|
||||
Tag string
|
||||
msgstream.InsertMsg
|
||||
}
|
||||
|
||||
func (tt *InsertTask) Marshal(input msgstream.TsMsg) (MarshalType, error) {
|
||||
testMsg := input.(*InsertTask)
|
||||
insertRequest := &testMsg.InsertRequest
|
||||
mb, err := proto.Marshal(insertRequest)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return mb, nil
|
||||
}
|
||||
|
||||
func (tt *InsertTask) Unmarshal(input MarshalType) (msgstream.TsMsg, error) {
|
||||
insertRequest := internalpb.InsertRequest{}
|
||||
in, err := msgstream.ConvertToByteArray(input)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = proto.Unmarshal(in, &insertRequest)
|
||||
testMsg := &InsertTask{InsertMsg: msgstream.InsertMsg{InsertRequest: insertRequest}}
|
||||
testMsg.Tag = testMsg.InsertRequest.PartitionName
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return testMsg, nil
|
||||
}
|
||||
|
||||
func newRepackFunc(tsMsgs []msgstream.TsMsg, hashKeys [][]int32) (map[int32]*msgstream.MsgPack, error) {
|
||||
result := make(map[int32]*msgstream.MsgPack)
|
||||
for i, request := range tsMsgs {
|
||||
if request.Type() != commonpb.MsgType_Insert {
|
||||
return nil, errors.New("msg's must be Insert")
|
||||
}
|
||||
insertRequest := request.(*InsertTask).InsertRequest
|
||||
keys := hashKeys[i]
|
||||
|
||||
timestampLen := len(insertRequest.Timestamps)
|
||||
rowIDLen := len(insertRequest.RowIDs)
|
||||
rowDataLen := len(insertRequest.RowData)
|
||||
keysLen := len(keys)
|
||||
|
||||
if keysLen != timestampLen || keysLen != rowIDLen || keysLen != rowDataLen {
|
||||
return nil, errors.New("the length of hashValue, timestamps, rowIDs, RowData are not equal")
|
||||
}
|
||||
for index, key := range keys {
|
||||
_, ok := result[key]
|
||||
if !ok {
|
||||
msgPack := msgstream.MsgPack{}
|
||||
result[key] = &msgPack
|
||||
}
|
||||
|
||||
sliceRequest := internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: insertRequest.Base.MsgID,
|
||||
Timestamp: insertRequest.Timestamps[index],
|
||||
SourceID: insertRequest.Base.SourceID,
|
||||
},
|
||||
CollectionName: insertRequest.CollectionName,
|
||||
PartitionName: insertRequest.PartitionName,
|
||||
SegmentID: insertRequest.SegmentID,
|
||||
ChannelID: insertRequest.ChannelID,
|
||||
Timestamps: []msgstream.Timestamp{insertRequest.Timestamps[index]},
|
||||
RowIDs: []int64{insertRequest.RowIDs[index]},
|
||||
RowData: []*commonpb.Blob{insertRequest.RowData[index]},
|
||||
}
|
||||
|
||||
insertMsg := &InsertTask{
|
||||
InsertMsg: msgstream.InsertMsg{InsertRequest: sliceRequest},
|
||||
}
|
||||
|
||||
result[key].Msgs = append(result[key].Msgs, insertMsg)
|
||||
}
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func getInsertTask(reqID msgstream.UniqueID, hashValue uint32) msgstream.TsMsg {
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{hashValue},
|
||||
}
|
||||
insertRequest := internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: reqID,
|
||||
Timestamp: 1,
|
||||
SourceID: 1,
|
||||
},
|
||||
CollectionName: "Collection",
|
||||
PartitionName: "Partition",
|
||||
SegmentID: 1,
|
||||
ChannelID: "1",
|
||||
Timestamps: []msgstream.Timestamp{1},
|
||||
RowIDs: []int64{1},
|
||||
RowData: []*commonpb.Blob{{}},
|
||||
}
|
||||
insertMsg := msgstream.InsertMsg{
|
||||
BaseMsg: baseMsg,
|
||||
InsertRequest: insertRequest,
|
||||
}
|
||||
|
||||
testTask := &InsertTask{
|
||||
InsertMsg: insertMsg,
|
||||
}
|
||||
|
||||
return testTask
|
||||
}
|
||||
|
||||
func TestStream_task_Insert(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
producerChannels := []string{"insert1", "insert2"}
|
||||
consumerChannels := []string{"insert1", "insert2"}
|
||||
consumerSubName := "subInsert"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getInsertTask(1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getInsertTask(3, 3))
|
||||
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
inputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
inputStream.SetRepackFunc(newRepackFunc)
|
||||
inputStream.Start()
|
||||
|
||||
dispatcher := factory.NewUnmarshalDispatcher()
|
||||
outputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, dispatcher)
|
||||
testTask := InsertTask{}
|
||||
dispatcher.AddMsgTemplate(commonpb.MsgType_Insert, testTask.Unmarshal)
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveCount := 0
|
||||
for {
|
||||
result := outputStream.Consume()
|
||||
if len(result.Msgs) > 0 {
|
||||
msgs := result.Msgs
|
||||
for _, v := range msgs {
|
||||
receiveCount++
|
||||
// variable v could be type of '*msgstream.TimeTickMsg', here need to check
|
||||
// if type conversation is successful
|
||||
if task, ok := v.(*InsertTask); ok {
|
||||
fmt.Println("msg type: ", v.Type(), ", msg value: ", v, "msg tag: ", task.Tag)
|
||||
}
|
||||
}
|
||||
}
|
||||
if receiveCount >= len(msgPack.Msgs) {
|
||||
break
|
||||
}
|
||||
}
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
|
@ -1,887 +0,0 @@
|
|||
package pulsarms
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"path/filepath"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"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/mqclient"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/trace"
|
||||
)
|
||||
|
||||
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 Consumer = pulsar.Consumer
|
||||
type Producer = pulsar.Producer
|
||||
type MessageID = pulsar.MessageID
|
||||
type UnmarshalDispatcher = msgstream.UnmarshalDispatcher
|
||||
|
||||
type PulsarMsgStream struct {
|
||||
ctx context.Context
|
||||
client pulsar.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()
|
||||
pulsarBufSize int64
|
||||
producerLock *sync.Mutex
|
||||
consumerLock *sync.Mutex
|
||||
consumerReflects []reflect.SelectCase
|
||||
}
|
||||
|
||||
func newPulsarMsgStream(ctx context.Context,
|
||||
address string,
|
||||
receiveBufSize int64,
|
||||
pulsarBufSize int64,
|
||||
unmarshal UnmarshalDispatcher) (*PulsarMsgStream, error) {
|
||||
|
||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
||||
producers := make(map[string]Producer)
|
||||
consumers := make(map[string]Consumer)
|
||||
producerChannels := make([]string, 0)
|
||||
consumerChannels := make([]string, 0)
|
||||
consumerReflects := make([]reflect.SelectCase, 0)
|
||||
receiveBuf := make(chan *MsgPack, receiveBufSize)
|
||||
|
||||
var client pulsar.Client
|
||||
var err error
|
||||
opts := pulsar.ClientOptions{
|
||||
URL: address,
|
||||
}
|
||||
client, err = pulsar.NewClient(opts)
|
||||
if err != nil {
|
||||
defer streamCancel()
|
||||
log.Error("Set pulsar client failed, error", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
stream := &PulsarMsgStream{
|
||||
ctx: streamCtx,
|
||||
client: client,
|
||||
producers: producers,
|
||||
producerChannels: producerChannels,
|
||||
consumers: consumers,
|
||||
consumerChannels: consumerChannels,
|
||||
unmarshal: unmarshal,
|
||||
pulsarBufSize: pulsarBufSize,
|
||||
receiveBuf: receiveBuf,
|
||||
streamCancel: streamCancel,
|
||||
consumerReflects: consumerReflects,
|
||||
producerLock: &sync.Mutex{},
|
||||
consumerLock: &sync.Mutex{},
|
||||
wait: &sync.WaitGroup{},
|
||||
}
|
||||
|
||||
return stream, nil
|
||||
}
|
||||
|
||||
func (ms *PulsarMsgStream) AsProducer(channels []string) {
|
||||
for _, channel := range channels {
|
||||
fn := func() error {
|
||||
pp, err := ms.client.CreateProducer(pulsar.ProducerOptions{Topic: channel})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pp == nil {
|
||||
return errors.New("pulsar is not ready, 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 *PulsarMsgStream) AsConsumer(channels []string,
|
||||
subName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := ms.consumers[channel]; ok {
|
||||
continue
|
||||
}
|
||||
fn := func() error {
|
||||
receiveChannel := make(chan pulsar.ConsumerMessage, ms.pulsarBufSize)
|
||||
pc, err := ms.client.Subscribe(pulsar.ConsumerOptions{
|
||||
Topic: channel,
|
||||
SubscriptionName: subName,
|
||||
Type: pulsar.KeyShared,
|
||||
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pc == nil {
|
||||
return errors.New("pulsar is not ready, consumer is nil")
|
||||
}
|
||||
|
||||
ms.consumers[channel] = pc
|
||||
ms.consumerChannels = append(ms.consumerChannels, channel)
|
||||
ms.consumerReflects = append(ms.consumerReflects, reflect.SelectCase{
|
||||
Dir: reflect.SelectRecv,
|
||||
Chan: reflect.ValueOf(pc.Chan()),
|
||||
})
|
||||
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 *PulsarMsgStream) SetRepackFunc(repackFunc RepackFunc) {
|
||||
ms.repackFunc = repackFunc
|
||||
}
|
||||
|
||||
func (ms *PulsarMsgStream) Start() {
|
||||
}
|
||||
|
||||
func (ms *PulsarMsgStream) 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 *PulsarMsgStream) Produce(msgPack *msgstream.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 pulsar 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 := &pulsar.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 *PulsarMsgStream) Broadcast(msgPack *msgstream.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 := &pulsar.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 *PulsarMsgStream) Consume() *msgstream.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 *PulsarMsgStream) receiveMsg(consumer Consumer) {
|
||||
defer ms.wait.Done()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
return
|
||||
case pulsarMsg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
consumer.Ack(pulsarMsg)
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(pulsarMsg.Topic()),
|
||||
MsgID: pulsarMsg.ID().Serialize(),
|
||||
})
|
||||
|
||||
sp, ok := trace.ExtractFromPulsarMsgProperties(tsMsg, pulsarMsg.Properties())
|
||||
if ok {
|
||||
tsMsg.SetTraceCtx(opentracing.ContextWithSpan(context.Background(), sp))
|
||||
}
|
||||
|
||||
msgPack := MsgPack{Msgs: []TsMsg{tsMsg}}
|
||||
ms.receiveBuf <- &msgPack
|
||||
|
||||
sp.Finish()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *PulsarMsgStream) bufMsgPackToChannel() {
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
log.Debug("done")
|
||||
return
|
||||
default:
|
||||
tsMsgList := make([]TsMsg, 0)
|
||||
|
||||
//for {
|
||||
// ms.consumerLock.Lock()
|
||||
// chosen, value, ok := reflect.Select(ms.consumerReflects)
|
||||
// ms.consumerLock.Unlock()
|
||||
// if !ok {
|
||||
// log.Printf("channel closed")
|
||||
// return
|
||||
// }
|
||||
//
|
||||
// pulsarMsg, ok := value.Interface().(pulsar.ConsumerMessage)
|
||||
//
|
||||
// if !ok {
|
||||
// log.Printf("type assertion failed, not consumer message type")
|
||||
// continue
|
||||
// }
|
||||
// ms.consumers[chosen].AckID(pulsarMsg.ID())
|
||||
//
|
||||
// headerMsg := commonpb.MsgHeader{}
|
||||
// err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
||||
// if err != nil {
|
||||
// log.Printf("Failed to unmarshal message header, error = %v", err)
|
||||
// continue
|
||||
// }
|
||||
// tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
||||
// if err != nil {
|
||||
// log.Printf("Failed to unmarshal tsMsg, error = %v", err)
|
||||
// continue
|
||||
// }
|
||||
//
|
||||
// tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
// ChannelName: filepath.Base(pulsarMsg.Topic()),
|
||||
// MsgID: typeutil.PulsarMsgIDToString(pulsarMsg.ID()),
|
||||
// })
|
||||
// tsMsgList = append(tsMsgList, tsMsg)
|
||||
//
|
||||
// noMoreMessage := true
|
||||
// for i := 0; i < len(ms.consumers); i++ {
|
||||
// if len(ms.consumers[i].Chan()) > 0 {
|
||||
// noMoreMessage = false
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// if noMoreMessage {
|
||||
// break
|
||||
// }
|
||||
//}
|
||||
|
||||
pulsarMsgBuffer := make([]pulsar.ConsumerMessage, 0)
|
||||
ms.consumerLock.Lock()
|
||||
consumers := ms.consumers
|
||||
ms.consumerLock.Unlock()
|
||||
for _, consumer := range consumers {
|
||||
msgLen := len(consumer.Chan())
|
||||
for i := 0; i < msgLen; i++ {
|
||||
msg := <-consumer.Chan()
|
||||
consumer.Ack(msg)
|
||||
pulsarMsgBuffer = append(pulsarMsgBuffer, msg)
|
||||
}
|
||||
}
|
||||
for _, pulsarMsg := range pulsarMsgBuffer {
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
sp, ok := trace.ExtractFromPulsarMsgProperties(tsMsg, pulsarMsg.Properties())
|
||||
if ok {
|
||||
tsMsg.SetTraceCtx(opentracing.ContextWithSpan(context.Background(), sp))
|
||||
}
|
||||
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(pulsarMsg.Topic()),
|
||||
MsgID: pulsarMsg.ID().Serialize(),
|
||||
})
|
||||
tsMsgList = append(tsMsgList, tsMsg)
|
||||
}
|
||||
|
||||
if len(tsMsgList) > 0 {
|
||||
msgPack := MsgPack{Msgs: tsMsgList}
|
||||
ms.receiveBuf <- &msgPack
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *PulsarMsgStream) Chan() <-chan *MsgPack {
|
||||
return ms.receiveBuf
|
||||
}
|
||||
|
||||
func (ms *PulsarMsgStream) Seek(mp *internalpb.MsgPosition) error {
|
||||
if _, ok := ms.consumers[mp.ChannelName]; ok {
|
||||
consumer := ms.consumers[mp.ChannelName]
|
||||
messageID, err := mqclient.DeserializePulsarMsgID(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 PulsarTtMsgStream struct {
|
||||
PulsarMsgStream
|
||||
unsolvedBuf map[Consumer][]TsMsg
|
||||
msgPositions map[Consumer]*internalpb.MsgPosition
|
||||
unsolvedMutex *sync.Mutex
|
||||
lastTimeStamp Timestamp
|
||||
syncConsumer chan int
|
||||
stopConsumeChan map[Consumer]chan bool
|
||||
}
|
||||
|
||||
func newPulsarTtMsgStream(ctx context.Context,
|
||||
address string,
|
||||
receiveBufSize int64,
|
||||
pulsarBufSize int64,
|
||||
unmarshal msgstream.UnmarshalDispatcher) (*PulsarTtMsgStream, error) {
|
||||
pulsarMsgStream, err := newPulsarMsgStream(ctx, address, receiveBufSize, pulsarBufSize, 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 &PulsarTtMsgStream{
|
||||
PulsarMsgStream: *pulsarMsgStream,
|
||||
unsolvedBuf: unsolvedBuf,
|
||||
msgPositions: msgPositions,
|
||||
unsolvedMutex: &sync.Mutex{},
|
||||
syncConsumer: syncConsumer,
|
||||
stopConsumeChan: stopChannel,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (ms *PulsarTtMsgStream) 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: make([]byte, 0),
|
||||
Timestamp: ms.lastTimeStamp,
|
||||
}
|
||||
stopConsumeChan := make(chan bool)
|
||||
ms.stopConsumeChan[consumer] = stopConsumeChan
|
||||
}
|
||||
|
||||
func (ms *PulsarTtMsgStream) AsConsumer(channels []string,
|
||||
subName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := ms.consumers[channel]; ok {
|
||||
continue
|
||||
}
|
||||
fn := func() error {
|
||||
receiveChannel := make(chan pulsar.ConsumerMessage, ms.pulsarBufSize)
|
||||
pc, err := ms.client.Subscribe(pulsar.ConsumerOptions{
|
||||
Topic: channel,
|
||||
SubscriptionName: subName,
|
||||
Type: pulsar.KeyShared,
|
||||
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pc == nil {
|
||||
return errors.New("pulsar is not ready, 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 *PulsarTtMsgStream) Start() {
|
||||
if ms.consumers != nil {
|
||||
ms.wait.Add(1)
|
||||
go ms.bufMsgPackToChannel()
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *PulsarTtMsgStream) 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 *PulsarTtMsgStream) 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 *PulsarTtMsgStream) findTimeTick(consumer Consumer,
|
||||
eofMsgMap map[Consumer]Timestamp,
|
||||
wg *sync.WaitGroup,
|
||||
findMapMutex *sync.RWMutex) {
|
||||
defer wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
return
|
||||
case pulsarMsg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
log.Debug("consumer closed!")
|
||||
return
|
||||
}
|
||||
consumer.Ack(pulsarMsg)
|
||||
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.Payload(), headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
// set pulsar info to tsMsg
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(pulsarMsg.Topic()),
|
||||
MsgID: pulsarMsg.ID().Serialize(),
|
||||
})
|
||||
|
||||
sp, ok := trace.ExtractFromPulsarMsgProperties(tsMsg, pulsarMsg.Properties())
|
||||
if ok {
|
||||
tsMsg.SetTraceCtx(opentracing.ContextWithSpan(context.Background(), sp))
|
||||
}
|
||||
|
||||
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 (ms *PulsarTtMsgStream) 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 pulsar.ConsumerMessage, ms.pulsarBufSize)
|
||||
consumer, err = ms.client.Subscribe(pulsar.ConsumerOptions{
|
||||
Topic: seekChannel,
|
||||
SubscriptionName: subName,
|
||||
Type: pulsar.KeyShared,
|
||||
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if consumer == nil {
|
||||
return errors.New("pulsar is not ready, consumer is nil")
|
||||
}
|
||||
seekMsgID, err := mqclient.DeserializePulsarMsgID(mp.MsgID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
consumer.Seek(seekMsgID)
|
||||
ms.addConsumer(consumer, seekChannel)
|
||||
|
||||
if len(consumer.Chan()) == 0 {
|
||||
return nil
|
||||
}
|
||||
for {
|
||||
select {
|
||||
case <-ms.ctx.Done():
|
||||
return nil
|
||||
case pulsarMsg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
return errors.New("consumer closed")
|
||||
}
|
||||
consumer.Ack(pulsarMsg)
|
||||
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(pulsarMsg.Payload(), &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
}
|
||||
tsMsg, err := ms.unmarshal.Unmarshal(pulsarMsg.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 {
|
||||
return nil
|
||||
}
|
||||
continue
|
||||
}
|
||||
if tsMsg.BeginTs() > mp.Timestamp {
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(pulsarMsg.Topic()),
|
||||
MsgID: pulsarMsg.ID().Serialize(),
|
||||
})
|
||||
ms.unsolvedBuf[consumer] = append(ms.unsolvedBuf[consumer], tsMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
//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
|
||||
}
|
||||
*/
|
|
@ -1,645 +0,0 @@
|
|||
package pulsarms
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"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/util/paramtable"
|
||||
)
|
||||
|
||||
var Params paramtable.BaseTable
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
Params.Init()
|
||||
exitCode := m.Run()
|
||||
os.Exit(exitCode)
|
||||
}
|
||||
|
||||
func repackFunc(msgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, error) {
|
||||
result := make(map[int32]*MsgPack)
|
||||
for i, request := range msgs {
|
||||
keys := hashKeys[i]
|
||||
for _, channelID := range keys {
|
||||
_, ok := result[channelID]
|
||||
if ok == false {
|
||||
msgPack := MsgPack{}
|
||||
result[channelID] = &msgPack
|
||||
}
|
||||
result[channelID].Msgs = append(result[channelID].Msgs, request)
|
||||
}
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
||||
baseMsg := BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{hashValue},
|
||||
}
|
||||
switch msgType {
|
||||
case commonpb.MsgType_Insert:
|
||||
insertRequest := internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: reqID,
|
||||
Timestamp: 11,
|
||||
SourceID: reqID,
|
||||
},
|
||||
CollectionName: "Collection",
|
||||
PartitionName: "Partition",
|
||||
SegmentID: 1,
|
||||
ChannelID: "0",
|
||||
Timestamps: []Timestamp{uint64(reqID)},
|
||||
RowIDs: []int64{1},
|
||||
RowData: []*commonpb.Blob{{}},
|
||||
}
|
||||
insertMsg := &msgstream.InsertMsg{
|
||||
BaseMsg: baseMsg,
|
||||
InsertRequest: insertRequest,
|
||||
}
|
||||
return insertMsg
|
||||
case commonpb.MsgType_Delete:
|
||||
deleteRequest := internalpb.DeleteRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Delete,
|
||||
MsgID: reqID,
|
||||
Timestamp: 11,
|
||||
SourceID: reqID,
|
||||
},
|
||||
CollectionName: "Collection",
|
||||
ChannelID: "1",
|
||||
Timestamps: []Timestamp{1},
|
||||
PrimaryKeys: []IntPrimaryKey{1},
|
||||
}
|
||||
deleteMsg := &msgstream.DeleteMsg{
|
||||
BaseMsg: baseMsg,
|
||||
DeleteRequest: deleteRequest,
|
||||
}
|
||||
return deleteMsg
|
||||
case commonpb.MsgType_Search:
|
||||
searchRequest := internalpb.SearchRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
MsgID: reqID,
|
||||
Timestamp: 11,
|
||||
SourceID: reqID,
|
||||
},
|
||||
Query: nil,
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchMsg := &msgstream.SearchMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchRequest: searchRequest,
|
||||
}
|
||||
return searchMsg
|
||||
case commonpb.MsgType_SearchResult:
|
||||
searchResult := internalpb.SearchResults{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_SearchResult,
|
||||
MsgID: reqID,
|
||||
Timestamp: 1,
|
||||
SourceID: reqID,
|
||||
},
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchResultMsg := &msgstream.SearchResultMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchResults: searchResult,
|
||||
}
|
||||
return searchResultMsg
|
||||
case commonpb.MsgType_TimeTick:
|
||||
timeTickResult := internalpb.TimeTickMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
MsgID: reqID,
|
||||
Timestamp: 1,
|
||||
SourceID: reqID,
|
||||
},
|
||||
}
|
||||
timeTickMsg := &TimeTickMsg{
|
||||
BaseMsg: baseMsg,
|
||||
TimeTickMsg: timeTickResult,
|
||||
}
|
||||
return timeTickMsg
|
||||
case commonpb.MsgType_QueryNodeStats:
|
||||
queryNodeSegStats := internalpb.QueryNodeStats{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_QueryNodeStats,
|
||||
SourceID: reqID,
|
||||
},
|
||||
}
|
||||
queryNodeSegStatsMsg := &QueryNodeStatsMsg{
|
||||
BaseMsg: baseMsg,
|
||||
QueryNodeStats: queryNodeSegStats,
|
||||
}
|
||||
return queryNodeSegStatsMsg
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func getTimeTickMsg(reqID UniqueID, hashValue uint32, time uint64) TsMsg {
|
||||
baseMsg := BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{hashValue},
|
||||
}
|
||||
timeTickResult := internalpb.TimeTickMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
MsgID: reqID,
|
||||
Timestamp: time,
|
||||
SourceID: reqID,
|
||||
},
|
||||
}
|
||||
timeTickMsg := &TimeTickMsg{
|
||||
BaseMsg: baseMsg,
|
||||
TimeTickMsg: timeTickResult,
|
||||
}
|
||||
return timeTickMsg
|
||||
}
|
||||
|
||||
func initPulsarStream(pulsarAddress string,
|
||||
producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerSubName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
|
||||
// set input stream
|
||||
inputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
|
||||
// set output stream
|
||||
outputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
||||
func initPulsarTtStream(pulsarAddress string,
|
||||
producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerSubName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
|
||||
// set input stream
|
||||
inputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
|
||||
// set output stream
|
||||
outputStream, _ := newPulsarTtMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
||||
func receiveMsg(outputStream msgstream.MsgStream, msgCount int) {
|
||||
receiveCount := 0
|
||||
for {
|
||||
result := outputStream.Consume()
|
||||
if len(result.Msgs) > 0 {
|
||||
msgs := result.Msgs
|
||||
for _, v := range msgs {
|
||||
receiveCount++
|
||||
fmt.Println("msg type: ", v.Type(), ", msg value: ", v)
|
||||
}
|
||||
}
|
||||
if receiveCount >= msgCount {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_Insert(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
inputStream, outputStream := initPulsarStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_Delete(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c := funcutil.RandomString(8)
|
||||
producerChannels := []string{c}
|
||||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Delete, 1, 1))
|
||||
//msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Delete, 3, 3))
|
||||
|
||||
inputStream, outputStream := initPulsarStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_Search(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c := funcutil.RandomString(8)
|
||||
producerChannels := []string{c}
|
||||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 3, 3))
|
||||
|
||||
inputStream, outputStream := initPulsarStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_SearchResult(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c := funcutil.RandomString(8)
|
||||
producerChannels := []string{c}
|
||||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 3, 3))
|
||||
|
||||
inputStream, outputStream := initPulsarStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_TimeTick(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c := funcutil.RandomString(8)
|
||||
producerChannels := []string{c}
|
||||
consumerChannels := []string{c}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 3, 3))
|
||||
|
||||
inputStream, outputStream := initPulsarStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_BroadCast(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 3, 3))
|
||||
|
||||
inputStream, outputStream := initPulsarStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Broadcast(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(consumerChannels)*len(msgPack.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_RepackFunc(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
inputStream, outputStream := initPulsarStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName, repackFunc)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_InsertRepackFunc(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{1, 3},
|
||||
}
|
||||
|
||||
insertRequest := internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: 1,
|
||||
Timestamp: 1,
|
||||
SourceID: 1,
|
||||
},
|
||||
CollectionName: "Collection",
|
||||
PartitionName: "Partition",
|
||||
SegmentID: 1,
|
||||
ChannelID: "1",
|
||||
Timestamps: []msgstream.Timestamp{1, 1},
|
||||
RowIDs: []int64{1, 3},
|
||||
RowData: []*commonpb.Blob{{}, {}},
|
||||
}
|
||||
insertMsg := &msgstream.InsertMsg{
|
||||
BaseMsg: baseMsg,
|
||||
InsertRequest: insertRequest,
|
||||
}
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, insertMsg)
|
||||
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
inputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
inputStream.Start()
|
||||
|
||||
outputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
|
||||
err := (*inputStream).Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(output, len(msgPack.Msgs)*2)
|
||||
(*inputStream).Close()
|
||||
(*outputStream).Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_DeleteRepackFunc(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
baseMsg := msgstream.BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{1, 3},
|
||||
}
|
||||
|
||||
deleteRequest := internalpb.DeleteRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Delete,
|
||||
MsgID: 1,
|
||||
Timestamp: 1,
|
||||
SourceID: 1,
|
||||
},
|
||||
CollectionName: "Collection",
|
||||
ChannelID: "1",
|
||||
Timestamps: []msgstream.Timestamp{1, 1},
|
||||
PrimaryKeys: []int64{1, 3},
|
||||
}
|
||||
deleteMsg := &msgstream.DeleteMsg{
|
||||
BaseMsg: baseMsg,
|
||||
DeleteRequest: deleteRequest,
|
||||
}
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, deleteMsg)
|
||||
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
inputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
inputStream.Start()
|
||||
|
||||
outputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
|
||||
err := (*inputStream).Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(output, len(msgPack.Msgs)*2)
|
||||
(*inputStream).Close()
|
||||
(*outputStream).Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarMsgStream_DefaultRepackFunc(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 2, 2))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 3, 3))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_QueryNodeStats, 4, 4))
|
||||
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
inputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
inputStream.Start()
|
||||
|
||||
outputStream, _ := newPulsarMsgStream(context.Background(), pulsarAddress, 100, 100, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerSubName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
|
||||
err := (*inputStream).Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(output, len(msgPack.Msgs))
|
||||
(*inputStream).Close()
|
||||
(*outputStream).Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarTtMsgStream_Insert(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
msgPack0 := msgstream.MsgPack{}
|
||||
msgPack0.Msgs = append(msgPack0.Msgs, getTimeTickMsg(0, 0, 0))
|
||||
|
||||
msgPack1 := msgstream.MsgPack{}
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
msgPack2 := msgstream.MsgPack{}
|
||||
msgPack2.Msgs = append(msgPack2.Msgs, getTimeTickMsg(5, 5, 5))
|
||||
|
||||
inputStream, outputStream := initPulsarTtStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Broadcast(&msgPack0)
|
||||
if err != nil {
|
||||
log.Fatalf("broadcast error = %v", err)
|
||||
}
|
||||
err = inputStream.Produce(&msgPack1)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
err = inputStream.Broadcast(&msgPack2)
|
||||
if err != nil {
|
||||
log.Fatalf("broadcast error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack1.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarTtMsgStream_Seek(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack0 := MsgPack{}
|
||||
msgPack0.Msgs = append(msgPack0.Msgs, getTimeTickMsg(0, 0, 0))
|
||||
|
||||
msgPack1 := MsgPack{}
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 19, 19))
|
||||
|
||||
msgPack2 := MsgPack{}
|
||||
msgPack2.Msgs = append(msgPack2.Msgs, getTimeTickMsg(5, 5, 5))
|
||||
|
||||
msgPack3 := MsgPack{}
|
||||
msgPack3.Msgs = append(msgPack3.Msgs, getTsMsg(commonpb.MsgType_Insert, 14, 14))
|
||||
msgPack3.Msgs = append(msgPack3.Msgs, getTsMsg(commonpb.MsgType_Insert, 9, 9))
|
||||
|
||||
msgPack4 := MsgPack{}
|
||||
msgPack4.Msgs = append(msgPack4.Msgs, getTimeTickMsg(11, 11, 11))
|
||||
|
||||
msgPack5 := MsgPack{}
|
||||
msgPack5.Msgs = append(msgPack5.Msgs, getTimeTickMsg(15, 15, 15))
|
||||
|
||||
inputStream, outputStream := initPulsarTtStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Broadcast(&msgPack0)
|
||||
assert.Nil(t, err)
|
||||
err = inputStream.Produce(&msgPack1)
|
||||
assert.Nil(t, err)
|
||||
err = inputStream.Broadcast(&msgPack2)
|
||||
assert.Nil(t, err)
|
||||
err = inputStream.Produce(&msgPack3)
|
||||
assert.Nil(t, err)
|
||||
err = inputStream.Broadcast(&msgPack4)
|
||||
assert.Nil(t, err)
|
||||
|
||||
outputStream.Consume()
|
||||
receivedMsg := outputStream.Consume()
|
||||
for _, position := range receivedMsg.StartPositions {
|
||||
outputStream.Seek(position)
|
||||
}
|
||||
err = inputStream.Broadcast(&msgPack5)
|
||||
assert.Nil(t, err)
|
||||
//seekMsg, _ := outputStream.Consume()
|
||||
//for _, msg := range seekMsg.Msgs {
|
||||
// assert.Equal(t, msg.BeginTs(), uint64(14))
|
||||
//}
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
||||
|
||||
func TestStream_PulsarTtMsgStream_UnMarshalHeader(t *testing.T) {
|
||||
pulsarAddress, _ := Params.Load("_PulsarAddress")
|
||||
c1, c2 := funcutil.RandomString(8), funcutil.RandomString(8)
|
||||
producerChannels := []string{c1, c2}
|
||||
consumerChannels := []string{c1, c2}
|
||||
consumerSubName := funcutil.RandomString(8)
|
||||
|
||||
msgPack0 := msgstream.MsgPack{}
|
||||
msgPack0.Msgs = append(msgPack0.Msgs, getTimeTickMsg(0, 0, 0))
|
||||
|
||||
msgPack1 := msgstream.MsgPack{}
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
msgPack2 := msgstream.MsgPack{}
|
||||
msgPack2.Msgs = append(msgPack2.Msgs, getTimeTickMsg(5, 5, 5))
|
||||
|
||||
inputStream, outputStream := initPulsarTtStream(pulsarAddress, producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Broadcast(&msgPack0)
|
||||
if err != nil {
|
||||
log.Fatalf("broadcast error = %v", err)
|
||||
}
|
||||
err = inputStream.Produce(&msgPack1)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
err = inputStream.Broadcast(&msgPack2)
|
||||
if err != nil {
|
||||
log.Fatalf("broadcast error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack1.Msgs))
|
||||
inputStream.Close()
|
||||
outputStream.Close()
|
||||
}
|
|
@ -1,25 +1,19 @@
|
|||
package util
|
||||
package msgstream
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
type MsgStream = msgstream.MsgStream
|
||||
type TsMsg = msgstream.TsMsg
|
||||
type MsgPack = msgstream.MsgPack
|
||||
type BaseMsg = msgstream.BaseMsg
|
||||
|
||||
func InsertRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, error) {
|
||||
result := make(map[int32]*MsgPack)
|
||||
for i, request := range tsMsgs {
|
||||
if request.Type() != commonpb.MsgType_Insert {
|
||||
return nil, errors.New("msg's must be Insert")
|
||||
}
|
||||
insertRequest := request.(*msgstream.InsertMsg)
|
||||
insertRequest := request.(*InsertMsg)
|
||||
keys := hashKeys[i]
|
||||
|
||||
timestampLen := len(insertRequest.Timestamps)
|
||||
|
@ -56,7 +50,7 @@ func InsertRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e
|
|||
RowData: []*commonpb.Blob{insertRequest.RowData[index]},
|
||||
}
|
||||
|
||||
insertMsg := &msgstream.InsertMsg{
|
||||
insertMsg := &InsertMsg{
|
||||
BaseMsg: BaseMsg{
|
||||
Ctx: request.TraceCtx(),
|
||||
},
|
||||
|
@ -74,7 +68,7 @@ func DeleteRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e
|
|||
if request.Type() != commonpb.MsgType_Delete {
|
||||
return nil, errors.New("msg's must be Delete")
|
||||
}
|
||||
deleteRequest := request.(*msgstream.DeleteMsg)
|
||||
deleteRequest := request.(*DeleteMsg)
|
||||
keys := hashKeys[i]
|
||||
|
||||
timestampLen := len(deleteRequest.Timestamps)
|
||||
|
@ -105,7 +99,7 @@ func DeleteRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e
|
|||
PrimaryKeys: []int64{deleteRequest.PrimaryKeys[index]},
|
||||
}
|
||||
|
||||
deleteMsg := &msgstream.DeleteMsg{
|
||||
deleteMsg := &DeleteMsg{
|
||||
BaseMsg: BaseMsg{
|
||||
Ctx: request.TraceCtx(),
|
||||
},
|
|
@ -1,4 +1,4 @@
|
|||
package util
|
||||
package msgstream
|
||||
|
||||
import (
|
||||
"log"
|
|
@ -1,62 +0,0 @@
|
|||
package rmqms
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/ms"
|
||||
rocksmq2 "github.com/zilliztech/milvus-distributed/internal/util/mqclient"
|
||||
client "github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
|
||||
"github.com/mitchellh/mapstructure"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/memms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/server/rocksmq"
|
||||
)
|
||||
|
||||
type Factory struct {
|
||||
dispatcherFactory msgstream.ProtoUDFactory
|
||||
// the following members must be public, so that mapstructure.Decode() can access them
|
||||
ReceiveBufSize int64
|
||||
RmqBufSize int64
|
||||
}
|
||||
|
||||
func (f *Factory) SetParams(params map[string]interface{}) error {
|
||||
err := mapstructure.Decode(params, f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (f *Factory) NewMsgStream(ctx context.Context) (msgstream.MsgStream, error) {
|
||||
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) {
|
||||
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) {
|
||||
memms.InitMmq()
|
||||
return memms.NewMemMsgStream(ctx, f.ReceiveBufSize)
|
||||
}
|
||||
|
||||
func NewFactory() msgstream.Factory {
|
||||
f := &Factory{
|
||||
dispatcherFactory: msgstream.ProtoUDFactory{},
|
||||
ReceiveBufSize: 1024,
|
||||
RmqBufSize: 1024,
|
||||
}
|
||||
|
||||
rocksmq.InitRocksMQ("/tmp/milvus_rdb")
|
||||
return f
|
||||
}
|
|
@ -1,669 +0,0 @@
|
|||
package rmqms
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"path/filepath"
|
||||
"reflect"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"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/util"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/mqclient"
|
||||
client "github.com/zilliztech/milvus-distributed/internal/util/rocksmq/client/rocksmq"
|
||||
)
|
||||
|
||||
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 Producer = client.Producer
|
||||
type Consumer = client.Consumer
|
||||
|
||||
type RmqMsgStream struct {
|
||||
ctx context.Context
|
||||
client client.Client
|
||||
producers map[string]Producer
|
||||
producerChannels []string
|
||||
consumers map[string]Consumer
|
||||
consumerChannels []string
|
||||
unmarshal msgstream.UnmarshalDispatcher
|
||||
repackFunc msgstream.RepackFunc
|
||||
|
||||
receiveBuf chan *MsgPack
|
||||
wait *sync.WaitGroup
|
||||
streamCancel func()
|
||||
rmqBufSize int64
|
||||
producerLock *sync.Mutex
|
||||
consumerLock *sync.Mutex
|
||||
consumerReflects []reflect.SelectCase
|
||||
|
||||
scMap *sync.Map
|
||||
}
|
||||
|
||||
func newRmqMsgStream(ctx context.Context, receiveBufSize int64, rmqBufSize int64,
|
||||
unmarshal msgstream.UnmarshalDispatcher) (*RmqMsgStream, error) {
|
||||
|
||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
||||
producers := make(map[string]Producer)
|
||||
producerChannels := make([]string, 0)
|
||||
consumerReflects := make([]reflect.SelectCase, 0)
|
||||
consumers := make(map[string]Consumer)
|
||||
consumerChannels := make([]string, 0)
|
||||
receiveBuf := make(chan *MsgPack, receiveBufSize)
|
||||
|
||||
var clientOpts client.ClientOptions
|
||||
client, err := client.NewClient(clientOpts)
|
||||
if err != nil {
|
||||
defer streamCancel()
|
||||
log.Error("Set rmq client failed, error", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
stream := &RmqMsgStream{
|
||||
ctx: streamCtx,
|
||||
client: client,
|
||||
producers: producers,
|
||||
producerChannels: producerChannels,
|
||||
consumers: consumers,
|
||||
consumerChannels: consumerChannels,
|
||||
unmarshal: unmarshal,
|
||||
receiveBuf: receiveBuf,
|
||||
streamCancel: streamCancel,
|
||||
consumerReflects: consumerReflects,
|
||||
producerLock: &sync.Mutex{},
|
||||
consumerLock: &sync.Mutex{},
|
||||
wait: &sync.WaitGroup{},
|
||||
scMap: &sync.Map{},
|
||||
}
|
||||
|
||||
return stream, nil
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) Start() {
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) Close() {
|
||||
rms.streamCancel()
|
||||
rms.wait.Wait()
|
||||
|
||||
if rms.client != nil {
|
||||
rms.client.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) SetRepackFunc(repackFunc RepackFunc) {
|
||||
rms.repackFunc = repackFunc
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) AsProducer(channels []string) {
|
||||
for _, channel := range channels {
|
||||
pp, err := rms.client.CreateProducer(client.ProducerOptions{Topic: channel})
|
||||
if err == nil {
|
||||
rms.producerLock.Lock()
|
||||
rms.producers[channel] = pp
|
||||
rms.producerChannels = append(rms.producerChannels, channel)
|
||||
rms.producerLock.Unlock()
|
||||
} else {
|
||||
errMsg := "Failed to create producer " + channel + ", error = " + err.Error()
|
||||
panic(errMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) AsConsumer(channels []string, groupName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := rms.consumers[channel]; ok {
|
||||
continue
|
||||
}
|
||||
fn := func() error {
|
||||
receiveChannel := make(chan client.ConsumerMessage, rms.rmqBufSize)
|
||||
pc, err := rms.client.Subscribe(client.ConsumerOptions{
|
||||
Topic: channel,
|
||||
SubscriptionName: groupName,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pc == nil {
|
||||
return errors.New("RocksMQ is not ready, consumer is nil")
|
||||
}
|
||||
|
||||
rms.consumers[channel] = pc
|
||||
rms.consumerChannels = append(rms.consumerChannels, channel)
|
||||
rms.consumerReflects = append(rms.consumerReflects, reflect.SelectCase{
|
||||
Dir: reflect.SelectRecv,
|
||||
Chan: reflect.ValueOf(pc.Chan()),
|
||||
})
|
||||
rms.wait.Add(1)
|
||||
go rms.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 (rms *RmqMsgStream) Produce(pack *msgstream.MsgPack) error {
|
||||
tsMsgs := pack.Msgs
|
||||
if len(tsMsgs) <= 0 {
|
||||
log.Debug("Warning: Receive empty msgPack")
|
||||
return nil
|
||||
}
|
||||
if len(rms.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_SearchResult {
|
||||
searchResult := tsMsg.(*msgstream.SearchResultMsg)
|
||||
channelID := searchResult.ResultChannelID
|
||||
channelIDInt, _ := strconv.ParseInt(channelID, 10, 64)
|
||||
if channelIDInt >= int64(len(rms.producers)) {
|
||||
return errors.New("Failed to produce rmq msg to unKnow channel")
|
||||
}
|
||||
bucketValues[index] = int32(channelIDInt)
|
||||
continue
|
||||
}
|
||||
bucketValues[index] = int32(hashValue % uint32(len(rms.producers)))
|
||||
}
|
||||
reBucketValues[channelID] = bucketValues
|
||||
}
|
||||
var result map[int32]*msgstream.MsgPack
|
||||
var err error
|
||||
if rms.repackFunc != nil {
|
||||
result, err = rms.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 := rms.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}
|
||||
if err := rms.producers[channel].Send(msg); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) Broadcast(msgPack *msgstream.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}
|
||||
|
||||
rms.producerLock.Lock()
|
||||
for _, producer := range rms.producers {
|
||||
if err := producer.Send(
|
||||
msg,
|
||||
); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
rms.producerLock.Unlock()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) Consume() *msgstream.MsgPack {
|
||||
for {
|
||||
select {
|
||||
case cm, ok := <-rms.receiveBuf:
|
||||
if !ok {
|
||||
log.Debug("buf chan closed")
|
||||
return nil
|
||||
}
|
||||
return cm
|
||||
case <-rms.ctx.Done():
|
||||
//log.Debug("context closed")
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
receiveMsg func is used to solve search timeout problem
|
||||
which is caused by selectcase
|
||||
*/
|
||||
func (rms *RmqMsgStream) receiveMsg(consumer Consumer) {
|
||||
defer rms.wait.Done()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-rms.ctx.Done():
|
||||
return
|
||||
case rmqMsg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(rmqMsg.Payload, &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
tsMsg, err := rms.unmarshal.Unmarshal(rmqMsg.Payload, headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(consumer.Topic()),
|
||||
MsgID: mqclient.SerializeRmqID(rmqMsg.MsgID),
|
||||
})
|
||||
|
||||
msgPack := MsgPack{Msgs: []TsMsg{tsMsg}}
|
||||
rms.receiveBuf <- &msgPack
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) Chan() <-chan *msgstream.MsgPack {
|
||||
return rms.receiveBuf
|
||||
}
|
||||
|
||||
func (rms *RmqMsgStream) Seek(mp *msgstream.MsgPosition) error {
|
||||
if _, ok := rms.consumers[mp.ChannelName]; ok {
|
||||
consumer := rms.consumers[mp.ChannelName]
|
||||
msgID, err := mqclient.DeserializeRmqID(mp.MsgID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = consumer.Seek(msgID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
return errors.New("msgStream seek fail")
|
||||
}
|
||||
|
||||
type RmqTtMsgStream struct {
|
||||
RmqMsgStream
|
||||
unsolvedBuf map[Consumer][]TsMsg
|
||||
msgPositions map[Consumer]*internalpb.MsgPosition
|
||||
unsolvedMutex *sync.Mutex
|
||||
lastTimeStamp Timestamp
|
||||
syncConsumer chan int
|
||||
}
|
||||
|
||||
func newRmqTtMsgStream(ctx context.Context, receiveBufSize int64, rmqBufSize int64,
|
||||
unmarshal msgstream.UnmarshalDispatcher) (*RmqTtMsgStream, error) {
|
||||
rmqMsgStream, err := newRmqMsgStream(ctx, receiveBufSize, rmqBufSize, unmarshal)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
unsolvedBuf := make(map[Consumer][]TsMsg)
|
||||
syncConsumer := make(chan int, 1)
|
||||
msgPositions := make(map[Consumer]*internalpb.MsgPosition)
|
||||
|
||||
return &RmqTtMsgStream{
|
||||
RmqMsgStream: *rmqMsgStream,
|
||||
unsolvedBuf: unsolvedBuf,
|
||||
msgPositions: msgPositions,
|
||||
unsolvedMutex: &sync.Mutex{},
|
||||
syncConsumer: syncConsumer,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (rtms *RmqTtMsgStream) addConsumer(consumer Consumer, channel string) {
|
||||
if len(rtms.consumers) == 0 {
|
||||
rtms.syncConsumer <- 1
|
||||
}
|
||||
rtms.consumers[channel] = consumer
|
||||
rtms.unsolvedBuf[consumer] = make([]TsMsg, 0)
|
||||
rtms.msgPositions[consumer] = &internalpb.MsgPosition{
|
||||
ChannelName: channel,
|
||||
MsgID: make([]byte, 0),
|
||||
Timestamp: rtms.lastTimeStamp,
|
||||
}
|
||||
rtms.consumerChannels = append(rtms.consumerChannels, channel)
|
||||
}
|
||||
|
||||
func (rtms *RmqTtMsgStream) AsConsumer(channels []string,
|
||||
groupName string) {
|
||||
for _, channel := range channels {
|
||||
if _, ok := rtms.consumers[channel]; ok {
|
||||
continue
|
||||
}
|
||||
fn := func() error {
|
||||
receiveChannel := make(chan client.ConsumerMessage, rtms.rmqBufSize)
|
||||
pc, err := rtms.client.Subscribe(client.ConsumerOptions{
|
||||
Topic: channel,
|
||||
SubscriptionName: groupName,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if pc == nil {
|
||||
return errors.New("RocksMQ is not ready, consumer is nil")
|
||||
}
|
||||
|
||||
rtms.consumerLock.Lock()
|
||||
rtms.addConsumer(pc, channel)
|
||||
rtms.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 (rtms *RmqTtMsgStream) Start() {
|
||||
if rtms.consumers != nil {
|
||||
rtms.wait.Add(1)
|
||||
go rtms.bufMsgPackToChannel()
|
||||
}
|
||||
}
|
||||
|
||||
func (rtms *RmqTtMsgStream) Close() {
|
||||
rtms.streamCancel()
|
||||
close(rtms.syncConsumer)
|
||||
rtms.wait.Wait()
|
||||
|
||||
if rtms.client != nil {
|
||||
rtms.client.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func (rtms *RmqTtMsgStream) bufMsgPackToChannel() {
|
||||
defer rtms.wait.Done()
|
||||
rtms.unsolvedBuf = make(map[Consumer][]TsMsg)
|
||||
isChannelReady := make(map[Consumer]bool)
|
||||
eofMsgTimeStamp := make(map[Consumer]Timestamp)
|
||||
|
||||
if _, ok := <-rtms.syncConsumer; !ok {
|
||||
log.Debug("consumer closed!")
|
||||
return
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-rtms.ctx.Done():
|
||||
return
|
||||
default:
|
||||
wg := sync.WaitGroup{}
|
||||
findMapMutex := sync.RWMutex{}
|
||||
rtms.consumerLock.Lock()
|
||||
for _, consumer := range rtms.consumers {
|
||||
if isChannelReady[consumer] {
|
||||
continue
|
||||
}
|
||||
wg.Add(1)
|
||||
go rtms.findTimeTick(consumer, eofMsgTimeStamp, &wg, &findMapMutex)
|
||||
}
|
||||
rtms.consumerLock.Unlock()
|
||||
wg.Wait()
|
||||
timeStamp, ok := checkTimeTickMsg(eofMsgTimeStamp, isChannelReady, &findMapMutex)
|
||||
if !ok || timeStamp <= rtms.lastTimeStamp {
|
||||
//log.Printf("All timeTick's timestamps are inconsistent")
|
||||
continue
|
||||
}
|
||||
timeTickBuf := make([]TsMsg, 0)
|
||||
startMsgPosition := make([]*internalpb.MsgPosition, 0)
|
||||
endMsgPositions := make([]*internalpb.MsgPosition, 0)
|
||||
rtms.unsolvedMutex.Lock()
|
||||
for consumer, msgs := range rtms.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)
|
||||
}
|
||||
}
|
||||
rtms.unsolvedBuf[consumer] = tempBuffer
|
||||
|
||||
startMsgPosition = append(startMsgPosition, rtms.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)
|
||||
}
|
||||
rtms.msgPositions[consumer] = newPos
|
||||
}
|
||||
rtms.unsolvedMutex.Unlock()
|
||||
|
||||
msgPack := MsgPack{
|
||||
BeginTs: rtms.lastTimeStamp,
|
||||
EndTs: timeStamp,
|
||||
Msgs: timeTickBuf,
|
||||
StartPositions: startMsgPosition,
|
||||
EndPositions: endMsgPositions,
|
||||
}
|
||||
|
||||
rtms.receiveBuf <- &msgPack
|
||||
rtms.lastTimeStamp = timeStamp
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (rtms *RmqTtMsgStream) findTimeTick(consumer Consumer,
|
||||
eofMsgMap map[Consumer]Timestamp,
|
||||
wg *sync.WaitGroup,
|
||||
findMapMutex *sync.RWMutex) {
|
||||
defer wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-rtms.ctx.Done():
|
||||
return
|
||||
case rmqMsg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
log.Debug("consumer closed!")
|
||||
return
|
||||
}
|
||||
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(rmqMsg.Payload, &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
tsMsg, err := rtms.unmarshal.Unmarshal(rmqMsg.Payload, headerMsg.Base.MsgType)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal tsMsg", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(consumer.Topic()),
|
||||
MsgID: mqclient.SerializeRmqID(rmqMsg.MsgID),
|
||||
})
|
||||
|
||||
rtms.unsolvedMutex.Lock()
|
||||
rtms.unsolvedBuf[consumer] = append(rtms.unsolvedBuf[consumer], tsMsg)
|
||||
rtms.unsolvedMutex.Unlock()
|
||||
|
||||
if headerMsg.Base.MsgType == commonpb.MsgType_TimeTick {
|
||||
findMapMutex.Lock()
|
||||
eofMsgMap[consumer] = tsMsg.(*TimeTickMsg).Base.Timestamp
|
||||
findMapMutex.Unlock()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (rtms *RmqTtMsgStream) Seek(mp *msgstream.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
|
||||
rtms.consumerLock.Lock()
|
||||
defer rtms.consumerLock.Unlock()
|
||||
consumer, hasWatched = rtms.consumers[seekChannel]
|
||||
|
||||
if hasWatched {
|
||||
return errors.New("the channel should has been subscribed")
|
||||
}
|
||||
|
||||
receiveChannel := make(chan client.ConsumerMessage, rtms.rmqBufSize)
|
||||
consumer, err = rtms.client.Subscribe(client.ConsumerOptions{
|
||||
Topic: seekChannel,
|
||||
SubscriptionName: subName,
|
||||
MessageChannel: receiveChannel,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if consumer == nil {
|
||||
return errors.New("RocksMQ is not ready, consumer is nil")
|
||||
}
|
||||
seekMsgID, err := mqclient.DeserializeRmqID(mp.MsgID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
consumer.Seek(seekMsgID)
|
||||
rtms.addConsumer(consumer, seekChannel)
|
||||
|
||||
if len(consumer.Chan()) == 0 {
|
||||
log.Debug("len(consumer.Chan()) = 0")
|
||||
return nil
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-rtms.ctx.Done():
|
||||
return nil
|
||||
case rmqMsg, ok := <-consumer.Chan():
|
||||
if !ok {
|
||||
return errors.New("consumer closed")
|
||||
}
|
||||
|
||||
headerMsg := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal(rmqMsg.Payload, &headerMsg)
|
||||
if err != nil {
|
||||
log.Error("Failed to unmarshal message header", zap.Error(err))
|
||||
}
|
||||
tsMsg, err := rtms.unmarshal.Unmarshal(rmqMsg.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 {
|
||||
return nil
|
||||
}
|
||||
continue
|
||||
}
|
||||
if tsMsg.BeginTs() > mp.Timestamp {
|
||||
tsMsg.SetPosition(&msgstream.MsgPosition{
|
||||
ChannelName: filepath.Base(consumer.Topic()),
|
||||
MsgID: mqclient.SerializeRmqID(rmqMsg.MsgID),
|
||||
})
|
||||
rtms.unsolvedBuf[consumer] = append(rtms.unsolvedBuf[consumer], tsMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
|
@ -1,430 +0,0 @@
|
|||
package rmqms
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||
|
||||
etcdkv "github.com/zilliztech/milvus-distributed/internal/kv/etcd"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/rocksmq/server/rocksmq"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
||||
func repackFunc(msgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, error) {
|
||||
result := make(map[int32]*MsgPack)
|
||||
for i, request := range msgs {
|
||||
keys := hashKeys[i]
|
||||
for _, channelID := range keys {
|
||||
_, ok := result[channelID]
|
||||
if ok == false {
|
||||
msgPack := MsgPack{}
|
||||
result[channelID] = &msgPack
|
||||
}
|
||||
result[channelID].Msgs = append(result[channelID].Msgs, request)
|
||||
}
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func getTsMsg(msgType MsgType, reqID UniqueID, hashValue uint32) TsMsg {
|
||||
baseMsg := BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{hashValue},
|
||||
}
|
||||
switch msgType {
|
||||
case commonpb.MsgType_Insert:
|
||||
insertRequest := internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: reqID,
|
||||
Timestamp: 11,
|
||||
SourceID: reqID,
|
||||
},
|
||||
CollectionName: "Collection",
|
||||
PartitionName: "Partition",
|
||||
SegmentID: 1,
|
||||
ChannelID: "0",
|
||||
Timestamps: []Timestamp{uint64(reqID)},
|
||||
RowIDs: []int64{1},
|
||||
RowData: []*commonpb.Blob{{}},
|
||||
}
|
||||
insertMsg := &msgstream.InsertMsg{
|
||||
BaseMsg: baseMsg,
|
||||
InsertRequest: insertRequest,
|
||||
}
|
||||
return insertMsg
|
||||
case commonpb.MsgType_Delete:
|
||||
deleteRequest := internalpb.DeleteRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Delete,
|
||||
MsgID: reqID,
|
||||
Timestamp: 11,
|
||||
SourceID: reqID,
|
||||
},
|
||||
CollectionName: "Collection",
|
||||
ChannelID: "1",
|
||||
Timestamps: []Timestamp{1},
|
||||
PrimaryKeys: []IntPrimaryKey{1},
|
||||
}
|
||||
deleteMsg := &msgstream.DeleteMsg{
|
||||
BaseMsg: baseMsg,
|
||||
DeleteRequest: deleteRequest,
|
||||
}
|
||||
return deleteMsg
|
||||
case commonpb.MsgType_Search:
|
||||
searchRequest := internalpb.SearchRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Search,
|
||||
MsgID: reqID,
|
||||
Timestamp: 11,
|
||||
SourceID: reqID,
|
||||
},
|
||||
Query: nil,
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchMsg := &msgstream.SearchMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchRequest: searchRequest,
|
||||
}
|
||||
return searchMsg
|
||||
case commonpb.MsgType_SearchResult:
|
||||
searchResult := internalpb.SearchResults{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_SearchResult,
|
||||
MsgID: reqID,
|
||||
Timestamp: 1,
|
||||
SourceID: reqID,
|
||||
},
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
||||
ResultChannelID: "0",
|
||||
}
|
||||
searchResultMsg := &msgstream.SearchResultMsg{
|
||||
BaseMsg: baseMsg,
|
||||
SearchResults: searchResult,
|
||||
}
|
||||
return searchResultMsg
|
||||
case commonpb.MsgType_TimeTick:
|
||||
timeTickResult := internalpb.TimeTickMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
MsgID: reqID,
|
||||
Timestamp: 1,
|
||||
SourceID: reqID,
|
||||
},
|
||||
}
|
||||
timeTickMsg := &TimeTickMsg{
|
||||
BaseMsg: baseMsg,
|
||||
TimeTickMsg: timeTickResult,
|
||||
}
|
||||
return timeTickMsg
|
||||
case commonpb.MsgType_QueryNodeStats:
|
||||
queryNodeSegStats := internalpb.QueryNodeStats{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_QueryNodeStats,
|
||||
SourceID: reqID,
|
||||
},
|
||||
}
|
||||
queryNodeSegStatsMsg := &QueryNodeStatsMsg{
|
||||
BaseMsg: baseMsg,
|
||||
QueryNodeStats: queryNodeSegStats,
|
||||
}
|
||||
return queryNodeSegStatsMsg
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func getTimeTickMsg(reqID UniqueID, hashValue uint32, time uint64) TsMsg {
|
||||
baseMsg := BaseMsg{
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: []uint32{hashValue},
|
||||
}
|
||||
timeTickResult := internalpb.TimeTickMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
MsgID: reqID,
|
||||
Timestamp: time,
|
||||
SourceID: reqID,
|
||||
},
|
||||
}
|
||||
timeTickMsg := &TimeTickMsg{
|
||||
BaseMsg: baseMsg,
|
||||
TimeTickMsg: timeTickResult,
|
||||
}
|
||||
return timeTickMsg
|
||||
}
|
||||
|
||||
func initRmq(name string) *etcdkv.EtcdKV {
|
||||
etcdAddr := os.Getenv("ETCD_ADDRESS")
|
||||
if etcdAddr == "" {
|
||||
etcdAddr = "localhost:2379"
|
||||
}
|
||||
cli, err := clientv3.New(clientv3.Config{Endpoints: []string{etcdAddr}})
|
||||
if err != nil {
|
||||
log.Fatalf("New clientv3 error = %v", err)
|
||||
}
|
||||
etcdKV := etcdkv.NewEtcdKV(cli, "/etcd/test/root")
|
||||
idAllocator := allocator.NewGlobalIDAllocator("dummy", etcdKV)
|
||||
_ = idAllocator.Initialize()
|
||||
|
||||
err = rocksmq.InitRmq(name, idAllocator)
|
||||
|
||||
if err != nil {
|
||||
log.Fatalf("InitRmq error = %v", err)
|
||||
}
|
||||
return etcdKV
|
||||
}
|
||||
|
||||
func Close(rocksdbName string, intputStream, outputStream msgstream.MsgStream, etcdKV *etcdkv.EtcdKV) {
|
||||
intputStream.Close()
|
||||
outputStream.Close()
|
||||
etcdKV.Close()
|
||||
err := os.RemoveAll(rocksdbName)
|
||||
fmt.Println(err)
|
||||
}
|
||||
|
||||
func initRmqStream(producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerGroupName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
|
||||
inputStream, _ := newRmqMsgStream(context.Background(), 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
|
||||
outputStream, _ := newRmqMsgStream(context.Background(), 100, 100, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerGroupName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
||||
func initRmqTtStream(producerChannels []string,
|
||||
consumerChannels []string,
|
||||
consumerGroupName string,
|
||||
opts ...RepackFunc) (msgstream.MsgStream, msgstream.MsgStream) {
|
||||
factory := msgstream.ProtoUDFactory{}
|
||||
|
||||
inputStream, _ := newRmqMsgStream(context.Background(), 100, 100, factory.NewUnmarshalDispatcher())
|
||||
inputStream.AsProducer(producerChannels)
|
||||
for _, opt := range opts {
|
||||
inputStream.SetRepackFunc(opt)
|
||||
}
|
||||
inputStream.Start()
|
||||
var input msgstream.MsgStream = inputStream
|
||||
|
||||
outputStream, _ := newRmqTtMsgStream(context.Background(), 100, 100, factory.NewUnmarshalDispatcher())
|
||||
outputStream.AsConsumer(consumerChannels, consumerGroupName)
|
||||
outputStream.Start()
|
||||
var output msgstream.MsgStream = outputStream
|
||||
|
||||
return input, output
|
||||
}
|
||||
|
||||
func receiveMsg(outputStream msgstream.MsgStream, msgCount int) {
|
||||
receiveCount := 0
|
||||
for {
|
||||
result := outputStream.Consume()
|
||||
if len(result.Msgs) > 0 {
|
||||
msgs := result.Msgs
|
||||
for _, v := range msgs {
|
||||
receiveCount++
|
||||
fmt.Println("msg type: ", v.Type(), ", msg value: ", v)
|
||||
}
|
||||
}
|
||||
if receiveCount >= msgCount {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestStream_RmqMsgStream_Insert(t *testing.T) {
|
||||
producerChannels := []string{"insert1", "insert2"}
|
||||
consumerChannels := []string{"insert1", "insert2"}
|
||||
consumerGroupName := "InsertGroup"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_insert"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqStream(producerChannels, consumerChannels, consumerGroupName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
||||
|
||||
func TestStream_RmqMsgStream_Delete(t *testing.T) {
|
||||
producerChannels := []string{"delete"}
|
||||
consumerChannels := []string{"delete"}
|
||||
consumerSubName := "subDelete"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Delete, 1, 1))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_delete"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqStream(producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
||||
|
||||
func TestStream_RmqMsgStream_Search(t *testing.T) {
|
||||
producerChannels := []string{"search"}
|
||||
consumerChannels := []string{"search"}
|
||||
consumerSubName := "subSearch"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Search, 3, 3))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_search"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqStream(producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
||||
|
||||
func TestStream_RmqMsgStream_SearchResult(t *testing.T) {
|
||||
producerChannels := []string{"searchResult"}
|
||||
consumerChannels := []string{"searchResult"}
|
||||
consumerSubName := "subSearchResult"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_SearchResult, 3, 3))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_searchresult"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqStream(producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
||||
|
||||
func TestStream_RmqMsgStream_TimeTick(t *testing.T) {
|
||||
producerChannels := []string{"timeTick"}
|
||||
consumerChannels := []string{"timeTick"}
|
||||
consumerSubName := "subTimeTick"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 3, 3))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_timetick"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqStream(producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
||||
|
||||
func TestStream_RmqMsgStream_BroadCast(t *testing.T) {
|
||||
producerChannels := []string{"insert1", "insert2"}
|
||||
consumerChannels := []string{"insert1", "insert2"}
|
||||
consumerSubName := "subInsert"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_TimeTick, 3, 3))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_broadcast"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqStream(producerChannels, consumerChannels, consumerSubName)
|
||||
err := inputStream.Broadcast(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(consumerChannels)*len(msgPack.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
||||
|
||||
func TestStream_RmqMsgStream_RepackFunc(t *testing.T) {
|
||||
producerChannels := []string{"insert1", "insert2"}
|
||||
consumerChannels := []string{"insert1", "insert2"}
|
||||
consumerSubName := "subInsert"
|
||||
|
||||
msgPack := msgstream.MsgPack{}
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack.Msgs = append(msgPack.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_repackfunc"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqStream(producerChannels, consumerChannels, consumerSubName, repackFunc)
|
||||
err := inputStream.Produce(&msgPack)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
receiveMsg(outputStream, len(msgPack.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
||||
|
||||
func TestStream_PulsarTtMsgStream_Insert(t *testing.T) {
|
||||
producerChannels := []string{"insert1", "insert2"}
|
||||
consumerChannels := []string{"insert1", "insert2"}
|
||||
consumerSubName := "subInsert"
|
||||
|
||||
msgPack0 := msgstream.MsgPack{}
|
||||
msgPack0.Msgs = append(msgPack0.Msgs, getTimeTickMsg(0, 0, 0))
|
||||
|
||||
msgPack1 := msgstream.MsgPack{}
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 1, 1))
|
||||
msgPack1.Msgs = append(msgPack1.Msgs, getTsMsg(commonpb.MsgType_Insert, 3, 3))
|
||||
|
||||
msgPack2 := msgstream.MsgPack{}
|
||||
msgPack2.Msgs = append(msgPack2.Msgs, getTimeTickMsg(5, 5, 5))
|
||||
|
||||
rocksdbName := "/tmp/rocksmq_insert_tt"
|
||||
etcdKV := initRmq(rocksdbName)
|
||||
inputStream, outputStream := initRmqTtStream(producerChannels, consumerChannels, consumerSubName)
|
||||
|
||||
err := inputStream.Broadcast(&msgPack0)
|
||||
if err != nil {
|
||||
log.Fatalf("broadcast error = %v", err)
|
||||
}
|
||||
err = inputStream.Produce(&msgPack1)
|
||||
if err != nil {
|
||||
log.Fatalf("produce error = %v", err)
|
||||
}
|
||||
err = inputStream.Broadcast(&msgPack2)
|
||||
if err != nil {
|
||||
log.Fatalf("broadcast error = %v", err)
|
||||
}
|
||||
|
||||
receiveMsg(outputStream, len(msgPack1.Msgs))
|
||||
Close(rocksdbName, inputStream, outputStream, etcdKV)
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
package msgstream
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"runtime"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/opentracing/opentracing-go/ext"
|
||||
"github.com/opentracing/opentracing-go/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/trace"
|
||||
)
|
||||
|
||||
func ExtractFromPulsarMsgProperties(msg TsMsg, properties map[string]string) (opentracing.Span, bool) {
|
||||
if !allowTrace(msg) {
|
||||
return trace.NoopSpan(), false
|
||||
}
|
||||
tracer := opentracing.GlobalTracer()
|
||||
sc, _ := tracer.Extract(opentracing.TextMap, trace.PropertiesReaderWriter{PpMap: properties})
|
||||
name := "receive pulsar msg"
|
||||
opts := []opentracing.StartSpanOption{
|
||||
ext.RPCServerOption(sc),
|
||||
opentracing.Tags{
|
||||
"ID": msg.ID(),
|
||||
"Type": msg.Type(),
|
||||
"HashKeys": msg.HashKeys(),
|
||||
"Position": msg.Position(),
|
||||
}}
|
||||
return opentracing.StartSpan(name, opts...), true
|
||||
}
|
||||
|
||||
func MsgSpanFromCtx(ctx context.Context, msg TsMsg, opts ...opentracing.StartSpanOption) (opentracing.Span, context.Context) {
|
||||
if ctx == nil {
|
||||
return trace.NoopSpan(), ctx
|
||||
}
|
||||
if !allowTrace(msg) {
|
||||
return trace.NoopSpan(), ctx
|
||||
}
|
||||
operationName := "send pulsar msg"
|
||||
opts = append(opts, opentracing.Tags{
|
||||
"ID": msg.ID(),
|
||||
"Type": msg.Type(),
|
||||
"HashKeys": msg.HashKeys(),
|
||||
"Position": msg.Position(),
|
||||
})
|
||||
|
||||
var pcs [1]uintptr
|
||||
n := runtime.Callers(2, pcs[:])
|
||||
if n < 1 {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, operationName, opts...)
|
||||
span.LogFields(log.Error(errors.New("runtime.Callers failed")))
|
||||
return span, ctx
|
||||
}
|
||||
file, line := runtime.FuncForPC(pcs[0]).FileLine(pcs[0])
|
||||
|
||||
if parentSpan := opentracing.SpanFromContext(ctx); parentSpan != nil {
|
||||
opts = append(opts, opentracing.ChildOf(parentSpan.Context()))
|
||||
}
|
||||
span := opentracing.StartSpan(operationName, opts...)
|
||||
ctx = opentracing.ContextWithSpan(ctx, span)
|
||||
|
||||
span.LogFields(log.String("filename", file), log.Int("line", line))
|
||||
|
||||
return span, ctx
|
||||
}
|
||||
|
||||
func allowTrace(in interface{}) bool {
|
||||
if in == nil {
|
||||
return false
|
||||
}
|
||||
switch res := in.(type) {
|
||||
case TsMsg:
|
||||
return !(res.Type() == commonpb.MsgType_TimeTick ||
|
||||
res.Type() == commonpb.MsgType_QueryNodeStats ||
|
||||
res.Type() == commonpb.MsgType_LoadIndex)
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
|
@ -8,11 +8,8 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
|
||||
)
|
||||
|
||||
var Params paramtable.BaseTable
|
||||
|
||||
func newInsertMsgUnmarshal(input []byte) (TsMsg, error) {
|
||||
insertRequest := internalpb.InsertRequest{}
|
||||
err := proto.Unmarshal(input, &insertRequest)
|
||||
|
|
|
@ -8,7 +8,6 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
)
|
||||
|
@ -110,7 +109,7 @@ func TestDataSyncService_Start(t *testing.T) {
|
|||
insertChannels := Params.InsertChannelNames
|
||||
pulsarURL := Params.PulsarAddress
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": receiveBufSize,
|
||||
"pulsarAddress": pulsarURL,
|
||||
|
|
|
@ -19,7 +19,7 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||
minioKV "github.com/zilliztech/milvus-distributed/internal/kv/minio"
|
||||
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/util"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
|
@ -94,7 +94,7 @@ func (loader *indexLoader) execute(l *loadIndex) error {
|
|||
}
|
||||
return nil
|
||||
}
|
||||
err = util.Retry(5, time.Millisecond*200, fn)
|
||||
err = msgstream.Retry(5, time.Millisecond*200, fn)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/indexnode"
|
||||
minioKV "github.com/zilliztech/milvus-distributed/internal/kv/minio"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
|
@ -1015,7 +1014,7 @@ func doInsert(ctx context.Context, collectionID UniqueID, partitionID UniqueID,
|
|||
insertChannels := Params.InsertChannelNames
|
||||
ddChannels := Params.DDChannelNames
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": receiveBufSize,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
|
@ -1081,7 +1080,7 @@ func sentTimeTick(ctx context.Context) error {
|
|||
insertChannels := Params.InsertChannelNames
|
||||
ddChannels := Params.DDChannelNames
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": receiveBufSize,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
|
|
|
@ -27,8 +27,6 @@ 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/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
|
@ -343,48 +341,48 @@ func (node *QueryNode) AddQueryChannel(ctx context.Context, in *queryPb.AddQuery
|
|||
}
|
||||
|
||||
func (node *QueryNode) RemoveQueryChannel(ctx context.Context, in *queryPb.RemoveQueryChannelRequest) (*commonpb.Status, error) {
|
||||
if node.searchService == nil || node.searchService.searchMsgStream == nil {
|
||||
errMsg := "null search service or null search result message stream"
|
||||
status := &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: errMsg,
|
||||
}
|
||||
// if node.searchService == nil || node.searchService.searchMsgStream == nil {
|
||||
// errMsg := "null search service or null search result message stream"
|
||||
// status := &commonpb.Status{
|
||||
// ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
// Reason: errMsg,
|
||||
// }
|
||||
|
||||
return status, errors.New(errMsg)
|
||||
}
|
||||
// return status, errors.New(errMsg)
|
||||
// }
|
||||
|
||||
searchStream, ok := node.searchService.searchMsgStream.(*pulsarms.PulsarMsgStream)
|
||||
if !ok {
|
||||
errMsg := "type assertion failed for search message stream"
|
||||
status := &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: errMsg,
|
||||
}
|
||||
// searchStream, ok := node.searchService.searchMsgStream.(*pulsarms.PulsarMsgStream)
|
||||
// if !ok {
|
||||
// errMsg := "type assertion failed for search message stream"
|
||||
// status := &commonpb.Status{
|
||||
// ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
// Reason: errMsg,
|
||||
// }
|
||||
|
||||
return status, errors.New(errMsg)
|
||||
}
|
||||
// return status, errors.New(errMsg)
|
||||
// }
|
||||
|
||||
resultStream, ok := node.searchService.searchResultMsgStream.(*pulsarms.PulsarMsgStream)
|
||||
if !ok {
|
||||
errMsg := "type assertion failed for search result message stream"
|
||||
status := &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
Reason: errMsg,
|
||||
}
|
||||
// resultStream, ok := node.searchService.searchResultMsgStream.(*pulsarms.PulsarMsgStream)
|
||||
// if !ok {
|
||||
// errMsg := "type assertion failed for search result message stream"
|
||||
// status := &commonpb.Status{
|
||||
// ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
// Reason: errMsg,
|
||||
// }
|
||||
|
||||
return status, errors.New(errMsg)
|
||||
}
|
||||
// return status, errors.New(errMsg)
|
||||
// }
|
||||
|
||||
// remove request channel
|
||||
consumeChannels := []string{in.RequestChannelID}
|
||||
consumeSubName := Params.MsgChannelSubName
|
||||
// TODO: searchStream.RemovePulsarConsumers(producerChannels)
|
||||
searchStream.AsConsumer(consumeChannels, consumeSubName)
|
||||
// // remove request channel
|
||||
// consumeChannels := []string{in.RequestChannelID}
|
||||
// consumeSubName := Params.MsgChannelSubName
|
||||
// // TODO: searchStream.RemovePulsarConsumers(producerChannels)
|
||||
// searchStream.AsConsumer(consumeChannels, consumeSubName)
|
||||
|
||||
// remove result channel
|
||||
producerChannels := []string{in.ResultChannelID}
|
||||
// TODO: resultStream.RemovePulsarProducer(producerChannels)
|
||||
resultStream.AsProducer(producerChannels)
|
||||
// // remove result channel
|
||||
// producerChannels := []string{in.ResultChannelID}
|
||||
// // TODO: resultStream.RemovePulsarProducer(producerChannels)
|
||||
// resultStream.AsProducer(producerChannels)
|
||||
|
||||
status := &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
|
@ -407,7 +405,7 @@ func (node *QueryNode) WatchDmChannels(ctx context.Context, in *queryPb.WatchDmC
|
|||
}
|
||||
|
||||
switch t := ds.dmStream.(type) {
|
||||
case *ms.TtMsgStream:
|
||||
case *msgstream.MqTtMsgStream:
|
||||
default:
|
||||
_ = t
|
||||
errMsg := "type assertion failed for dm message stream"
|
||||
|
|
|
@ -12,7 +12,7 @@ import (
|
|||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
|
@ -166,7 +166,7 @@ func newQueryNodeMock() *QueryNode {
|
|||
}()
|
||||
}
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
svr := NewQueryNode(ctx, Params.QueryNodeID, msFactory)
|
||||
err := svr.SetQueryService(&queryServiceMock{})
|
||||
if err != nil {
|
||||
|
|
|
@ -12,7 +12,6 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
|
@ -95,7 +94,7 @@ func TestSearch_Search(t *testing.T) {
|
|||
msgPackSearch := msgstream.MsgPack{}
|
||||
msgPackSearch.Msgs = append(msgPackSearch.Msgs, searchMsg)
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": receiveBufSize,
|
||||
"pulsarAddress": pulsarURL,
|
||||
|
@ -224,7 +223,7 @@ func TestSearch_SearchMultiSegments(t *testing.T) {
|
|||
pulsarURL := Params.PulsarAddress
|
||||
const receiveBufSize = 1024
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": receiveBufSize,
|
||||
"pulsarAddress": pulsarURL,
|
||||
|
|
|
@ -5,7 +5,6 @@ import (
|
|||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
)
|
||||
|
||||
// NOTE: start pulsar before test
|
||||
|
@ -13,7 +12,7 @@ func TestStatsService_start(t *testing.T) {
|
|||
node := newQueryNodeMock()
|
||||
initTestMeta(t, node, 0, 0)
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"PulsarAddress": Params.PulsarAddress,
|
||||
"ReceiveBufSize": 1024,
|
||||
|
@ -33,7 +32,7 @@ func TestSegmentManagement_sendSegmentStatistic(t *testing.T) {
|
|||
// start pulsar
|
||||
producerChannels := []string{Params.StatsChannelName}
|
||||
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
m := map[string]interface{}{
|
||||
"receiveBufSize": receiveBufSize,
|
||||
"pulsarAddress": Params.PulsarAddress,
|
||||
|
|
|
@ -6,14 +6,14 @@ import (
|
|||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
|
||||
)
|
||||
|
||||
func TestQueryService_Init(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
service, err := NewQueryService(context.Background(), msFactory)
|
||||
assert.Nil(t, err)
|
||||
service.Init()
|
||||
|
@ -43,7 +43,7 @@ func TestQueryService_Init(t *testing.T) {
|
|||
|
||||
func TestQueryService_load(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
msFactory := pulsarms.NewFactory()
|
||||
msFactory := msgstream.NewPmsFactory()
|
||||
service, err := NewQueryService(context.Background(), msFactory)
|
||||
assert.Nil(t, err)
|
||||
service.Init()
|
||||
|
|
|
@ -2,19 +2,15 @@ package trace
|
|||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"io"
|
||||
"runtime"
|
||||
"strings"
|
||||
|
||||
"errors"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/opentracing/opentracing-go/ext"
|
||||
"github.com/opentracing/opentracing-go/log"
|
||||
"github.com/uber/jaeger-client-go"
|
||||
"github.com/uber/jaeger-client-go/config"
|
||||
"github.com/zilliztech/milvus-distributed/internal/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
)
|
||||
|
||||
func InitTracing(serviceName string) io.Closer {
|
||||
|
@ -50,7 +46,7 @@ func InitTracing(serviceName string) io.Closer {
|
|||
|
||||
func StartSpanFromContext(ctx context.Context, opts ...opentracing.StartSpanOption) (opentracing.Span, context.Context) {
|
||||
if ctx == nil {
|
||||
return noopSpan(), ctx
|
||||
return NoopSpan(), ctx
|
||||
}
|
||||
|
||||
var pcs [1]uintptr
|
||||
|
@ -79,7 +75,7 @@ func StartSpanFromContext(ctx context.Context, opts ...opentracing.StartSpanOpti
|
|||
|
||||
func StartSpanFromContextWithOperationName(ctx context.Context, operationName string, opts ...opentracing.StartSpanOption) (opentracing.Span, context.Context) {
|
||||
if ctx == nil {
|
||||
return noopSpan(), ctx
|
||||
return NoopSpan(), ctx
|
||||
}
|
||||
|
||||
var pcs [1]uintptr
|
||||
|
@ -140,73 +136,20 @@ func InfoFromContext(ctx context.Context) (traceID string, sampled bool, found b
|
|||
|
||||
func InjectContextToPulsarMsgProperties(sc opentracing.SpanContext, properties map[string]string) {
|
||||
tracer := opentracing.GlobalTracer()
|
||||
tracer.Inject(sc, opentracing.TextMap, propertiesReaderWriter{properties})
|
||||
tracer.Inject(sc, opentracing.TextMap, PropertiesReaderWriter{properties})
|
||||
}
|
||||
|
||||
func ExtractFromPulsarMsgProperties(msg msgstream.TsMsg, properties map[string]string) (opentracing.Span, bool) {
|
||||
if !allowTrace(msg) {
|
||||
return noopSpan(), false
|
||||
}
|
||||
tracer := opentracing.GlobalTracer()
|
||||
sc, _ := tracer.Extract(opentracing.TextMap, propertiesReaderWriter{properties})
|
||||
name := "receive pulsar msg"
|
||||
opts := []opentracing.StartSpanOption{
|
||||
ext.RPCServerOption(sc),
|
||||
opentracing.Tags{
|
||||
"ID": msg.ID(),
|
||||
"Type": msg.Type(),
|
||||
"HashKeys": msg.HashKeys(),
|
||||
"Position": msg.Position(),
|
||||
}}
|
||||
return opentracing.StartSpan(name, opts...), true
|
||||
type PropertiesReaderWriter struct {
|
||||
PpMap map[string]string
|
||||
}
|
||||
|
||||
func MsgSpanFromCtx(ctx context.Context, msg msgstream.TsMsg, opts ...opentracing.StartSpanOption) (opentracing.Span, context.Context) {
|
||||
if ctx == nil {
|
||||
return noopSpan(), ctx
|
||||
}
|
||||
if !allowTrace(msg) {
|
||||
return noopSpan(), ctx
|
||||
}
|
||||
operationName := "send pulsar msg"
|
||||
opts = append(opts, opentracing.Tags{
|
||||
"ID": msg.ID(),
|
||||
"Type": msg.Type(),
|
||||
"HashKeys": msg.HashKeys(),
|
||||
"Position": msg.Position(),
|
||||
})
|
||||
|
||||
var pcs [1]uintptr
|
||||
n := runtime.Callers(2, pcs[:])
|
||||
if n < 1 {
|
||||
span, ctx := opentracing.StartSpanFromContext(ctx, operationName, opts...)
|
||||
span.LogFields(log.Error(errors.New("runtime.Callers failed")))
|
||||
return span, ctx
|
||||
}
|
||||
file, line := runtime.FuncForPC(pcs[0]).FileLine(pcs[0])
|
||||
|
||||
if parentSpan := opentracing.SpanFromContext(ctx); parentSpan != nil {
|
||||
opts = append(opts, opentracing.ChildOf(parentSpan.Context()))
|
||||
}
|
||||
span := opentracing.StartSpan(operationName, opts...)
|
||||
ctx = opentracing.ContextWithSpan(ctx, span)
|
||||
|
||||
span.LogFields(log.String("filename", file), log.Int("line", line))
|
||||
|
||||
return span, ctx
|
||||
}
|
||||
|
||||
type propertiesReaderWriter struct {
|
||||
ppMap map[string]string
|
||||
}
|
||||
|
||||
func (ppRW propertiesReaderWriter) Set(key, val string) {
|
||||
func (ppRW PropertiesReaderWriter) Set(key, val string) {
|
||||
key = strings.ToLower(key)
|
||||
ppRW.ppMap[key] = val
|
||||
ppRW.PpMap[key] = val
|
||||
}
|
||||
|
||||
func (ppRW propertiesReaderWriter) ForeachKey(handler func(key, val string) error) error {
|
||||
for k, val := range ppRW.ppMap {
|
||||
func (ppRW PropertiesReaderWriter) ForeachKey(handler func(key, val string) error) error {
|
||||
for k, val := range ppRW.PpMap {
|
||||
if err := handler(k, val); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -214,20 +157,6 @@ func (ppRW propertiesReaderWriter) ForeachKey(handler func(key, val string) erro
|
|||
return nil
|
||||
}
|
||||
|
||||
func allowTrace(in interface{}) bool {
|
||||
if in == nil {
|
||||
return false
|
||||
}
|
||||
switch res := in.(type) {
|
||||
case msgstream.TsMsg:
|
||||
return !(res.Type() == commonpb.MsgType_TimeTick ||
|
||||
res.Type() == commonpb.MsgType_QueryNodeStats ||
|
||||
res.Type() == commonpb.MsgType_LoadIndex)
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
func noopSpan() opentracing.Span {
|
||||
func NoopSpan() opentracing.Span {
|
||||
return opentracing.NoopTracer{}.StartSpan("Default-span")
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue