enhance: move streaming proto into pkg (#35284)

issue: #33285

- move streaming related proto into pkg.
- add v2 message type and change flush message into v2 message.

Signed-off-by: chyezh <chyezh@outlook.com>
pull/35338/head
chyezh 2024-08-07 10:34:16 +08:00 committed by GitHub
parent b5022b0647
commit c725416288
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
106 changed files with 586 additions and 770 deletions

View File

@ -40,17 +40,6 @@ packages:
google.golang.org/grpc:
interfaces:
ClientStream:
github.com/milvus-io/milvus/internal/proto/streamingpb:
interfaces:
StreamingNodeHandlerService_ConsumeServer:
StreamingNodeHandlerService_ProduceServer:
StreamingCoordAssignmentServiceClient:
StreamingCoordAssignmentService_AssignmentDiscoverClient:
StreamingCoordAssignmentService_AssignmentDiscoverServer:
StreamingNodeManagerServiceClient:
StreamingNodeHandlerServiceClient:
StreamingNodeHandlerService_ConsumeClient:
StreamingNodeHandlerService_ProduceClient:
github.com/milvus-io/milvus/internal/streamingnode/server/walmanager:
interfaces:
Manager:

View File

@ -75,7 +75,7 @@ func (w *walAccesserImpl) dispatchByPChannel(ctx context.Context, msgs ...messag
pchannel := pchannel
msgs := msgs
idxes := indexes[pchannel]
go func() {
w.appendExecutionPool.Submit(func() (struct{}, error) {
defer wg.Done()
singleResp := w.appendToPChannel(ctx, pchannel, msgs...)
mu.Lock()
@ -83,7 +83,8 @@ func (w *walAccesserImpl) dispatchByPChannel(ctx context.Context, msgs ...messag
resp.fillResponseAtIdx(singleResp.Responses[i], idx)
}
mu.Unlock()
}()
return struct{}{}, nil
})
}
wg.Wait()
return resp
@ -137,7 +138,7 @@ func (w *walAccesserImpl) appendToPChannel(ctx context.Context, pchannel string,
for i, msg := range msgs {
i := i
msg := msg
go func() {
w.appendExecutionPool.Submit(func() (struct{}, error) {
defer wg.Done()
msgID, err := p.Produce(ctx, msg)
@ -147,7 +148,8 @@ func (w *walAccesserImpl) appendToPChannel(ctx context.Context, pchannel string,
Error: err,
}, i)
mu.Unlock()
}()
return struct{}{}, nil
})
}
wg.Wait()
return resp

View File

@ -13,6 +13,7 @@ import (
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/util/conc"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/lifetime"
)
@ -29,6 +30,8 @@ func newWALAccesser(c *clientv3.Client) *walAccesserImpl {
handlerClient: handlerClient,
producerMutex: sync.Mutex{},
producers: make(map[string]*producer.ResumableProducer),
// TODO: make the pool size configurable.
appendExecutionPool: conc.NewPool[struct{}](10),
}
}
@ -40,8 +43,9 @@ type walAccesserImpl struct {
streamingCoordAssignmentClient client.Client
handlerClient handler.HandlerClient
producerMutex sync.Mutex
producers map[string]*producer.ResumableProducer
producerMutex sync.Mutex
producers map[string]*producer.ResumableProducer
appendExecutionPool *conc.Pool[struct{}]
}
// Append writes a record to the log.

View File

@ -9,7 +9,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

View File

@ -7,8 +7,8 @@ import (
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/kv"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/etcd"
)

View File

@ -8,8 +8,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/mocks/mock_kv"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestCatalog(t *testing.T) {

View File

@ -9,8 +9,8 @@ import (
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/kv"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/etcd"
)

View File

@ -9,7 +9,7 @@ import (
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv/mocks"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestCatalog(t *testing.T) {

View File

@ -55,39 +55,6 @@ func (_c *MockClientConn_NewAddress_Call) RunAndReturn(run func([]resolver.Addre
return _c
}
// NewServiceConfig provides a mock function with given fields: serviceConfig
func (_m *MockClientConn) NewServiceConfig(serviceConfig string) {
_m.Called(serviceConfig)
}
// MockClientConn_NewServiceConfig_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'NewServiceConfig'
type MockClientConn_NewServiceConfig_Call struct {
*mock.Call
}
// NewServiceConfig is a helper method to define mock.On call
// - serviceConfig string
func (_e *MockClientConn_Expecter) NewServiceConfig(serviceConfig interface{}) *MockClientConn_NewServiceConfig_Call {
return &MockClientConn_NewServiceConfig_Call{Call: _e.mock.On("NewServiceConfig", serviceConfig)}
}
func (_c *MockClientConn_NewServiceConfig_Call) Run(run func(serviceConfig string)) *MockClientConn_NewServiceConfig_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(string))
})
return _c
}
func (_c *MockClientConn_NewServiceConfig_Call) Return() *MockClientConn_NewServiceConfig_Call {
_c.Call.Return()
return _c
}
func (_c *MockClientConn_NewServiceConfig_Call) RunAndReturn(run func(string)) *MockClientConn_NewServiceConfig_Call {
_c.Call.Return(run)
return _c
}
// ParseServiceConfig provides a mock function with given fields: serviceConfigJSON
func (_m *MockClientConn) ParseServiceConfig(serviceConfigJSON string) *serviceconfig.ParseResult {
ret := _m.Called(serviceConfigJSON)

View File

@ -7,7 +7,7 @@ import (
mock "github.com/stretchr/testify/mock"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingCoordCataLog is an autogenerated mock type for the StreamingCoordCataLog type

View File

@ -7,7 +7,7 @@ import (
mock "github.com/stretchr/testify/mock"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingNodeCataLog is an autogenerated mock type for the StreamingNodeCataLog type

View File

@ -40,8 +40,8 @@ type MockFlusher_RegisterPChannel_Call struct {
}
// RegisterPChannel is a helper method to define mock.On call
// - pchannel string
// - w wal.WAL
// - pchannel string
// - w wal.WAL
func (_e *MockFlusher_Expecter) RegisterPChannel(pchannel interface{}, w interface{}) *MockFlusher_RegisterPChannel_Call {
return &MockFlusher_RegisterPChannel_Call{Call: _e.mock.On("RegisterPChannel", pchannel, w)}
}
@ -74,8 +74,8 @@ type MockFlusher_RegisterVChannel_Call struct {
}
// RegisterVChannel is a helper method to define mock.On call
// - vchannel string
// - _a1 wal.WAL
// - vchannel string
// - _a1 wal.WAL
func (_e *MockFlusher_Expecter) RegisterVChannel(vchannel interface{}, _a1 interface{}) *MockFlusher_RegisterVChannel_Call {
return &MockFlusher_RegisterVChannel_Call{Call: _e.mock.On("RegisterVChannel", vchannel, _a1)}
}
@ -172,7 +172,7 @@ type MockFlusher_UnregisterPChannel_Call struct {
}
// UnregisterPChannel is a helper method to define mock.On call
// - pchannel string
// - pchannel string
func (_e *MockFlusher_Expecter) UnregisterPChannel(pchannel interface{}) *MockFlusher_UnregisterPChannel_Call {
return &MockFlusher_UnregisterPChannel_Call{Call: _e.mock.On("UnregisterPChannel", pchannel)}
}
@ -205,7 +205,7 @@ type MockFlusher_UnregisterVChannel_Call struct {
}
// UnregisterVChannel is a helper method to define mock.On call
// - vchannel string
// - vchannel string
func (_e *MockFlusher_Expecter) UnregisterVChannel(vchannel interface{}) *MockFlusher_UnregisterVChannel_Call {
return &MockFlusher_UnregisterVChannel_Call{Call: _e.mock.On("UnregisterVChannel", vchannel)}
}

View File

@ -8,10 +8,10 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/lifetime"
"github.com/milvus-io/milvus/pkg/util/syncutil"

View File

@ -10,10 +10,10 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

View File

@ -4,10 +4,9 @@ import (
"io"
"sync"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/lifetime"
"github.com/milvus-io/milvus/pkg/util/typeutil"
@ -51,7 +50,7 @@ func (c *assignmentDiscoverClient) ReportAssignmentError(pchannel types.PChannel
case c.requestCh <- &streamingpb.AssignmentDiscoverRequest{
Command: &streamingpb.AssignmentDiscoverRequest_ReportError{
ReportError: &streamingpb.ReportAssignmentErrorRequest{
Pchannel: typeconverter.NewProtoFromPChannelInfo(pchannel),
Pchannel: types.NewProtoFromPChannelInfo(pchannel),
Err: statusErr,
},
},
@ -136,10 +135,10 @@ func (c *assignmentDiscoverClient) recvLoop() (err error) {
for _, assignment := range resp.FullAssignment.Assignments {
channels := make(map[string]types.PChannelInfo, len(assignment.Channels))
for _, channel := range assignment.Channels {
channels[channel.Name] = typeconverter.NewPChannelInfoFromProto(channel)
channels[channel.Name] = types.NewPChannelInfoFromProto(channel)
}
newIncomingAssignments[assignment.GetNode().GetServerId()] = types.StreamingNodeAssignment{
NodeInfo: typeconverter.NewStreamingNodeInfoFromProto(assignment.Node),
NodeInfo: types.NewStreamingNodeInfoFromProto(assignment.Node),
Channels: channels,
}
}

View File

@ -10,13 +10,13 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingcoord/client/assignment"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/balancer/picker"
streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/tracer"
"github.com/milvus-io/milvus/pkg/util/interceptor"

View File

@ -10,10 +10,10 @@ import (
"github.com/milvus-io/milvus/internal/mocks/mock_metastore"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/client/mock_manager"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer"
_ "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/policy"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/typeutil"

View File

@ -6,9 +6,9 @@ import (
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/syncutil"

View File

@ -9,8 +9,8 @@ import (
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/mock_metastore"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingcoord/server/resource"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

View File

@ -3,8 +3,7 @@ package channel
import (
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)
@ -43,7 +42,7 @@ func (c *PChannelMeta) Name() string {
// ChannelInfo returns the channel info.
func (c *PChannelMeta) ChannelInfo() types.PChannelInfo {
return typeconverter.NewPChannelInfoFromProto(c.inner.Channel)
return types.NewPChannelInfoFromProto(c.inner.Channel)
}
// Term returns the current term of the channel.
@ -60,8 +59,8 @@ func (c *PChannelMeta) CurrentServerID() int64 {
// CurrentAssignment returns the current assignment of the channel.
func (c *PChannelMeta) CurrentAssignment() types.PChannelInfoAssigned {
return types.PChannelInfoAssigned{
Channel: typeconverter.NewPChannelInfoFromProto(c.inner.Channel),
Node: typeconverter.NewStreamingNodeInfoFromProto(c.inner.Node),
Channel: types.NewPChannelInfoFromProto(c.inner.Channel),
Node: types.NewStreamingNodeInfoFromProto(c.inner.Node),
}
}
@ -74,7 +73,7 @@ func (c *PChannelMeta) AssignHistories() []types.PChannelInfoAssigned {
Name: c.inner.GetChannel().GetName(),
Term: h.Term,
},
Node: typeconverter.NewStreamingNodeInfoFromProto(h.Node),
Node: types.NewStreamingNodeInfoFromProto(h.Node),
})
}
return history
@ -122,7 +121,7 @@ func (m *mutablePChannel) TryAssignToServerID(streamingNode types.StreamingNodeI
// otherwise update the channel into assgining state.
m.inner.Channel.Term++
m.inner.Node = typeconverter.NewProtoFromStreamingNodeInfo(streamingNode)
m.inner.Node = types.NewProtoFromStreamingNodeInfo(streamingNode)
m.inner.State = streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNING
return true
}

View File

@ -5,7 +5,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)

View File

@ -6,7 +6,6 @@ import (
"go.uber.org/zap"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer"
_ "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/policy" // register the balancer policy
"github.com/milvus-io/milvus/internal/streamingcoord/server/service"
@ -14,6 +13,7 @@ import (
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/util"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// Server is the streamingcoord server.

View File

@ -1,10 +1,10 @@
package service
import (
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer"
"github.com/milvus-io/milvus/internal/streamingcoord/server/service/discover"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)

View File

@ -1,8 +1,7 @@
package discover
import (
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
@ -18,12 +17,12 @@ func (h *discoverGrpcServerHelper) SendFullAssignment(v typeutil.VersionInt64Pai
for _, relation := range relations {
if assignmentsMap[relation.Node.ServerID] == nil {
assignmentsMap[relation.Node.ServerID] = &streamingpb.StreamingNodeAssignment{
Node: typeconverter.NewProtoFromStreamingNodeInfo(relation.Node),
Node: types.NewProtoFromStreamingNodeInfo(relation.Node),
Channels: make([]*streamingpb.PChannelInfo, 0),
}
}
assignmentsMap[relation.Node.ServerID].Channels = append(
assignmentsMap[relation.Node.ServerID].Channels, typeconverter.NewProtoFromPChannelInfo(relation.Channel))
assignmentsMap[relation.Node.ServerID].Channels, types.NewProtoFromPChannelInfo(relation.Channel))
}
assignments := make([]*streamingpb.StreamingNodeAssignment, 0, len(assignmentsMap))

View File

@ -7,10 +7,9 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)
@ -78,7 +77,7 @@ func (s *AssignmentDiscoverServer) recvLoop() (err error) {
}
switch req := req.Command.(type) {
case *streamingpb.AssignmentDiscoverRequest_ReportError:
channel := typeconverter.NewPChannelInfoFromProto(req.ReportError.GetPchannel())
channel := types.NewPChannelInfoFromProto(req.ReportError.GetPchannel())
// mark the channel as unavailable and trigger a recover right away.
s.balancer.MarkAsUnavailable(s.ctx, []types.PChannelInfo{channel})
case *streamingpb.AssignmentDiscoverRequest_Close:

View File

@ -7,9 +7,9 @@ import (
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_balancer"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

View File

@ -8,11 +8,10 @@ import (
"go.uber.org/zap"
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
@ -83,18 +82,10 @@ func createConsumeRequest(ctx context.Context, opts *ConsumerOptions) (context.C
// select server to consume.
ctx = contextutil.WithPickServerID(ctx, opts.Assignment.Node.ServerID)
// create the consumer request.
deliverPolicy, err := typeconverter.NewProtoFromDeliverPolicy(opts.DeliverPolicy)
if err != nil {
return nil, errors.Wrap(err, "at convert deliver policy")
}
deliverFilters, err := typeconverter.NewProtosFromDeliverFilters(opts.DeliverFilters)
if err != nil {
return nil, errors.Wrap(err, "at convert deliver filters")
}
return contextutil.WithCreateConsumer(ctx, &streamingpb.CreateConsumerRequest{
Pchannel: typeconverter.NewProtoFromPChannelInfo(opts.Assignment.Channel),
DeliverPolicy: deliverPolicy,
DeliverFilters: deliverFilters,
Pchannel: types.NewProtoFromPChannelInfo(opts.Assignment.Channel),
DeliverPolicy: opts.DeliverPolicy,
DeliverFilters: opts.DeliverFilters,
}), nil
}
@ -162,7 +153,7 @@ func (c *consumerImpl) recvLoop() (err error) {
}
switch resp := resp.Response.(type) {
case *streamingpb.ConsumeResponse_Consume:
msgID, err := message.UnmarshalMessageID(c.walName, resp.Consume.GetId().GetId())
msgID, err := message.UnmarshalMessageID(c.walName, resp.Consume.GetMessage().GetId().GetId())
if err != nil {
return err
}

View File

@ -8,8 +8,9 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
@ -64,10 +65,10 @@ func TestConsumer(t *testing.T) {
recvCh <- &streamingpb.ConsumeResponse{
Response: &streamingpb.ConsumeResponse_Consume{
Consume: &streamingpb.ConsumeMessageReponse{
Id: &streamingpb.MessageID{
Id: walimplstest.NewTestMessageID(1).Marshal(),
},
Message: &streamingpb.Message{
Message: &messagespb.ImmutableMessage{
Id: &messagespb.MessageID{
Id: walimplstest.NewTestMessageID(1).Marshal(),
},
Payload: []byte{},
Properties: make(map[string]string),
},

View File

@ -10,7 +10,6 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/assignment"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/producer"
@ -18,6 +17,7 @@ import (
streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/streaming/util/types"

View File

@ -8,7 +8,6 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/assignment"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/producer"
@ -17,6 +16,7 @@ import (
"github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/lifetime"
)

View File

@ -8,17 +8,17 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/client/handler/mock_assignment"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/client/handler/mock_consumer"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/client/handler/mock_producer"
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_resolver"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer"
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/producer"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_types"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/streaming/util/types"

View File

@ -1,7 +1,8 @@
package producer
import (
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
@ -16,7 +17,7 @@ func (p *produceGrpcClient) SendProduceMessage(requestID int64, msg message.Muta
Request: &streamingpb.ProduceRequest_Produce{
Produce: &streamingpb.ProduceMessageRequest{
RequestId: requestID,
Message: &streamingpb.Message{
Message: &messagespb.Message{
Payload: msg.Payload(),
Properties: msg.Properties().ToRawMap(),
},

View File

@ -9,11 +9,10 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/lifetime"
@ -84,7 +83,7 @@ func createProduceRequest(ctx context.Context, opts *ProducerOptions) context.Co
ctx = contextutil.WithPickServerID(ctx, opts.Assignment.Node.ServerID)
// select channel to consume.
return contextutil.WithCreateProducer(ctx, &streamingpb.CreateProducerRequest{
Pchannel: typeconverter.NewProtoFromPChannelInfo(opts.Assignment.Channel),
Pchannel: types.NewProtoFromPChannelInfo(opts.Assignment.Channel),
})
}

View File

@ -9,8 +9,9 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest"
@ -88,7 +89,7 @@ func TestProducer(t *testing.T) {
RequestId: 2,
Response: &streamingpb.ProduceMessageResponse_Result{
Result: &streamingpb.ProduceMessageResponseResult{
Id: &streamingpb.MessageID{Id: walimplstest.NewTestMessageID(1).Marshal()},
Id: &messagespb.MessageID{Id: walimplstest.NewTestMessageID(1).Marshal()},
},
},
},

View File

@ -10,12 +10,12 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/balancer/picker"
streamingserviceinterceptor "github.com/milvus-io/milvus/internal/util/streamingutil/service/interceptor"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/tracer"
"github.com/milvus-io/milvus/pkg/util/interceptor"

View File

@ -7,15 +7,14 @@ import (
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/balancer/picker"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/discoverer"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/lifetime"
)
@ -143,7 +142,7 @@ func (c *managerClientImpl) Assign(ctx context.Context, pchannel types.PChannelI
// Select a log node to assign the wal instance.
ctx = contextutil.WithPickServerID(ctx, pchannel.Node.ServerID)
_, err = manager.Assign(ctx, &streamingpb.StreamingNodeManagerAssignRequest{
Pchannel: typeconverter.NewProtoFromPChannelInfo(pchannel.Channel),
Pchannel: types.NewProtoFromPChannelInfo(pchannel.Channel),
})
return err
}
@ -164,7 +163,7 @@ func (c *managerClientImpl) Remove(ctx context.Context, pchannel types.PChannelI
// Select a streaming node to remove the wal instance.
ctx = contextutil.WithPickServerID(ctx, pchannel.Node.ServerID)
_, err = manager.Remove(ctx, &streamingpb.StreamingNodeManagerRemoveRequest{
Pchannel: typeconverter.NewProtoFromPChannelInfo(pchannel.Channel),
Pchannel: types.NewProtoFromPChannelInfo(pchannel.Channel),
})
// The following error can be treated as success.
// 1. err is nil, a real remove operation at streaming node has been happened.

View File

@ -11,14 +11,14 @@ import (
"google.golang.org/grpc"
"google.golang.org/grpc/resolver"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_lazygrpc"
"github.com/milvus-io/milvus/internal/mocks/util/streamingutil/service/mock_resolver"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/attributes"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/discoverer"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/etcd"
"github.com/milvus-io/milvus/pkg/util/lifetime"

View File

@ -7,12 +7,12 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/service"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/internal/util/componentutil"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
_ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/pulsar"
_ "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq"
)

View File

@ -1,11 +1,11 @@
package service
import (
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/service/handler/consumer"
"github.com/milvus-io/milvus/internal/streamingnode/server/service/handler/producer"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)

View File

@ -1,6 +1,6 @@
package consumer
import "github.com/milvus-io/milvus/internal/proto/streamingpb"
import "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
// consumeGrpcServerHelper is a wrapped consumer server of log messages.
type consumeGrpcServerHelper struct {

View File

@ -7,15 +7,16 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
@ -31,22 +32,17 @@ func CreateConsumeServer(walManager walmanager.Manager, streamServer streamingpb
if err != nil {
return nil, status.NewInvaildArgument("create consumer request is required")
}
l, err := walManager.GetAvailableWAL(typeconverter.NewPChannelInfoFromProto(createReq.GetPchannel()))
l, err := walManager.GetAvailableWAL(types.NewPChannelInfoFromProto(createReq.GetPchannel()))
if err != nil {
return nil, err
}
deliverPolicy, err := typeconverter.NewDeliverPolicyFromProto(l.WALName(), createReq.GetDeliverPolicy())
filter, err := options.GetFilterFunc(createReq.DeliverFilters)
if err != nil {
return nil, status.NewInvaildArgument("at convert deliver policy, err: %s", err.Error())
}
deliverFilters, err := newMessageFilter(createReq.DeliverFilters)
if err != nil {
return nil, status.NewInvaildArgument("at convert deliver filters, err: %s", err.Error())
return nil, err
}
scanner, err := l.Read(streamServer.Context(), wal.ReadOption{
DeliverPolicy: deliverPolicy,
MessageFilter: deliverFilters,
DeliverPolicy: createReq.GetDeliverPolicy(),
MessageFilter: filter,
})
if err != nil {
return nil, err
@ -117,10 +113,10 @@ func (c *ConsumeServer) sendLoop() (err error) {
// Send Consumed message to client and do metrics.
messageSize := msg.EstimateSize()
if err := c.consumeServer.SendConsumeMessage(&streamingpb.ConsumeMessageReponse{
Id: &streamingpb.MessageID{
Id: msg.MessageID().Marshal(),
},
Message: &streamingpb.Message{
Message: &messagespb.ImmutableMessage{
Id: &messagespb.MessageID{
Id: msg.MessageID().Marshal(),
},
Payload: msg.Payload(),
Properties: msg.Properties().ToRawMap(),
},
@ -174,18 +170,3 @@ func (c *ConsumeServer) recvLoop() (err error) {
}
}
}
func newMessageFilter(filters []*streamingpb.DeliverFilter) (wal.MessageFilter, error) {
fs, err := typeconverter.NewDeliverFiltersFromProtos(filters)
if err != nil {
return nil, err
}
return func(msg message.ImmutableMessage) bool {
for _, f := range fs {
if !f.Filter(msg) {
return false
}
}
return true
}, nil
}

View File

@ -11,16 +11,17 @@ import (
"github.com/stretchr/testify/mock"
"google.golang.org/grpc/metadata"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_walmanager"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest"
"github.com/milvus-io/milvus/pkg/util/paramtable"
@ -48,7 +49,7 @@ func TestNewMessageFilter(t *testing.T) {
},
},
}
filterFunc, err := newMessageFilter(filters)
filterFunc, err := options.GetFilterFunc(filters)
assert.NoError(t, err)
msg := mock_message.NewMockImmutableMessage(t)
@ -82,7 +83,7 @@ func TestNewMessageFilter(t *testing.T) {
},
},
}
filterFunc, err = newMessageFilter(filters)
filterFunc, err = options.GetFilterFunc(filters)
assert.NoError(t, err)
msg = mock_message.NewMockImmutableMessage(t)

View File

@ -1,7 +1,7 @@
package producer
import (
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// produceGrpcServerHelper is a wrapped producer server of log messages.

View File

@ -9,15 +9,16 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
@ -33,7 +34,7 @@ func CreateProduceServer(walManager walmanager.Manager, streamServer streamingpb
if err != nil {
return nil, status.NewInvaildArgument("create producer request is required")
}
l, err := walManager.GetAvailableWAL(typeconverter.NewPChannelInfoFromProto(createReq.GetPchannel()))
l, err := walManager.GetAvailableWAL(types.NewPChannelInfoFromProto(createReq.GetPchannel()))
if err != nil {
return nil, err
}
@ -125,9 +126,10 @@ func (p *ProduceServer) sendLoop() (err error) {
func (p *ProduceServer) getWaitAppendChan() <-chan struct{} {
ch := make(chan struct{})
go func() {
p.appendWG.Wait()
close(ch)
}()
return p.wal.Available()
return ch
}
// recvLoop receives the message from client.
@ -222,7 +224,7 @@ func (p *ProduceServer) sendProduceResult(reqID int64, id message.MessageID, err
} else {
resp.Response = &streamingpb.ProduceMessageResponse_Result{
Result: &streamingpb.ProduceMessageResponseResult{
Id: &streamingpb.MessageID{
Id: &messagespb.MessageID{
Id: id.Marshal(),
},
},

View File

@ -14,13 +14,14 @@ import (
"go.uber.org/atomic"
"google.golang.org/grpc/metadata"
"github.com/milvus-io/milvus/internal/mocks/proto/mock_streamingpb"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_walmanager"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/internal/util/streamingutil/service/contextutil"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/mocks/streaming/proto/mock_streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest"
@ -114,7 +115,7 @@ func TestProduceSendArm(t *testing.T) {
RequestId: 1,
Response: &streamingpb.ProduceMessageResponse_Result{
Result: &streamingpb.ProduceMessageResponseResult{
Id: &streamingpb.MessageID{
Id: &messagespb.MessageID{
Id: walimplstest.NewTestMessageID(1).Marshal(),
},
},
@ -147,7 +148,7 @@ func TestProduceSendArm(t *testing.T) {
RequestId: 1,
Response: &streamingpb.ProduceMessageResponse_Result{
Result: &streamingpb.ProduceMessageResponseResult{
Id: &streamingpb.MessageID{
Id: &messagespb.MessageID{
Id: walimplstest.NewTestMessageID(1).Marshal(),
},
},
@ -219,7 +220,7 @@ func TestProduceServerRecvArm(t *testing.T) {
Request: &streamingpb.ProduceRequest_Produce{
Produce: &streamingpb.ProduceMessageRequest{
RequestId: 1,
Message: &streamingpb.Message{
Message: &messagespb.Message{
Payload: []byte("test"),
Properties: map[string]string{
"_v": "1",

View File

@ -3,9 +3,9 @@ package service
import (
"context"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/internal/util/streamingutil/typeconverter"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)
var _ ManagerService = (*managerServiceImpl)(nil)
@ -31,7 +31,7 @@ type managerServiceImpl struct {
// Assign assigns a wal instance for the channel on this Manager.
// After assign returns, the wal instance is ready to use.
func (ms *managerServiceImpl) Assign(ctx context.Context, req *streamingpb.StreamingNodeManagerAssignRequest) (*streamingpb.StreamingNodeManagerAssignResponse, error) {
pchannelInfo := typeconverter.NewPChannelInfoFromProto(req.GetPchannel())
pchannelInfo := types.NewPChannelInfoFromProto(req.GetPchannel())
if err := ms.walManager.Open(ctx, pchannelInfo); err != nil {
return nil, err
}
@ -41,7 +41,7 @@ func (ms *managerServiceImpl) Assign(ctx context.Context, req *streamingpb.Strea
// Remove removes the wal instance for the channel.
// After remove returns, the wal instance is removed and all underlying read write operation should be rejected.
func (ms *managerServiceImpl) Remove(ctx context.Context, req *streamingpb.StreamingNodeManagerRemoveRequest) (*streamingpb.StreamingNodeManagerRemoveResponse, error) {
pchannelInfo := typeconverter.NewPChannelInfoFromProto(req.GetPchannel())
pchannelInfo := types.NewPChannelInfoFromProto(req.GetPchannel())
if err := ms.walManager.Remove(ctx, pchannelInfo); err != nil {
return nil, err
}

View File

@ -11,11 +11,11 @@ import (
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/wal/mock_interceptors"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/mocks/streaming/mock_walimpls"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/streaming/walimpls"

View File

@ -8,11 +8,11 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/policy"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/merr"
)

View File

@ -7,8 +7,8 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

View File

@ -7,11 +7,11 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/lifetime"
"github.com/milvus-io/milvus/pkg/util/merr"

View File

@ -14,11 +14,11 @@ import (
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"

View File

@ -7,9 +7,9 @@ import (
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util/syncutil"
)
@ -181,10 +181,10 @@ func (m *sealQueue) sendFlushMessageIntoWAL(ctx context.Context, collectionID in
// createNewFlushMessage creates a new flush message.
func (m *sealQueue) createNewFlushMessage(collectionID int64, vchannel string, segmentIDs []int64) (message.MutableMessage, error) {
// Create a flush message.
msg, err := message.NewFlushMessageBuilderV1().
msg, err := message.NewFlushMessageBuilderV2().
WithVChannel(vchannel).
WithHeader(&message.FlushMessageHeader{}).
WithBody(&message.FlushMessagePayload{
WithBody(&message.FlushMessageBody{
CollectionId: collectionID,
SegmentId: segmentIDs,
}).BuildMutable()

View File

@ -8,11 +8,11 @@ import (
"go.uber.org/zap"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/policy"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)

View File

@ -3,7 +3,7 @@ package stats
import (
"time"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// SegmentStats is the usage stats of a segment.

View File

@ -10,10 +10,10 @@ import (
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)

View File

@ -8,8 +8,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)

View File

@ -9,7 +9,7 @@ import (
"google.golang.org/grpc/metadata"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
const (

View File

@ -8,7 +8,7 @@ import (
"github.com/stretchr/testify/assert"
"google.golang.org/grpc/metadata"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestWithCreateConsumer(t *testing.T) {

View File

@ -9,7 +9,7 @@ import (
"google.golang.org/grpc/metadata"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
const (

View File

@ -8,7 +8,7 @@ import (
"github.com/stretchr/testify/assert"
"google.golang.org/grpc/metadata"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestWithCreateProducer(t *testing.T) {

View File

@ -6,8 +6,8 @@ import (
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// NewStreamingServiceUnaryClientInterceptor returns a new unary client interceptor for error handling.

View File

@ -6,8 +6,8 @@ import (
"google.golang.org/grpc"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/internal/util/streamingutil/status"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// NewStreamingServiceUnaryServerInterceptor returns a new unary server interceptor for error handling, metric...

View File

@ -8,7 +8,7 @@ import (
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/mocks/google.golang.org/mock_grpc"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestClientStreamWrapper(t *testing.T) {

View File

@ -9,7 +9,7 @@ import (
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
var streamingErrorToGRPCStatus = map[streamingpb.StreamingCode]codes.Code{

View File

@ -8,7 +8,7 @@ import (
"github.com/stretchr/testify/assert"
"google.golang.org/grpc/codes"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestStreamingStatus(t *testing.T) {

View File

@ -6,7 +6,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
var _ error = (*StreamingError)(nil)

View File

@ -5,7 +5,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestStreamingError(t *testing.T) {

View File

@ -1,137 +0,0 @@
package typeconverter
import (
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
)
// NewDeliverPolicyFromProto converts protobuf DeliverPolicy to DeliverPolicy
func NewDeliverPolicyFromProto(name string, policy *streamingpb.DeliverPolicy) (options.DeliverPolicy, error) {
switch policy := policy.GetPolicy().(type) {
case *streamingpb.DeliverPolicy_All:
return options.DeliverPolicyAll(), nil
case *streamingpb.DeliverPolicy_Latest:
return options.DeliverPolicyLatest(), nil
case *streamingpb.DeliverPolicy_StartFrom:
msgID, err := message.UnmarshalMessageID(name, policy.StartFrom.GetId())
if err != nil {
return nil, err
}
return options.DeliverPolicyStartFrom(msgID), nil
case *streamingpb.DeliverPolicy_StartAfter:
msgID, err := message.UnmarshalMessageID(name, policy.StartAfter.GetId())
if err != nil {
return nil, err
}
return options.DeliverPolicyStartAfter(msgID), nil
default:
return nil, errors.New("unknown deliver policy")
}
}
// NewProtoFromDeliverPolicy converts DeliverPolicy to protobuf DeliverPolicy
func NewProtoFromDeliverPolicy(policy options.DeliverPolicy) (*streamingpb.DeliverPolicy, error) {
switch policy.Policy() {
case options.DeliverPolicyTypeAll:
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_All{},
}, nil
case options.DeliverPolicyTypeLatest:
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_Latest{},
}, nil
case options.DeliverPolicyTypeStartFrom:
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_StartFrom{
StartFrom: &streamingpb.MessageID{
Id: policy.MessageID().Marshal(),
},
},
}, nil
case options.DeliverPolicyTypeStartAfter:
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_StartAfter{
StartAfter: &streamingpb.MessageID{
Id: policy.MessageID().Marshal(),
},
},
}, nil
default:
return nil, errors.New("unknown deliver policy")
}
}
// NewProtosFromDeliverFilters converts DeliverFilter to protobuf DeliverFilter
func NewProtosFromDeliverFilters(filter []options.DeliverFilter) ([]*streamingpb.DeliverFilter, error) {
protos := make([]*streamingpb.DeliverFilter, 0, len(filter))
for _, f := range filter {
proto, err := NewProtoFromDeliverFilter(f)
if err != nil {
return nil, err
}
protos = append(protos, proto)
}
return protos, nil
}
// NewProtoFromDeliverFilter converts DeliverFilter to protobuf DeliverFilter
func NewProtoFromDeliverFilter(filter options.DeliverFilter) (*streamingpb.DeliverFilter, error) {
switch filter.Type() {
case options.DeliverFilterTypeTimeTickGT:
return &streamingpb.DeliverFilter{
Filter: &streamingpb.DeliverFilter_TimeTickGt{
TimeTickGt: &streamingpb.DeliverFilterTimeTickGT{
TimeTick: filter.(interface{ TimeTick() uint64 }).TimeTick(),
},
},
}, nil
case options.DeliverFilterTypeTimeTickGTE:
return &streamingpb.DeliverFilter{
Filter: &streamingpb.DeliverFilter_TimeTickGte{
TimeTickGte: &streamingpb.DeliverFilterTimeTickGTE{
TimeTick: filter.(interface{ TimeTick() uint64 }).TimeTick(),
},
},
}, nil
case options.DeliverFilterTypeVChannel:
return &streamingpb.DeliverFilter{
Filter: &streamingpb.DeliverFilter_Vchannel{
Vchannel: &streamingpb.DeliverFilterVChannel{
Vchannel: filter.(interface{ VChannel() string }).VChannel(),
},
},
}, nil
default:
return nil, errors.New("unknown deliver filter")
}
}
// NewDeliverFiltersFromProtos converts protobuf DeliverFilter to DeliverFilter
func NewDeliverFiltersFromProtos(protos []*streamingpb.DeliverFilter) ([]options.DeliverFilter, error) {
filters := make([]options.DeliverFilter, 0, len(protos))
for _, p := range protos {
f, err := NewDeliverFilterFromProto(p)
if err != nil {
return nil, err
}
filters = append(filters, f)
}
return filters, nil
}
// NewDeliverFilterFromProto converts protobuf DeliverFilter to DeliverFilter
func NewDeliverFilterFromProto(proto *streamingpb.DeliverFilter) (options.DeliverFilter, error) {
switch proto.Filter.(type) {
case *streamingpb.DeliverFilter_TimeTickGt:
return options.DeliverFilterTimeTickGT(proto.GetTimeTickGt().GetTimeTick()), nil
case *streamingpb.DeliverFilter_TimeTickGte:
return options.DeliverFilterTimeTickGTE(proto.GetTimeTickGte().GetTimeTick()), nil
case *streamingpb.DeliverFilter_Vchannel:
return options.DeliverFilterVChannel(proto.GetVchannel().GetVchannel()), nil
default:
return nil, errors.New("unknown deliver filter")
}
}

View File

@ -1,73 +0,0 @@
package typeconverter
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/options"
)
func TestDeliverFilter(t *testing.T) {
filters := []options.DeliverFilter{
options.DeliverFilterTimeTickGT(1),
options.DeliverFilterTimeTickGTE(2),
options.DeliverFilterVChannel("vchannel"),
}
pbFilters, err := NewProtosFromDeliverFilters(filters)
assert.NoError(t, err)
assert.Equal(t, len(filters), len(pbFilters))
filters2, err := NewDeliverFiltersFromProtos(pbFilters)
assert.NoError(t, err)
assert.Equal(t, len(filters), len(filters2))
for idx, filter := range filters {
filter2 := filters2[idx]
assert.Equal(t, filter.Type(), filter2.Type())
switch filter.Type() {
case options.DeliverFilterTypeTimeTickGT:
assert.Equal(t, filter.(interface{ TimeTick() uint64 }).TimeTick(), filter2.(interface{ TimeTick() uint64 }).TimeTick())
case options.DeliverFilterTypeTimeTickGTE:
assert.Equal(t, filter.(interface{ TimeTick() uint64 }).TimeTick(), filter2.(interface{ TimeTick() uint64 }).TimeTick())
case options.DeliverFilterTypeVChannel:
assert.Equal(t, filter.(interface{ VChannel() string }).VChannel(), filter2.(interface{ VChannel() string }).VChannel())
}
}
}
func TestDeliverPolicy(t *testing.T) {
policy := options.DeliverPolicyAll()
pbPolicy, err := NewProtoFromDeliverPolicy(policy)
assert.NoError(t, err)
policy2, err := NewDeliverPolicyFromProto("mock", pbPolicy)
assert.NoError(t, err)
assert.Equal(t, policy.Policy(), policy2.Policy())
policy = options.DeliverPolicyLatest()
pbPolicy, err = NewProtoFromDeliverPolicy(policy)
assert.NoError(t, err)
policy2, err = NewDeliverPolicyFromProto("mock", pbPolicy)
assert.NoError(t, err)
assert.Equal(t, policy.Policy(), policy2.Policy())
msgID := mock_message.NewMockMessageID(t)
msgID.EXPECT().Marshal().Return("mock")
message.RegisterMessageIDUnmsarshaler("mock", func(b string) (message.MessageID, error) {
return msgID, nil
})
policy = options.DeliverPolicyStartFrom(msgID)
pbPolicy, err = NewProtoFromDeliverPolicy(policy)
assert.NoError(t, err)
policy2, err = NewDeliverPolicyFromProto("mock", pbPolicy)
assert.NoError(t, err)
assert.Equal(t, policy.Policy(), policy2.Policy())
policy = options.DeliverPolicyStartAfter(msgID)
pbPolicy, err = NewProtoFromDeliverPolicy(policy)
assert.NoError(t, err)
policy2, err = NewDeliverPolicyFromProto("mock", pbPolicy)
assert.NoError(t, err)
assert.Equal(t, policy.Policy(), policy2.Policy())
}

View File

@ -1,34 +0,0 @@
package typeconverter
import (
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)
// NewPChannelInfoFromProto converts protobuf PChannelInfo to PChannelInfo
func NewPChannelInfoFromProto(pchannel *streamingpb.PChannelInfo) types.PChannelInfo {
if pchannel.GetName() == "" {
panic("pchannel name is empty")
}
if pchannel.GetTerm() <= 0 {
panic("pchannel term is empty or negetive")
}
return types.PChannelInfo{
Name: pchannel.GetName(),
Term: pchannel.GetTerm(),
}
}
// NewProtoFromPChannelInfo converts PChannelInfo to protobuf PChannelInfo
func NewProtoFromPChannelInfo(pchannel types.PChannelInfo) *streamingpb.PChannelInfo {
if pchannel.Name == "" {
panic("pchannel name is empty")
}
if pchannel.Term <= 0 {
panic("pchannel term is empty or negetive")
}
return &streamingpb.PChannelInfo{
Name: pchannel.Name,
Term: pchannel.Term,
}
}

View File

@ -1,20 +0,0 @@
package typeconverter
import (
"github.com/milvus-io/milvus/internal/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
)
func NewStreamingNodeInfoFromProto(proto *streamingpb.StreamingNodeInfo) types.StreamingNodeInfo {
return types.StreamingNodeInfo{
ServerID: proto.ServerId,
Address: proto.Address,
}
}
func NewProtoFromStreamingNodeInfo(info types.StreamingNodeInfo) *streamingpb.StreamingNodeInfo {
return &streamingpb.StreamingNodeInfo{
ServerId: info.ServerID,
Address: info.Address,
}
}

View File

@ -27,5 +27,15 @@ packages:
interfaces:
AssignmentDiscoverWatcher:
AssignmentRebalanceTrigger:
github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb:
interfaces:
StreamingNodeHandlerService_ConsumeServer:
StreamingNodeHandlerService_ProduceServer:
StreamingCoordAssignmentServiceClient:
StreamingCoordAssignmentService_AssignmentDiscoverClient:
StreamingCoordAssignmentService_AssignmentDiscoverServer:
StreamingNodeManagerServiceClient:
StreamingNodeHandlerServiceClient:
StreamingNodeHandlerService_ConsumeClient:
StreamingNodeHandlerService_ProduceClient:

View File

@ -18,6 +18,3 @@ generate-mockery: getdeps
$(INSTALL_PATH)/mockery --name=Client --dir=$(PWD)/mq/msgdispatcher --output=$(PWD)/mq/msgsdispatcher --filename=mock_client.go --with-expecter --structname=MockClient --outpkg=msgdispatcher --inpackage
$(INSTALL_PATH)/mockery --name=Logger --dir=$(PWD)/eventlog --output=$(PWD)/eventlog --filename=mock_logger.go --with-expecter --structname=MockLogger --outpkg=eventlog --inpackage
$(INSTALL_PATH)/mockery --name=MessageID --dir=$(PWD)/mq/msgstream/mqwrapper --output=$(PWD)/mq/msgstream/mqwrapper --filename=mock_id.go --with-expecter --structname=MockMessageID --outpkg=mqwrapper --inpackage
generate-proto:
$(ROOTPATH)/cmake_build/bin/protoc --proto_path=$(PWD)/streaming/util/message/messagepb --go_out=plugins=grpc,paths=source_relative:./streaming/util/message/messagepb $(PWD)/streaming/util/message/messagepb/message.proto

View File

@ -9,7 +9,7 @@ import (
mock "github.com/stretchr/testify/mock"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingCoordAssignmentServiceClient is an autogenerated mock type for the StreamingCoordAssignmentServiceClient type

View File

@ -8,7 +8,7 @@ import (
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingCoordAssignmentService_AssignmentDiscoverClient is an autogenerated mock type for the StreamingCoordAssignmentService_AssignmentDiscoverClient type

View File

@ -8,7 +8,7 @@ import (
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingCoordAssignmentService_AssignmentDiscoverServer is an autogenerated mock type for the StreamingCoordAssignmentService_AssignmentDiscoverServer type

View File

@ -9,7 +9,7 @@ import (
mock "github.com/stretchr/testify/mock"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingNodeHandlerServiceClient is an autogenerated mock type for the StreamingNodeHandlerServiceClient type

View File

@ -8,7 +8,7 @@ import (
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingNodeHandlerService_ConsumeClient is an autogenerated mock type for the StreamingNodeHandlerService_ConsumeClient type

View File

@ -8,7 +8,7 @@ import (
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingNodeHandlerService_ConsumeServer is an autogenerated mock type for the StreamingNodeHandlerService_ConsumeServer type

View File

@ -8,7 +8,7 @@ import (
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingNodeHandlerService_ProduceClient is an autogenerated mock type for the StreamingNodeHandlerService_ProduceClient type

View File

@ -8,7 +8,7 @@ import (
mock "github.com/stretchr/testify/mock"
metadata "google.golang.org/grpc/metadata"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingNodeHandlerService_ProduceServer is an autogenerated mock type for the StreamingNodeHandlerService_ProduceServer type

View File

@ -9,7 +9,7 @@ import (
mock "github.com/stretchr/testify/mock"
streamingpb "github.com/milvus-io/milvus/internal/proto/streamingpb"
streamingpb "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
// MockStreamingNodeManagerServiceClient is an autogenerated mock type for the StreamingNodeManagerServiceClient type

View File

@ -443,50 +443,6 @@ func (_c *MockMutableMessage_WithTimeTick_Call) RunAndReturn(run func(uint64) me
return _c
}
// WithVChannel provides a mock function with given fields: vChannel
func (_m *MockMutableMessage) WithVChannel(vChannel string) message.MutableMessage {
ret := _m.Called(vChannel)
var r0 message.MutableMessage
if rf, ok := ret.Get(0).(func(string) message.MutableMessage); ok {
r0 = rf(vChannel)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(message.MutableMessage)
}
}
return r0
}
// MockMutableMessage_WithVChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'WithVChannel'
type MockMutableMessage_WithVChannel_Call struct {
*mock.Call
}
// WithVChannel is a helper method to define mock.On call
// - vChannel string
func (_e *MockMutableMessage_Expecter) WithVChannel(vChannel interface{}) *MockMutableMessage_WithVChannel_Call {
return &MockMutableMessage_WithVChannel_Call{Call: _e.mock.On("WithVChannel", vChannel)}
}
func (_c *MockMutableMessage_WithVChannel_Call) Run(run func(vChannel string)) *MockMutableMessage_WithVChannel_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(string))
})
return _c
}
func (_c *MockMutableMessage_WithVChannel_Call) Return(_a0 message.MutableMessage) *MockMutableMessage_WithVChannel_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMutableMessage_WithVChannel_Call) RunAndReturn(run func(string) message.MutableMessage) *MockMutableMessage_WithVChannel_Call {
_c.Call.Return(run)
return _c
}
// NewMockMutableMessage creates a new instance of MockMutableMessage. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockMutableMessage(t interface {

View File

@ -1,8 +1,39 @@
syntax = "proto3";
package milvus.proto.message;
package milvus.proto.messages;
option go_package = "github.com/milvus-io/milvus/pkg/streaming/util/message/messagepb";
option go_package = "github.com/milvus-io/milvus/pkg/streaming/proto/messagespb";
// MessageID is the unique identifier of a message.
message MessageID {
string id = 1;
}
// Message is the basic unit of communication between publisher and consumer.
message Message {
bytes payload = 1; // message body
map<string, string> properties = 2; // message properties
}
// ImmutableMessage is the message that can not be modified anymore.
message ImmutableMessage {
MessageID id = 1;
bytes payload = 2; // message body
map<string, string> properties = 3; // message properties
}
// MessageType is the type of message.
enum MessageType {
Unknown = 0;
TimeTick = 1;
Insert = 2;
Delete = 3;
Flush = 4;
CreateCollection = 5;
DropCollection = 6;
CreatePartition = 7;
DropPartition = 8;
}
///
/// Message Payload Definitions

View File

@ -2,8 +2,9 @@ syntax = "proto3";
package milvus.proto.streaming;
option go_package = "github.com/milvus-io/milvus/internal/proto/streamingpb";
option go_package = "github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb";
import "messages.proto";
import "milvus.proto";
import "google/protobuf/empty.proto";
@ -11,17 +12,6 @@ import "google/protobuf/empty.proto";
// Common
//
// MessageID is the unique identifier of a message.
message MessageID {
string id = 1;
}
// Message is the basic unit of communication between publisher and consumer.
message Message {
bytes payload = 1; // message body
map<string, string> properties = 2; // message properties
}
// PChannelInfo is the information of a pchannel info, should only keep the
// basic info of a pchannel. It's used in many rpc and meta, so keep it simple.
message PChannelInfo {
@ -153,9 +143,9 @@ message DeliverPolicy {
oneof policy {
google.protobuf.Empty all = 1; // deliver all messages.
google.protobuf.Empty latest = 2; // deliver the latest message.
MessageID start_from =
messages.MessageID start_from =
3; // deliver message from this message id. [startFrom, ...]
MessageID start_after =
messages.MessageID start_after =
4; // deliver message after this message id. (startAfter, ...]
}
}
@ -253,8 +243,8 @@ message CreateProducerRequest {
// ProduceMessageRequest is the request of the Produce RPC.
message ProduceMessageRequest {
int64 request_id = 1; // request id for reply.
Message message = 2; // message to be sent.
int64 request_id = 1; // request id for reply.
messages.Message message = 2; // message to be sent.
}
// CloseProducerRequest is the request of the CloseProducer RPC.
@ -291,7 +281,8 @@ message ProduceMessageResponse {
// ProduceMessageResponseResult is the result of the produce message streaming
// RPC.
message ProduceMessageResponseResult {
MessageID id = 1; // the offset of the message in the channel
messages.MessageID id = 1; // the offset of the message in the channel.
uint64 timetick = 2; // the timetick of that message sent.
}
// CloseProducerResponse is the result of the CloseProducer RPC.
@ -331,8 +322,7 @@ message CreateConsumerResponse {
}
message ConsumeMessageReponse {
MessageID id = 1; // message id of message.
Message message = 2; // message to be consumed.
messages.ImmutableMessage message = 1;
}
message CloseConsumerResponse {}

View File

@ -3,7 +3,6 @@ package adaptor
import (
"github.com/cockroachdb/errors"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/pkg/mq/msgstream"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
@ -30,6 +29,8 @@ func NewMsgPackFromMessage(msgs ...message.ImmutableMessage) (*msgstream.MsgPack
tsMsg, err = fromMessageToTsMsgVOld(msg)
case message.VersionV1:
tsMsg, err = fromMessageToTsMsgV1(msg)
case message.VersionV2:
tsMsg, err = fromMessageToTsMsgV2(msg)
default:
panic("unsupported message version")
}
@ -63,7 +64,7 @@ func fromMessageToTsMsgVOld(msg message.ImmutableMessage) (msgstream.TsMsg, erro
// fromMessageToTsMsgV1 converts message to ts message.
func fromMessageToTsMsgV1(msg message.ImmutableMessage) (msgstream.TsMsg, error) {
tsMsg, err := unmashalerDispatcher.Unmarshal(msg.Payload(), commonpb.MsgType(msg.MessageType()))
tsMsg, err := unmashalerDispatcher.Unmarshal(msg.Payload(), MustGetCommonpbMsgTypeFromMessageType(msg.MessageType()))
if err != nil {
return nil, errors.Wrap(err, "Failed to unmarshal message")
}
@ -79,6 +80,30 @@ func fromMessageToTsMsgV1(msg message.ImmutableMessage) (msgstream.TsMsg, error)
return recoverMessageFromHeader(tsMsg, msg)
}
// fromMessageToTsMsgV2 converts message to ts message.
func fromMessageToTsMsgV2(msg message.ImmutableMessage) (msgstream.TsMsg, error) {
var tsMsg msgstream.TsMsg
var err error
switch msg.MessageType() {
case message.MessageTypeFlush:
tsMsg, err = NewFlushMessageBody(msg)
default:
panic("unsupported message type")
}
if err != nil {
return nil, err
}
tsMsg.SetTs(msg.TimeTick())
tsMsg.SetPosition(&msgpb.MsgPosition{
ChannelName: msg.VChannel(),
// from the last confirmed message id, you can read all messages which timetick is greater or equal than current message id.
MsgID: MustGetMQWrapperIDFromMessage(msg.LastConfirmedMessageID()).Serialize(),
MsgGroup: "", // Not important any more.
Timestamp: msg.TimeTick(),
})
return tsMsg, nil
}
// recoverMessageFromHeader recovers message from header.
func recoverMessageFromHeader(tsMsg msgstream.TsMsg, msg message.ImmutableMessage) (msgstream.TsMsg, error) {
switch msg.MessageType() {

View File

@ -0,0 +1,25 @@
package adaptor
import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
var messageTypeToCommonpbMsgType = map[message.MessageType]commonpb.MsgType{
message.MessageTypeTimeTick: commonpb.MsgType_TimeTick,
message.MessageTypeInsert: commonpb.MsgType_Insert,
message.MessageTypeDelete: commonpb.MsgType_Delete,
message.MessageTypeFlush: commonpb.MsgType_Flush,
message.MessageTypeCreateCollection: commonpb.MsgType_CreateCollection,
message.MessageTypeDropCollection: commonpb.MsgType_DropCollection,
message.MessageTypeCreatePartition: commonpb.MsgType_CreatePartition,
message.MessageTypeDropPartition: commonpb.MsgType_DropPartition,
}
// MustGetCommonpbMsgTypeFromMessageType returns the commonpb.MsgType from message.MessageType.
func MustGetCommonpbMsgTypeFromMessageType(t message.MessageType) commonpb.MsgType {
if v, ok := messageTypeToCommonpbMsgType[t]; ok {
return v
}
panic("unsupported message type")
}

View File

@ -0,0 +1,79 @@
package adaptor
import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/pkg/mq/msgstream"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
var (
_ msgstream.TsMsg = &tsMsgImpl{}
_ msgstream.TsMsg = &FlushMessageBody{}
)
type tsMsgImpl struct {
msgstream.BaseMsg
ts uint64
sz int
msgType commonpb.MsgType
}
func (t *tsMsgImpl) ID() msgstream.UniqueID {
panic("should never use")
}
func (t *tsMsgImpl) SetID(id msgstream.UniqueID) {
panic("should never use")
}
func (t *tsMsgImpl) Type() commonpb.MsgType {
return t.msgType
}
func (t *tsMsgImpl) SourceID() int64 {
panic("should never use")
}
func (t *tsMsgImpl) Marshal(msgstream.TsMsg) (msgstream.MarshalType, error) {
panic("should never use")
}
func (t *tsMsgImpl) Unmarshal(msgstream.MarshalType) (msgstream.TsMsg, error) {
panic("should never use")
}
func (t *tsMsgImpl) Size() int {
return t.sz
}
func (t *tsMsgImpl) SetTs(ts uint64) {
t.ts = ts
}
type FlushMessageBody struct {
*tsMsgImpl
*message.FlushMessageBody
}
func NewFlushMessageBody(msg message.ImmutableMessage) (msgstream.TsMsg, error) {
flushMsg, err := message.AsImmutableFlushMessageV2(msg)
if err != nil {
return nil, err
}
body, err := flushMsg.Body()
if err != nil {
return nil, err
}
return &FlushMessageBody{
tsMsgImpl: &tsMsgImpl{
BaseMsg: msgstream.BaseMsg{
BeginTimestamp: msg.TimeTick(),
EndTimestamp: msg.TimeTick(),
},
ts: msg.TimeTick(),
sz: msg.EstimateSize(),
msgType: commonpb.MsgType(msg.MessageType()),
},
FlushMessageBody: body,
}, nil
}

View File

@ -42,7 +42,7 @@ var (
NewDropCollectionMessageBuilderV1 = createNewMessageBuilderV1[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]()
NewCreatePartitionMessageBuilderV1 = createNewMessageBuilderV1[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]()
NewDropPartitionMessageBuilderV1 = createNewMessageBuilderV1[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]()
NewFlushMessageBuilderV1 = createNewMessageBuilderV1[*FlushMessageHeader, *FlushMessagePayload]()
NewFlushMessageBuilderV2 = createNewMessageBuilderV2[*FlushMessageHeader, *FlushMessageBody]()
)
// createNewMessageBuilderV1 creates a new message builder with v1 marker.
@ -52,6 +52,13 @@ func createNewMessageBuilderV1[H proto.Message, B proto.Message]() func() *mutab
}
}
// List all type-safe mutable message builders here.
func createNewMessageBuilderV2[H proto.Message, B proto.Message]() func() *mutableMesasgeBuilder[H, B] {
return func() *mutableMesasgeBuilder[H, B] {
return newMutableMessageBuilder[H, B](VersionV2)
}
}
// newMutableMessageBuilder creates a new builder.
// Should only used at client side.
func newMutableMessageBuilder[H proto.Message, B proto.Message](v Version) *mutableMesasgeBuilder[H, B] {

View File

@ -45,11 +45,11 @@ type MutableMessage interface {
BasicMessage
// WithLastConfirmed sets the last confirmed message id of current message.
// !!! preserved for streaming system internal usage, don't call it outside of log system.
// !!! preserved for streaming system internal usage, don't call it outside of streaming system.
WithLastConfirmed(id MessageID) MutableMessage
// WithTimeTick sets the time tick of current message.
// !!! preserved for streaming system internal usage, don't call it outside of log system.
// !!! preserved for streaming system internal usage, don't call it outside of streaming system.
WithTimeTick(tt uint64) MutableMessage
// IntoImmutableMessage converts the mutable message to immutable message.

View File

@ -32,7 +32,7 @@ func TestMessage(t *testing.T) {
assert.Equal(t, "value", v)
assert.True(t, ok)
assert.Equal(t, message.MessageTypeTimeTick, mutableMessage.MessageType())
assert.Equal(t, 35, mutableMessage.EstimateSize())
assert.Equal(t, 32, mutableMessage.EstimateSize())
mutableMessage.WithTimeTick(123)
v, ok = mutableMessage.Properties().Get("_tt")
assert.True(t, ok)
@ -67,7 +67,7 @@ func TestMessage(t *testing.T) {
[]byte("payload"),
map[string]string{
"key": "value",
"_t": "1200",
"_t": "1",
"_tt": message.EncodeUint64(456),
"_v": "1",
"_lc": "lcMsgID",
@ -80,7 +80,7 @@ func TestMessage(t *testing.T) {
assert.Equal(t, "value", v)
assert.True(t, ok)
assert.Equal(t, message.MessageTypeTimeTick, immutableMessage.MessageType())
assert.Equal(t, 39, immutableMessage.EstimateSize())
assert.Equal(t, 36, immutableMessage.EstimateSize())
assert.Equal(t, message.Version(1), immutableMessage.Version())
assert.Equal(t, uint64(456), immutableMessage.TimeTick())
assert.NotNil(t, immutableMessage.LastConfirmedMessageID())
@ -90,7 +90,7 @@ func TestMessage(t *testing.T) {
[]byte("payload"),
map[string]string{
"key": "value",
"_t": "1200",
"_t": "1",
})
assert.True(t, immutableMessage.MessageID().EQ(msgID))
@ -100,7 +100,7 @@ func TestMessage(t *testing.T) {
assert.Equal(t, "value", v)
assert.True(t, ok)
assert.Equal(t, message.MessageTypeTimeTick, immutableMessage.MessageType())
assert.Equal(t, 21, immutableMessage.EstimateSize())
assert.Equal(t, 18, immutableMessage.EstimateSize())
assert.Equal(t, message.Version(0), immutableMessage.Version())
assert.Panics(t, func() {
immutableMessage.TimeTick()

View File

@ -45,18 +45,27 @@ func (m *messageImpl) EstimateSize() int {
// WithVChannel sets the virtual channel of current message.
func (m *messageImpl) WithVChannel(vChannel string) MutableMessage {
if m.properties.Exist(messageVChannel) {
panic("vchannel already set in properties of message")
}
m.properties.Set(messageVChannel, vChannel)
return m
}
// WithTimeTick sets the time tick of current message.
func (m *messageImpl) WithTimeTick(tt uint64) MutableMessage {
if m.properties.Exist(messageTimeTick) {
panic("time tick already set in properties of message")
}
m.properties.Set(messageTimeTick, EncodeUint64(tt))
return m
}
// WithLastConfirmed sets the last confirmed message id of current message.
func (m *messageImpl) WithLastConfirmed(id MessageID) MutableMessage {
if m.properties.Exist(messageLastConfirmed) {
panic("last confirmed message already set in properties of message")
}
m.properties.Set(messageLastConfirmed, id.Marshal())
return m
}

View File

@ -3,21 +3,21 @@ package message
import (
"strconv"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
)
type MessageType int32
type MessageType messagespb.MessageType
const (
MessageTypeUnknown MessageType = MessageType(commonpb.MsgType_Undefined)
MessageTypeTimeTick MessageType = MessageType(commonpb.MsgType_TimeTick)
MessageTypeInsert MessageType = MessageType(commonpb.MsgType_Insert)
MessageTypeDelete MessageType = MessageType(commonpb.MsgType_Delete)
MessageTypeFlush MessageType = MessageType(commonpb.MsgType_Flush)
MessageTypeCreateCollection MessageType = MessageType(commonpb.MsgType_CreateCollection)
MessageTypeDropCollection MessageType = MessageType(commonpb.MsgType_DropCollection)
MessageTypeCreatePartition MessageType = MessageType(commonpb.MsgType_CreatePartition)
MessageTypeDropPartition MessageType = MessageType(commonpb.MsgType_DropPartition)
MessageTypeUnknown MessageType = MessageType(messagespb.MessageType_Unknown)
MessageTypeTimeTick MessageType = MessageType(messagespb.MessageType_TimeTick)
MessageTypeInsert MessageType = MessageType(messagespb.MessageType_Insert)
MessageTypeDelete MessageType = MessageType(messagespb.MessageType_Delete)
MessageTypeFlush MessageType = MessageType(messagespb.MessageType_Flush)
MessageTypeCreateCollection MessageType = MessageType(messagespb.MessageType_CreateCollection)
MessageTypeDropCollection MessageType = MessageType(messagespb.MessageType_DropCollection)
MessageTypeCreatePartition MessageType = MessageType(messagespb.MessageType_CreatePartition)
MessageTypeDropPartition MessageType = MessageType(messagespb.MessageType_DropPartition)
)
var messageTypeName = map[MessageType]string{

View File

@ -8,21 +8,21 @@ import (
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message/messagepb"
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
)
type (
SegmentAssignment = messagepb.SegmentAssignment
PartitionSegmentAssignment = messagepb.PartitionSegmentAssignment
TimeTickMessageHeader = messagepb.TimeTickMessageHeader
InsertMessageHeader = messagepb.InsertMessageHeader
DeleteMessageHeader = messagepb.DeleteMessageHeader
CreateCollectionMessageHeader = messagepb.CreateCollectionMessageHeader
DropCollectionMessageHeader = messagepb.DropCollectionMessageHeader
CreatePartitionMessageHeader = messagepb.CreatePartitionMessageHeader
DropPartitionMessageHeader = messagepb.DropPartitionMessageHeader
FlushMessageHeader = messagepb.FlushMessageHeader
FlushMessagePayload = messagepb.FlushMessageBody
SegmentAssignment = messagespb.SegmentAssignment
PartitionSegmentAssignment = messagespb.PartitionSegmentAssignment
TimeTickMessageHeader = messagespb.TimeTickMessageHeader
InsertMessageHeader = messagespb.InsertMessageHeader
DeleteMessageHeader = messagespb.DeleteMessageHeader
CreateCollectionMessageHeader = messagespb.CreateCollectionMessageHeader
DropCollectionMessageHeader = messagespb.DropCollectionMessageHeader
CreatePartitionMessageHeader = messagespb.CreatePartitionMessageHeader
DropPartitionMessageHeader = messagespb.DropPartitionMessageHeader
FlushMessageHeader = messagespb.FlushMessageHeader
FlushMessageBody = messagespb.FlushMessageBody
)
// messageTypeMap maps the proto message type to the message type.
@ -46,6 +46,7 @@ type (
MutableDropCollectionMessageV1 = specializedMutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
MutableCreatePartitionMessageV1 = specializedMutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
MutableDropPartitionMessageV1 = specializedMutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
MutableFlushMessageV2 = specializedMutableMessage[*FlushMessageHeader, *FlushMessageBody]
ImmutableTimeTickMessageV1 = specializedImmutableMessage[*TimeTickMessageHeader, *msgpb.TimeTickMsg]
ImmutableInsertMessageV1 = specializedImmutableMessage[*InsertMessageHeader, *msgpb.InsertRequest]
@ -54,6 +55,7 @@ type (
ImmutableDropCollectionMessageV1 = specializedImmutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
ImmutableCreatePartitionMessageV1 = specializedImmutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
ImmutableDropPartitionMessageV1 = specializedImmutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
ImmutableFlushMessageV2 = specializedImmutableMessage[*FlushMessageHeader, *FlushMessageBody]
)
// List all as functions for specialized messages.
@ -65,6 +67,7 @@ var (
AsMutableDropCollectionMessageV1 = asSpecializedMutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
AsMutableCreatePartitionMessageV1 = asSpecializedMutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
AsMutableDropPartitionMessageV1 = asSpecializedMutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
AsMutableFlushMessageV2 = asSpecializedMutableMessage[*FlushMessageHeader, *FlushMessageBody]
AsImmutableTimeTickMessageV1 = asSpecializedImmutableMessage[*TimeTickMessageHeader, *msgpb.TimeTickMsg]
AsImmutableInsertMessageV1 = asSpecializedImmutableMessage[*InsertMessageHeader, *msgpb.InsertRequest]
@ -73,6 +76,7 @@ var (
AsImmutableDropCollectionMessageV1 = asSpecializedImmutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
AsImmutableCreatePartitionMessageV1 = asSpecializedImmutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
AsImmutableDropPartitionMessageV1 = asSpecializedImmutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
AsImmutableFlushMessageV2 = asSpecializedImmutableMessage[*FlushMessageHeader, *FlushMessageBody]
)
// asSpecializedMutableMessage converts a MutableMessage to a specialized MutableMessage.

View File

@ -4,7 +4,8 @@ import "strconv"
var (
VersionOld Version = 0 // old version before streamingnode.
VersionV1 Version = 1
VersionV1 Version = 1 // The message marshal unmarshal still use msgstream.
VersionV2 Version = 2 // The message marshal unmsarhsl is not rely on msgstream.
)
type Version int // message version for compatibility.

View File

@ -1,6 +1,8 @@
package options
import (
"github.com/milvus-io/milvus/pkg/streaming/proto/messagespb"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
@ -20,49 +22,45 @@ type (
deliverFilterType int
)
// DeliverPolicy is the policy of delivering messages.
type DeliverPolicy interface {
Policy() deliverPolicyType
MessageID() message.MessageID
}
type DeliverPolicy = *streamingpb.DeliverPolicy
// DeliverPolicyAll delivers all messages.
func DeliverPolicyAll() DeliverPolicy {
return &deliverPolicyWithoutMessageID{
policy: DeliverPolicyTypeAll,
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_All{},
}
}
// DeliverLatest delivers the latest message.
func DeliverPolicyLatest() DeliverPolicy {
return &deliverPolicyWithoutMessageID{
policy: DeliverPolicyTypeLatest,
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_Latest{},
}
}
// DeliverEarliest delivers the earliest message.
func DeliverPolicyStartFrom(messageID message.MessageID) DeliverPolicy {
return &deliverPolicyWithMessageID{
policy: DeliverPolicyTypeStartFrom,
messageID: messageID,
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_StartFrom{
StartFrom: &messagespb.MessageID{
Id: messageID.Marshal(),
},
},
}
}
// DeliverPolicyStartAfter delivers the message after the specified message.
func DeliverPolicyStartAfter(messageID message.MessageID) DeliverPolicy {
return &deliverPolicyWithMessageID{
policy: DeliverPolicyTypeStartAfter,
messageID: messageID,
return &streamingpb.DeliverPolicy{
Policy: &streamingpb.DeliverPolicy_StartAfter{
StartAfter: &messagespb.MessageID{
Id: messageID.Marshal(),
},
},
}
}
// DeliverFilter is the filter of delivering messages.
type DeliverFilter interface {
Type() deliverFilterType
Filter(message.ImmutableMessage) bool
}
type DeliverFilter = *streamingpb.DeliverFilter
//
// DeliverFilters
@ -70,26 +68,75 @@ type DeliverFilter interface {
// DeliverFilterTimeTickGT delivers messages by time tick greater than the specified time tick.
func DeliverFilterTimeTickGT(timeTick uint64) DeliverFilter {
return &deliverFilterTimeTickGT{
timeTick: timeTick,
return &streamingpb.DeliverFilter{
Filter: &streamingpb.DeliverFilter_TimeTickGt{
TimeTickGt: &streamingpb.DeliverFilterTimeTickGT{
TimeTick: timeTick,
},
},
}
}
// DeliverFilterTimeTickGTE delivers messages by time tick greater than or equal to the specified time tick.
func DeliverFilterTimeTickGTE(timeTick uint64) DeliverFilter {
return &deliverFilterTimeTickGTE{
timeTick: timeTick,
return &streamingpb.DeliverFilter{
Filter: &streamingpb.DeliverFilter_TimeTickGte{
TimeTickGte: &streamingpb.DeliverFilterTimeTickGTE{
TimeTick: timeTick,
},
},
}
}
// DeliverFilterVChannel delivers messages filtered by vchannel.
func DeliverFilterVChannel(vchannel string) DeliverFilter {
return &deliverFilterVChannel{
vchannel: vchannel,
return &streamingpb.DeliverFilter{
Filter: &streamingpb.DeliverFilter_Vchannel{
Vchannel: &streamingpb.DeliverFilterVChannel{
Vchannel: vchannel,
},
},
}
}
// IsDeliverFilterTimeTick checks if the filter is time tick filter.
func IsDeliverFilterTimeTick(filter DeliverFilter) bool {
return filter.Type() == DeliverFilterTypeTimeTickGT || filter.Type() == DeliverFilterTypeTimeTickGTE
switch filter.GetFilter().(type) {
case *streamingpb.DeliverFilter_TimeTickGt, *streamingpb.DeliverFilter_TimeTickGte:
return true
default:
return false
}
}
// GetFilterFunc returns the filter function.
func GetFilterFunc(filters []DeliverFilter) (func(message.ImmutableMessage) bool, error) {
filterFuncs := make([]func(message.ImmutableMessage) bool, 0, len(filters))
for _, filter := range filters {
filter := filter
switch filter.GetFilter().(type) {
case *streamingpb.DeliverFilter_TimeTickGt:
filterFuncs = append(filterFuncs, func(im message.ImmutableMessage) bool {
return im.TimeTick() > filter.GetTimeTickGt().TimeTick
})
case *streamingpb.DeliverFilter_TimeTickGte:
filterFuncs = append(filterFuncs, func(im message.ImmutableMessage) bool {
return im.TimeTick() >= filter.GetTimeTickGte().TimeTick
})
case *streamingpb.DeliverFilter_Vchannel:
filterFuncs = append(filterFuncs, func(im message.ImmutableMessage) bool {
return im.VChannel() == filter.GetVchannel().Vchannel
})
default:
panic("unimplemented")
}
}
return func(msg message.ImmutableMessage) bool {
for _, f := range filterFuncs {
if !f(msg) {
return false
}
}
return true
}, nil
}

View File

@ -1,81 +0,0 @@
package options
import "github.com/milvus-io/milvus/pkg/streaming/util/message"
// deliverPolicyWithoutMessageID is the policy of delivering messages without messageID.
type deliverPolicyWithoutMessageID struct {
policy deliverPolicyType
}
func (d *deliverPolicyWithoutMessageID) Policy() deliverPolicyType {
return d.policy
}
func (d *deliverPolicyWithoutMessageID) MessageID() message.MessageID {
panic("not implemented")
}
// deliverPolicyWithMessageID is the policy of delivering messages with messageID.
type deliverPolicyWithMessageID struct {
policy deliverPolicyType
messageID message.MessageID
}
func (d *deliverPolicyWithMessageID) Policy() deliverPolicyType {
return d.policy
}
func (d *deliverPolicyWithMessageID) MessageID() message.MessageID {
return d.messageID
}
// deliverFilterTimeTickGT delivers messages by time tick greater than the specified time tick.
type deliverFilterTimeTickGT struct {
timeTick uint64
}
func (f *deliverFilterTimeTickGT) Type() deliverFilterType {
return DeliverFilterTypeTimeTickGT
}
func (f *deliverFilterTimeTickGT) TimeTick() uint64 {
return f.timeTick
}
func (f *deliverFilterTimeTickGT) Filter(msg message.ImmutableMessage) bool {
return msg.TimeTick() > f.timeTick
}
// deliverFilterTimeTickGTE delivers messages by time tick greater than or equal to the specified time tick.
type deliverFilterTimeTickGTE struct {
timeTick uint64
}
func (f *deliverFilterTimeTickGTE) Type() deliverFilterType {
return DeliverFilterTypeTimeTickGTE
}
func (f *deliverFilterTimeTickGTE) TimeTick() uint64 {
return f.timeTick
}
func (f *deliverFilterTimeTickGTE) Filter(msg message.ImmutableMessage) bool {
return msg.TimeTick() >= f.timeTick
}
// deliverFilterVChannel delivers messages by vchannel.
type deliverFilterVChannel struct {
vchannel string
}
func (f *deliverFilterVChannel) Type() deliverFilterType {
return DeliverFilterTypeVChannel
}
func (f *deliverFilterVChannel) VChannel() string {
return f.vchannel
}
func (f *deliverFilterVChannel) Filter(msg message.ImmutableMessage) bool {
return msg.VChannel() == f.vchannel
}

View File

@ -3,62 +3,33 @@ package options
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
func TestDeliverPolicy(t *testing.T) {
policy := DeliverPolicyAll()
assert.Equal(t, DeliverPolicyTypeAll, policy.Policy())
assert.Panics(t, func() {
policy.MessageID()
})
_ = policy.GetPolicy().(*streamingpb.DeliverPolicy_All)
policy = DeliverPolicyLatest()
assert.Equal(t, DeliverPolicyTypeLatest, policy.Policy())
assert.Panics(t, func() {
policy.MessageID()
})
_ = policy.GetPolicy().(*streamingpb.DeliverPolicy_Latest)
messageID := mock_message.NewMockMessageID(t)
messageID.EXPECT().Marshal().Return("messageID")
policy = DeliverPolicyStartFrom(messageID)
assert.Equal(t, DeliverPolicyTypeStartFrom, policy.Policy())
assert.Equal(t, messageID, policy.MessageID())
_ = policy.GetPolicy().(*streamingpb.DeliverPolicy_StartFrom)
policy = DeliverPolicyStartAfter(messageID)
assert.Equal(t, DeliverPolicyTypeStartAfter, policy.Policy())
assert.Equal(t, messageID, policy.MessageID())
_ = policy.GetPolicy().(*streamingpb.DeliverPolicy_StartAfter)
}
func TestDeliverFilter(t *testing.T) {
filter := DeliverFilterTimeTickGT(1)
assert.Equal(t, uint64(1), filter.(interface{ TimeTick() uint64 }).TimeTick())
assert.Equal(t, DeliverFilterTypeTimeTickGT, filter.Type())
msg := mock_message.NewMockImmutableMessage(t)
msg.EXPECT().TimeTick().Return(uint64(1))
assert.False(t, filter.Filter(msg))
msg.EXPECT().TimeTick().Unset()
msg.EXPECT().TimeTick().Return(uint64(2))
assert.True(t, filter.Filter(msg))
_ = filter.GetFilter().(*streamingpb.DeliverFilter_TimeTickGt)
filter = DeliverFilterTimeTickGTE(2)
assert.Equal(t, uint64(2), filter.(interface{ TimeTick() uint64 }).TimeTick())
assert.Equal(t, DeliverFilterTypeTimeTickGTE, filter.Type())
msg.EXPECT().TimeTick().Unset()
msg.EXPECT().TimeTick().Return(uint64(1))
assert.False(t, filter.Filter(msg))
msg.EXPECT().TimeTick().Unset()
msg.EXPECT().TimeTick().Return(uint64(2))
assert.True(t, filter.Filter(msg))
_ = filter.GetFilter().(*streamingpb.DeliverFilter_TimeTickGte)
filter = DeliverFilterVChannel("vchannel")
assert.Equal(t, "vchannel", filter.(interface{ VChannel() string }).VChannel())
assert.Equal(t, DeliverFilterTypeVChannel, filter.Type())
msg.EXPECT().VChannel().Unset()
msg.EXPECT().VChannel().Return("vchannel2")
assert.False(t, filter.Filter(msg))
msg.EXPECT().VChannel().Unset()
msg.EXPECT().VChannel().Return("vchannel")
assert.True(t, filter.Filter(msg))
_ = filter.GetFilter().(*streamingpb.DeliverFilter_Vchannel)
}

View File

@ -1,11 +1,43 @@
package types
import "fmt"
import (
"fmt"
"github.com/milvus-io/milvus/pkg/streaming/proto/streamingpb"
)
const (
InitialTerm int64 = -1
)
// NewPChannelInfoFromProto converts protobuf PChannelInfo to PChannelInfo
func NewPChannelInfoFromProto(pchannel *streamingpb.PChannelInfo) PChannelInfo {
if pchannel.GetName() == "" {
panic("pchannel name is empty")
}
if pchannel.GetTerm() <= 0 {
panic("pchannel term is empty or negetive")
}
return PChannelInfo{
Name: pchannel.GetName(),
Term: pchannel.GetTerm(),
}
}
// NewProtoFromPChannelInfo converts PChannelInfo to protobuf PChannelInfo
func NewProtoFromPChannelInfo(pchannel PChannelInfo) *streamingpb.PChannelInfo {
if pchannel.Name == "" {
panic("pchannel name is empty")
}
if pchannel.Term <= 0 {
panic("pchannel term is empty or negetive")
}
return &streamingpb.PChannelInfo{
Name: pchannel.Name,
Term: pchannel.Term,
}
}
// PChannelInfo is the struct for pchannel info.
type PChannelInfo struct {
Name string // name of pchannel.

Some files were not shown because too many files have changed in this diff Show More