mirror of https://github.com/milvus-io/milvus.git
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
parent
b5022b0647
commit
c725416288
|
@ -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:
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)}
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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),
|
||||
},
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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(),
|
||||
},
|
||||
|
|
|
@ -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),
|
||||
})
|
||||
}
|
||||
|
||||
|
|
|
@ -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()},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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(),
|
||||
},
|
||||
},
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
|
|
@ -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"
|
||||
)
|
||||
|
||||
|
|
|
@ -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 (
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 (
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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...
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
}
|
|
@ -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())
|
||||
}
|
|
@ -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,
|
||||
}
|
||||
}
|
|
@ -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,
|
||||
}
|
||||
}
|
|
@ -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:
|
||||
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
|
@ -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 {}
|
|
@ -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() {
|
||||
|
|
|
@ -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")
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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] {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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{
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue