mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 17:48:29 +08:00
enhance: make streaming message as a log field for easier coding (#41545)
issue: #41544 - implement message can be logged as a field by zap. - fix too many slow log for woodpecker. Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
6c12a1c7aa
commit
dfbb02a5f7
@ -1210,6 +1210,10 @@ streaming:
|
|||||||
walWriteAheadBuffer:
|
walWriteAheadBuffer:
|
||||||
capacity: 64m # The capacity of write ahead buffer of each wal, 64M by default
|
capacity: 64m # The capacity of write ahead buffer of each wal, 64M by default
|
||||||
keepalive: 30s # The keepalive duration for entries in write ahead buffer of each wal, 30s by default
|
keepalive: 30s # The keepalive duration for entries in write ahead buffer of each wal, 30s by default
|
||||||
|
logging:
|
||||||
|
# The threshold of slow log, 1s by default.
|
||||||
|
# If the wal implementation is woodpecker, the minimum threshold is 3s
|
||||||
|
appendSlowThreshold: 1s
|
||||||
|
|
||||||
# Any configuration related to the knowhere vector search engine
|
# Any configuration related to the knowhere vector search engine
|
||||||
knowhere:
|
knowhere:
|
||||||
|
|||||||
@ -195,11 +195,11 @@ func (impl *flusherComponents) buildDataSyncServiceWithRetry(ctx context.Context
|
|||||||
if len(segmentIDs) > 0 {
|
if len(segmentIDs) > 0 {
|
||||||
msg := message.NewFlushMessageBuilderV2().
|
msg := message.NewFlushMessageBuilderV2().
|
||||||
WithVChannel(recoverInfo.GetInfo().GetChannelName()).
|
WithVChannel(recoverInfo.GetInfo().GetChannelName()).
|
||||||
WithHeader(&message.FlushMessageHeader{}).
|
WithHeader(&message.FlushMessageHeader{
|
||||||
WithBody(&message.FlushMessageBody{
|
|
||||||
CollectionId: recoverInfo.GetInfo().GetCollectionID(),
|
CollectionId: recoverInfo.GetInfo().GetCollectionID(),
|
||||||
SegmentId: segmentIDs,
|
SegmentIds: segmentIDs,
|
||||||
}).MustBuildMutable()
|
}).
|
||||||
|
WithBody(&message.FlushMessageBody{}).MustBuildMutable()
|
||||||
if err := retry.Do(ctx, func() error {
|
if err := retry.Do(ctx, func() error {
|
||||||
appendResult, err := impl.wal.Append(ctx, msg)
|
appendResult, err := impl.wal.Append(ctx, msg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
|||||||
@ -71,11 +71,7 @@ func (impl *msgHandlerImpl) HandleCreateSegment(ctx context.Context, vchannel st
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (impl *msgHandlerImpl) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error {
|
func (impl *msgHandlerImpl) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error {
|
||||||
body, err := flushMsg.Body()
|
if err := impl.wbMgr.SealSegments(context.Background(), vchannel, flushMsg.Header().SegmentIds); err != nil {
|
||||||
if err != nil {
|
|
||||||
return errors.Wrap(err, "failed to get flush message body")
|
|
||||||
}
|
|
||||||
if err := impl.wbMgr.SealSegments(context.Background(), vchannel, body.GetSegmentId()); err != nil {
|
|
||||||
return errors.Wrap(err, "failed to seal segments")
|
return errors.Wrap(err, "failed to seal segments")
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
|
|||||||
@ -37,11 +37,11 @@ func TestFlushMsgHandler_HandleFlush(t *testing.T) {
|
|||||||
|
|
||||||
msg, err := message.NewFlushMessageBuilderV2().
|
msg, err := message.NewFlushMessageBuilderV2().
|
||||||
WithVChannel(vchannel).
|
WithVChannel(vchannel).
|
||||||
WithHeader(&message.FlushMessageHeader{}).
|
WithHeader(&message.FlushMessageHeader{
|
||||||
WithBody(&message.FlushMessageBody{
|
|
||||||
CollectionId: 0,
|
CollectionId: 0,
|
||||||
SegmentId: []int64{1, 2, 3},
|
SegmentIds: []int64{1, 2, 3},
|
||||||
}).
|
}).
|
||||||
|
WithBody(&message.FlushMessageBody{}).
|
||||||
BuildMutable()
|
BuildMutable()
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
|||||||
@ -87,6 +87,7 @@ func TestOpenerAdaptor(t *testing.T) {
|
|||||||
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
||||||
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
||||||
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
||||||
|
msg.EXPECT().IsPersisted().Return(false).Maybe()
|
||||||
|
|
||||||
msgID, err := wal.Append(context.Background(), msg)
|
msgID, err := wal.Append(context.Background(), msg)
|
||||||
time.Sleep(time.Millisecond * 10)
|
time.Sleep(time.Millisecond * 10)
|
||||||
|
|||||||
@ -223,9 +223,7 @@ func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) {
|
|||||||
s.metrics.ObserveTimeTickViolation(isTailing, msg.MessageType())
|
s.metrics.ObserveTimeTickViolation(isTailing, msg.MessageType())
|
||||||
}
|
}
|
||||||
s.logger.Warn("failed to push message into reorder buffer",
|
s.logger.Warn("failed to push message into reorder buffer",
|
||||||
zap.Any("msgID", msg.MessageID()),
|
zap.Object("message", msg),
|
||||||
zap.Uint64("timetick", msg.TimeTick()),
|
|
||||||
zap.String("vchannel", msg.VChannel()),
|
|
||||||
zap.Bool("tailing", isTailing),
|
zap.Bool("tailing", isTailing),
|
||||||
zap.Error(err))
|
zap.Error(err))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -129,7 +129,6 @@ func (w *walAdaptorImpl) Append(ctx context.Context, msg message.MutableMessage)
|
|||||||
func(ctx context.Context, msg message.MutableMessage) (message.MessageID, error) {
|
func(ctx context.Context, msg message.MutableMessage) (message.MessageID, error) {
|
||||||
if notPersistHint := utility.GetNotPersisted(ctx); notPersistHint != nil {
|
if notPersistHint := utility.GetNotPersisted(ctx); notPersistHint != nil {
|
||||||
// do not persist the message if the hint is set.
|
// do not persist the message if the hint is set.
|
||||||
appendMetrics.NotPersisted()
|
|
||||||
return notPersistHint.MessageID, nil
|
return notPersistHint.MessageID, nil
|
||||||
}
|
}
|
||||||
metricsGuard.StartWALImplAppend()
|
metricsGuard.StartWALImplAppend()
|
||||||
|
|||||||
@ -149,6 +149,8 @@ func TestWALAdaptor(t *testing.T) {
|
|||||||
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
||||||
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
||||||
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
||||||
|
msg.EXPECT().IsPersisted().Return(true).Maybe()
|
||||||
|
msg.EXPECT().MarshalLogObject(mock.Anything).Return(nil).Maybe()
|
||||||
_, err = lAdapted.Append(context.Background(), msg)
|
_, err = lAdapted.Append(context.Background(), msg)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
lAdapted.AppendAsync(context.Background(), msg, func(mi *wal.AppendResult, err error) {
|
lAdapted.AppendAsync(context.Background(), msg, func(mi *wal.AppendResult, err error) {
|
||||||
@ -214,9 +216,11 @@ func TestNoInterceptor(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
msg := mock_message.NewMockMutableMessage(t)
|
msg := mock_message.NewMockMutableMessage(t)
|
||||||
|
msg.EXPECT().IsPersisted().Return(true).Maybe()
|
||||||
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
||||||
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
||||||
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
||||||
|
msg.EXPECT().MarshalLogObject(mock.Anything).Return(nil).Maybe()
|
||||||
_, err = lWithInterceptors.Append(context.Background(), msg)
|
_, err = lWithInterceptors.Append(context.Background(), msg)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
lWithInterceptors.Close()
|
lWithInterceptors.Close()
|
||||||
@ -253,6 +257,8 @@ func TestWALWithInterceptor(t *testing.T) {
|
|||||||
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
msg.EXPECT().WithWALTerm(mock.Anything).Return(msg).Maybe()
|
||||||
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe()
|
||||||
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
msg.EXPECT().EstimateSize().Return(1).Maybe()
|
||||||
|
msg.EXPECT().IsPersisted().Return(true).Maybe()
|
||||||
|
msg.EXPECT().MarshalLogObject(mock.Anything).Return(nil).Maybe()
|
||||||
_, err = lWithInterceptors.Append(ctx, msg)
|
_, err = lWithInterceptors.Append(ctx, msg)
|
||||||
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
assert.ErrorIs(t, err, context.DeadlineExceeded)
|
||||||
|
|
||||||
|
|||||||
@ -3,6 +3,7 @@ package interceptors_test
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"os"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
@ -16,8 +17,17 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/mocks/streaming/util/mock_message"
|
"github.com/milvus-io/milvus/pkg/v2/mocks/streaming/util/mock_message"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
func TestMain(m *testing.M) {
|
||||||
|
paramtable.Init()
|
||||||
|
code := m.Run()
|
||||||
|
if code != 0 {
|
||||||
|
os.Exit(code)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func TestChainInterceptor(t *testing.T) {
|
func TestChainInterceptor(t *testing.T) {
|
||||||
for i := 0; i < 5; i++ {
|
for i := 0; i < 5; i++ {
|
||||||
testChainInterceptor(t, i, false)
|
testChainInterceptor(t, i, false)
|
||||||
|
|||||||
@ -241,7 +241,10 @@ func (m *partitionSegmentManager) allocNewGrowingSegment(ctx context.Context) (*
|
|||||||
}
|
}
|
||||||
msg, err := message.NewCreateSegmentMessageBuilderV2().
|
msg, err := message.NewCreateSegmentMessageBuilderV2().
|
||||||
WithVChannel(pendingSegment.GetVChannel()).
|
WithVChannel(pendingSegment.GetVChannel()).
|
||||||
WithHeader(&message.CreateSegmentMessageHeader{}).
|
WithHeader(&message.CreateSegmentMessageHeader{
|
||||||
|
CollectionId: pendingSegment.GetCollectionID(),
|
||||||
|
SegmentIds: []int64{pendingSegment.GetSegmentID()},
|
||||||
|
}).
|
||||||
WithBody(&message.CreateSegmentMessageBody{
|
WithBody(&message.CreateSegmentMessageBody{
|
||||||
CollectionId: pendingSegment.GetCollectionID(),
|
CollectionId: pendingSegment.GetCollectionID(),
|
||||||
Segments: []*messagespb.CreateSegmentInfo{{
|
Segments: []*messagespb.CreateSegmentInfo{{
|
||||||
|
|||||||
@ -209,11 +209,11 @@ func (m *sealQueue) sendFlushSegmentsMessageIntoWAL(ctx context.Context, collect
|
|||||||
}
|
}
|
||||||
msg, err := message.NewFlushMessageBuilderV2().
|
msg, err := message.NewFlushMessageBuilderV2().
|
||||||
WithVChannel(vchannel).
|
WithVChannel(vchannel).
|
||||||
WithHeader(&message.FlushMessageHeader{}).
|
WithHeader(&message.FlushMessageHeader{
|
||||||
WithBody(&message.FlushMessageBody{
|
|
||||||
CollectionId: collectionID,
|
CollectionId: collectionID,
|
||||||
SegmentId: segmentIDs,
|
SegmentIds: segmentIDs,
|
||||||
}).BuildMutable()
|
}).
|
||||||
|
WithBody(&message.FlushMessageBody{}).BuildMutable()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Wrap(err, "at create new flush segments message")
|
return errors.Wrap(err, "at create new flush segments message")
|
||||||
}
|
}
|
||||||
|
|||||||
@ -176,7 +176,7 @@ func TestManager(t *testing.T) {
|
|||||||
assert.Equal(t, int32(0), count.Load())
|
assert.Equal(t, int32(0), count.Load())
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestWithCo(t *testing.T) {
|
func TestWithContext(t *testing.T) {
|
||||||
session := &TxnSession{}
|
session := &TxnSession{}
|
||||||
ctx := WithTxnSession(context.Background(), session)
|
ctx := WithTxnSession(context.Background(), session)
|
||||||
|
|
||||||
|
|||||||
@ -24,14 +24,13 @@ func (im *InterceptorMetrics) String() string {
|
|||||||
type AppendMetrics struct {
|
type AppendMetrics struct {
|
||||||
wm *WriteMetrics
|
wm *WriteMetrics
|
||||||
bytes int
|
bytes int
|
||||||
messageType message.MessageType
|
msg message.MutableMessage
|
||||||
|
|
||||||
result *types.AppendResult
|
result *types.AppendResult
|
||||||
err error
|
err error
|
||||||
appendDuration time.Duration
|
appendDuration time.Duration
|
||||||
implAppendDuration time.Duration
|
implAppendDuration time.Duration
|
||||||
interceptors map[string][]*InterceptorMetrics
|
interceptors map[string][]*InterceptorMetrics
|
||||||
persisted bool
|
|
||||||
}
|
}
|
||||||
|
|
||||||
type AppendMetricsGuard struct {
|
type AppendMetricsGuard struct {
|
||||||
@ -65,11 +64,9 @@ func (m *AppendMetrics) StartAppendGuard() *AppendMetricsGuard {
|
|||||||
// IntoLogFields convert the metrics to log fields.
|
// IntoLogFields convert the metrics to log fields.
|
||||||
func (m *AppendMetrics) IntoLogFields() []zap.Field {
|
func (m *AppendMetrics) IntoLogFields() []zap.Field {
|
||||||
fields := []zap.Field{
|
fields := []zap.Field{
|
||||||
zap.String("message_type", m.messageType.String()),
|
zap.Object("message", m.msg),
|
||||||
zap.Int("bytes", m.bytes),
|
|
||||||
zap.Duration("append_duration", m.appendDuration),
|
zap.Duration("append_duration", m.appendDuration),
|
||||||
zap.Duration("impl_append_duration", m.implAppendDuration),
|
zap.Duration("impl_append_duration", m.implAppendDuration),
|
||||||
zap.Bool("presisted", m.persisted),
|
|
||||||
}
|
}
|
||||||
for name, ims := range m.interceptors {
|
for name, ims := range m.interceptors {
|
||||||
for i, im := range ims {
|
for i, im := range ims {
|
||||||
@ -110,11 +107,6 @@ func (m *AppendMetrics) RangeOverInterceptors(f func(name string, ims []*Interce
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// NotPersisted mark the message is not persisted.
|
|
||||||
func (m *AppendMetrics) NotPersisted() {
|
|
||||||
m.persisted = false
|
|
||||||
}
|
|
||||||
|
|
||||||
// Done push the metrics.
|
// Done push the metrics.
|
||||||
func (m *AppendMetrics) Done(result *types.AppendResult, err error) {
|
func (m *AppendMetrics) Done(result *types.AppendResult, err error) {
|
||||||
m.err = err
|
m.err = err
|
||||||
|
|||||||
@ -12,6 +12,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/wp"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -27,6 +28,14 @@ func NewWriteMetrics(pchannel types.PChannelInfo, walName string) *WriteMetrics
|
|||||||
strconv.FormatInt(pchannel.Term, 10),
|
strconv.FormatInt(pchannel.Term, 10),
|
||||||
walName).Set(1)
|
walName).Set(1)
|
||||||
|
|
||||||
|
slowLogThreshold := paramtable.Get().StreamingCfg.LoggingAppendSlowThreshold.GetAsDurationByParse()
|
||||||
|
if slowLogThreshold <= 0 {
|
||||||
|
slowLogThreshold = time.Second
|
||||||
|
}
|
||||||
|
if walName == wp.WALName && slowLogThreshold < 3*time.Second {
|
||||||
|
// slow log threshold is not set in woodpecker, so we set it to 0.
|
||||||
|
slowLogThreshold = 3 * time.Second
|
||||||
|
}
|
||||||
return &WriteMetrics{
|
return &WriteMetrics{
|
||||||
walName: walName,
|
walName: walName,
|
||||||
pchannel: pchannel,
|
pchannel: pchannel,
|
||||||
@ -37,6 +46,7 @@ func NewWriteMetrics(pchannel types.PChannelInfo, walName string) *WriteMetrics
|
|||||||
walimplsDuration: metrics.WALImplsAppendMessageDurationSeconds.MustCurryWith(constLabel),
|
walimplsDuration: metrics.WALImplsAppendMessageDurationSeconds.MustCurryWith(constLabel),
|
||||||
walBeforeInterceptorDuration: metrics.WALAppendMessageBeforeInterceptorDurationSeconds.MustCurryWith(constLabel),
|
walBeforeInterceptorDuration: metrics.WALAppendMessageBeforeInterceptorDurationSeconds.MustCurryWith(constLabel),
|
||||||
walAfterInterceptorDuration: metrics.WALAppendMessageAfterInterceptorDurationSeconds.MustCurryWith(constLabel),
|
walAfterInterceptorDuration: metrics.WALAppendMessageAfterInterceptorDurationSeconds.MustCurryWith(constLabel),
|
||||||
|
slowLogThreshold: time.Second,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -52,30 +62,27 @@ type WriteMetrics struct {
|
|||||||
walimplsDuration prometheus.ObserverVec
|
walimplsDuration prometheus.ObserverVec
|
||||||
walBeforeInterceptorDuration prometheus.ObserverVec
|
walBeforeInterceptorDuration prometheus.ObserverVec
|
||||||
walAfterInterceptorDuration prometheus.ObserverVec
|
walAfterInterceptorDuration prometheus.ObserverVec
|
||||||
|
slowLogThreshold time.Duration
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *WriteMetrics) StartAppend(msg message.MutableMessage) *AppendMetrics {
|
func (m *WriteMetrics) StartAppend(msg message.MutableMessage) *AppendMetrics {
|
||||||
return &AppendMetrics{
|
return &AppendMetrics{
|
||||||
wm: m,
|
wm: m,
|
||||||
messageType: msg.MessageType(),
|
msg: msg,
|
||||||
bytes: msg.EstimateSize(),
|
|
||||||
persisted: true,
|
|
||||||
interceptors: make(map[string][]*InterceptorMetrics),
|
interceptors: make(map[string][]*InterceptorMetrics),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *WriteMetrics) done(appendMetrics *AppendMetrics) {
|
func (m *WriteMetrics) done(appendMetrics *AppendMetrics) {
|
||||||
if !appendMetrics.persisted {
|
if !appendMetrics.msg.IsPersisted() {
|
||||||
// ignore all the metrics if the message is not persisted.
|
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
status := parseError(appendMetrics.err)
|
status := parseError(appendMetrics.err)
|
||||||
if appendMetrics.implAppendDuration != 0 {
|
if appendMetrics.implAppendDuration != 0 {
|
||||||
m.walimplsDuration.WithLabelValues(status).Observe(appendMetrics.implAppendDuration.Seconds())
|
m.walimplsDuration.WithLabelValues(status).Observe(appendMetrics.implAppendDuration.Seconds())
|
||||||
}
|
}
|
||||||
m.bytes.WithLabelValues(status).Observe(float64(appendMetrics.bytes))
|
m.bytes.WithLabelValues(status).Observe(float64(appendMetrics.bytes))
|
||||||
m.total.WithLabelValues(appendMetrics.messageType.String(), status).Inc()
|
m.total.WithLabelValues(appendMetrics.msg.MessageType().String(), status).Inc()
|
||||||
m.walDuration.WithLabelValues(status).Observe(appendMetrics.appendDuration.Seconds())
|
m.walDuration.WithLabelValues(status).Observe(appendMetrics.appendDuration.Seconds())
|
||||||
for name, ims := range appendMetrics.interceptors {
|
for name, ims := range appendMetrics.interceptors {
|
||||||
for _, im := range ims {
|
for _, im := range ims {
|
||||||
@ -91,7 +98,7 @@ func (m *WriteMetrics) done(appendMetrics *AppendMetrics) {
|
|||||||
m.Logger().Warn("append message into wal failed", appendMetrics.IntoLogFields()...)
|
m.Logger().Warn("append message into wal failed", appendMetrics.IntoLogFields()...)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
if appendMetrics.appendDuration >= time.Second {
|
if appendMetrics.appendDuration >= m.slowLogThreshold {
|
||||||
// log slow append catch
|
// log slow append catch
|
||||||
m.Logger().Warn("append message into wal too slow", appendMetrics.IntoLogFields()...)
|
m.Logger().Warn("append message into wal too slow", appendMetrics.IntoLogFields()...)
|
||||||
return
|
return
|
||||||
|
|||||||
@ -5,6 +5,8 @@ package mock_message
|
|||||||
import (
|
import (
|
||||||
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
mock "github.com/stretchr/testify/mock"
|
mock "github.com/stretchr/testify/mock"
|
||||||
|
|
||||||
|
zapcore "go.uber.org/zap/zapcore"
|
||||||
)
|
)
|
||||||
|
|
||||||
// MockImmutableMessage is an autogenerated mock type for the ImmutableMessage type
|
// MockImmutableMessage is an autogenerated mock type for the ImmutableMessage type
|
||||||
@ -249,6 +251,52 @@ func (_c *MockImmutableMessage_LastConfirmedMessageID_Call) RunAndReturn(run fun
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MarshalLogObject provides a mock function with given fields: _a0
|
||||||
|
func (_m *MockImmutableMessage) MarshalLogObject(_a0 zapcore.ObjectEncoder) error {
|
||||||
|
ret := _m.Called(_a0)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for MarshalLogObject")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 error
|
||||||
|
if rf, ok := ret.Get(0).(func(zapcore.ObjectEncoder) error); ok {
|
||||||
|
r0 = rf(_a0)
|
||||||
|
} else {
|
||||||
|
r0 = ret.Error(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockImmutableMessage_MarshalLogObject_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MarshalLogObject'
|
||||||
|
type MockImmutableMessage_MarshalLogObject_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// MarshalLogObject is a helper method to define mock.On call
|
||||||
|
// - _a0 zapcore.ObjectEncoder
|
||||||
|
func (_e *MockImmutableMessage_Expecter) MarshalLogObject(_a0 interface{}) *MockImmutableMessage_MarshalLogObject_Call {
|
||||||
|
return &MockImmutableMessage_MarshalLogObject_Call{Call: _e.mock.On("MarshalLogObject", _a0)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockImmutableMessage_MarshalLogObject_Call) Run(run func(_a0 zapcore.ObjectEncoder)) *MockImmutableMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(zapcore.ObjectEncoder))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockImmutableMessage_MarshalLogObject_Call) Return(_a0 error) *MockImmutableMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Return(_a0)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockImmutableMessage_MarshalLogObject_Call) RunAndReturn(run func(zapcore.ObjectEncoder) error) *MockImmutableMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// MessageID provides a mock function with no fields
|
// MessageID provides a mock function with no fields
|
||||||
func (_m *MockImmutableMessage) MessageID() message.MessageID {
|
func (_m *MockImmutableMessage) MessageID() message.MessageID {
|
||||||
ret := _m.Called()
|
ret := _m.Called()
|
||||||
|
|||||||
@ -5,6 +5,8 @@ package mock_message
|
|||||||
import (
|
import (
|
||||||
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
mock "github.com/stretchr/testify/mock"
|
mock "github.com/stretchr/testify/mock"
|
||||||
|
|
||||||
|
zapcore "go.uber.org/zap/zapcore"
|
||||||
)
|
)
|
||||||
|
|
||||||
// MockImmutableTxnMessage is an autogenerated mock type for the ImmutableTxnMessage type
|
// MockImmutableTxnMessage is an autogenerated mock type for the ImmutableTxnMessage type
|
||||||
@ -343,6 +345,52 @@ func (_c *MockImmutableTxnMessage_LastConfirmedMessageID_Call) RunAndReturn(run
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MarshalLogObject provides a mock function with given fields: _a0
|
||||||
|
func (_m *MockImmutableTxnMessage) MarshalLogObject(_a0 zapcore.ObjectEncoder) error {
|
||||||
|
ret := _m.Called(_a0)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for MarshalLogObject")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 error
|
||||||
|
if rf, ok := ret.Get(0).(func(zapcore.ObjectEncoder) error); ok {
|
||||||
|
r0 = rf(_a0)
|
||||||
|
} else {
|
||||||
|
r0 = ret.Error(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockImmutableTxnMessage_MarshalLogObject_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MarshalLogObject'
|
||||||
|
type MockImmutableTxnMessage_MarshalLogObject_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// MarshalLogObject is a helper method to define mock.On call
|
||||||
|
// - _a0 zapcore.ObjectEncoder
|
||||||
|
func (_e *MockImmutableTxnMessage_Expecter) MarshalLogObject(_a0 interface{}) *MockImmutableTxnMessage_MarshalLogObject_Call {
|
||||||
|
return &MockImmutableTxnMessage_MarshalLogObject_Call{Call: _e.mock.On("MarshalLogObject", _a0)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockImmutableTxnMessage_MarshalLogObject_Call) Run(run func(_a0 zapcore.ObjectEncoder)) *MockImmutableTxnMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(zapcore.ObjectEncoder))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockImmutableTxnMessage_MarshalLogObject_Call) Return(_a0 error) *MockImmutableTxnMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Return(_a0)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockImmutableTxnMessage_MarshalLogObject_Call) RunAndReturn(run func(zapcore.ObjectEncoder) error) *MockImmutableTxnMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// MessageID provides a mock function with no fields
|
// MessageID provides a mock function with no fields
|
||||||
func (_m *MockImmutableTxnMessage) MessageID() message.MessageID {
|
func (_m *MockImmutableTxnMessage) MessageID() message.MessageID {
|
||||||
ret := _m.Called()
|
ret := _m.Called()
|
||||||
|
|||||||
@ -5,6 +5,8 @@ package mock_message
|
|||||||
import (
|
import (
|
||||||
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
mock "github.com/stretchr/testify/mock"
|
mock "github.com/stretchr/testify/mock"
|
||||||
|
|
||||||
|
zapcore "go.uber.org/zap/zapcore"
|
||||||
)
|
)
|
||||||
|
|
||||||
// MockMutableMessage is an autogenerated mock type for the MutableMessage type
|
// MockMutableMessage is an autogenerated mock type for the MutableMessage type
|
||||||
@ -250,6 +252,52 @@ func (_c *MockMutableMessage_IsPersisted_Call) RunAndReturn(run func() bool) *Mo
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MarshalLogObject provides a mock function with given fields: _a0
|
||||||
|
func (_m *MockMutableMessage) MarshalLogObject(_a0 zapcore.ObjectEncoder) error {
|
||||||
|
ret := _m.Called(_a0)
|
||||||
|
|
||||||
|
if len(ret) == 0 {
|
||||||
|
panic("no return value specified for MarshalLogObject")
|
||||||
|
}
|
||||||
|
|
||||||
|
var r0 error
|
||||||
|
if rf, ok := ret.Get(0).(func(zapcore.ObjectEncoder) error); ok {
|
||||||
|
r0 = rf(_a0)
|
||||||
|
} else {
|
||||||
|
r0 = ret.Error(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
return r0
|
||||||
|
}
|
||||||
|
|
||||||
|
// MockMutableMessage_MarshalLogObject_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MarshalLogObject'
|
||||||
|
type MockMutableMessage_MarshalLogObject_Call struct {
|
||||||
|
*mock.Call
|
||||||
|
}
|
||||||
|
|
||||||
|
// MarshalLogObject is a helper method to define mock.On call
|
||||||
|
// - _a0 zapcore.ObjectEncoder
|
||||||
|
func (_e *MockMutableMessage_Expecter) MarshalLogObject(_a0 interface{}) *MockMutableMessage_MarshalLogObject_Call {
|
||||||
|
return &MockMutableMessage_MarshalLogObject_Call{Call: _e.mock.On("MarshalLogObject", _a0)}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMutableMessage_MarshalLogObject_Call) Run(run func(_a0 zapcore.ObjectEncoder)) *MockMutableMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Run(func(args mock.Arguments) {
|
||||||
|
run(args[0].(zapcore.ObjectEncoder))
|
||||||
|
})
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMutableMessage_MarshalLogObject_Call) Return(_a0 error) *MockMutableMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Return(_a0)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
|
func (_c *MockMutableMessage_MarshalLogObject_Call) RunAndReturn(run func(zapcore.ObjectEncoder) error) *MockMutableMessage_MarshalLogObject_Call {
|
||||||
|
_c.Call.Return(run)
|
||||||
|
return _c
|
||||||
|
}
|
||||||
|
|
||||||
// MessageType provides a mock function with no fields
|
// MessageType provides a mock function with no fields
|
||||||
func (_m *MockMutableMessage) MessageType() message.MessageType {
|
func (_m *MockMutableMessage) MessageType() message.MessageType {
|
||||||
ret := _m.Called()
|
ret := _m.Called()
|
||||||
|
|||||||
@ -76,11 +76,7 @@ enum MessageType {
|
|||||||
///
|
///
|
||||||
|
|
||||||
// FlushMessageBody is the body of flush message.
|
// FlushMessageBody is the body of flush message.
|
||||||
message FlushMessageBody {
|
message FlushMessageBody {}
|
||||||
// indicate which the collection that segment belong to.
|
|
||||||
int64 collection_id = 1;
|
|
||||||
repeated int64 segment_id = 2; // indicate which segment to flush.
|
|
||||||
}
|
|
||||||
|
|
||||||
// ManualFlushMessageBody is the body of manual flush message.
|
// ManualFlushMessageBody is the body of manual flush message.
|
||||||
message ManualFlushMessageBody {}
|
message ManualFlushMessageBody {}
|
||||||
@ -154,10 +150,16 @@ message DeleteMessageHeader {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// FlushMessageHeader just nothing.
|
// FlushMessageHeader just nothing.
|
||||||
message FlushMessageHeader {}
|
message FlushMessageHeader {
|
||||||
|
int64 collection_id = 1;
|
||||||
|
repeated int64 segment_ids = 2;
|
||||||
|
}
|
||||||
|
|
||||||
// CreateSegmentMessageHeader just nothing.
|
// CreateSegmentMessageHeader just nothing.
|
||||||
message CreateSegmentMessageHeader {}
|
message CreateSegmentMessageHeader {
|
||||||
|
int64 collection_id = 1;
|
||||||
|
repeated int64 segment_ids = 2;
|
||||||
|
}
|
||||||
|
|
||||||
message ManualFlushMessageHeader {
|
message ManualFlushMessageHeader {
|
||||||
int64 collection_id = 1;
|
int64 collection_id = 1;
|
||||||
|
|||||||
@ -421,10 +421,6 @@ type FlushMessageBody struct {
|
|||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
sizeCache protoimpl.SizeCache
|
||||||
unknownFields protoimpl.UnknownFields
|
unknownFields protoimpl.UnknownFields
|
||||||
|
|
||||||
// indicate which the collection that segment belong to.
|
|
||||||
CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
|
|
||||||
SegmentId []int64 `protobuf:"varint,2,rep,packed,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` // indicate which segment to flush.
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *FlushMessageBody) Reset() {
|
func (x *FlushMessageBody) Reset() {
|
||||||
@ -459,20 +455,6 @@ func (*FlushMessageBody) Descriptor() ([]byte, []int) {
|
|||||||
return file_messages_proto_rawDescGZIP(), []int{3}
|
return file_messages_proto_rawDescGZIP(), []int{3}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *FlushMessageBody) GetCollectionId() int64 {
|
|
||||||
if x != nil {
|
|
||||||
return x.CollectionId
|
|
||||||
}
|
|
||||||
return 0
|
|
||||||
}
|
|
||||||
|
|
||||||
func (x *FlushMessageBody) GetSegmentId() []int64 {
|
|
||||||
if x != nil {
|
|
||||||
return x.SegmentId
|
|
||||||
}
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// ManualFlushMessageBody is the body of manual flush message.
|
// ManualFlushMessageBody is the body of manual flush message.
|
||||||
type ManualFlushMessageBody struct {
|
type ManualFlushMessageBody struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
@ -1072,6 +1054,9 @@ type FlushMessageHeader struct {
|
|||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
sizeCache protoimpl.SizeCache
|
||||||
unknownFields protoimpl.UnknownFields
|
unknownFields protoimpl.UnknownFields
|
||||||
|
|
||||||
|
CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
|
||||||
|
SegmentIds []int64 `protobuf:"varint,2,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *FlushMessageHeader) Reset() {
|
func (x *FlushMessageHeader) Reset() {
|
||||||
@ -1106,11 +1091,28 @@ func (*FlushMessageHeader) Descriptor() ([]byte, []int) {
|
|||||||
return file_messages_proto_rawDescGZIP(), []int{16}
|
return file_messages_proto_rawDescGZIP(), []int{16}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (x *FlushMessageHeader) GetCollectionId() int64 {
|
||||||
|
if x != nil {
|
||||||
|
return x.CollectionId
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *FlushMessageHeader) GetSegmentIds() []int64 {
|
||||||
|
if x != nil {
|
||||||
|
return x.SegmentIds
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
// CreateSegmentMessageHeader just nothing.
|
// CreateSegmentMessageHeader just nothing.
|
||||||
type CreateSegmentMessageHeader struct {
|
type CreateSegmentMessageHeader struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
sizeCache protoimpl.SizeCache
|
||||||
unknownFields protoimpl.UnknownFields
|
unknownFields protoimpl.UnknownFields
|
||||||
|
|
||||||
|
CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
|
||||||
|
SegmentIds []int64 `protobuf:"varint,2,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func (x *CreateSegmentMessageHeader) Reset() {
|
func (x *CreateSegmentMessageHeader) Reset() {
|
||||||
@ -1145,6 +1147,20 @@ func (*CreateSegmentMessageHeader) Descriptor() ([]byte, []int) {
|
|||||||
return file_messages_proto_rawDescGZIP(), []int{17}
|
return file_messages_proto_rawDescGZIP(), []int{17}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (x *CreateSegmentMessageHeader) GetCollectionId() int64 {
|
||||||
|
if x != nil {
|
||||||
|
return x.CollectionId
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
func (x *CreateSegmentMessageHeader) GetSegmentIds() []int64 {
|
||||||
|
if x != nil {
|
||||||
|
return x.SegmentIds
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
type ManualFlushMessageHeader struct {
|
type ManualFlushMessageHeader struct {
|
||||||
state protoimpl.MessageState
|
state protoimpl.MessageState
|
||||||
sizeCache protoimpl.SizeCache
|
sizeCache protoimpl.SizeCache
|
||||||
@ -2117,207 +2133,211 @@ var file_messages_proto_rawDesc = []byte{
|
|||||||
0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
|
0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||||
0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
|
0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
|
||||||
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
|
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
|
||||||
0x22, 0x56, 0x0a, 0x10, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
0x22, 0x12, 0x0a, 0x10, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
0x42, 0x6f, 0x64, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
0x42, 0x6f, 0x64, 0x79, 0x22, 0x18, 0x0a, 0x16, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c,
|
||||||
0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c,
|
0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x85,
|
||||||
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67,
|
0x01, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74,
|
||||||
0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x73,
|
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63,
|
||||||
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x4d, 0x61, 0x6e, 0x75,
|
|
||||||
0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f,
|
|
||||||
0x64, 0x79, 0x22, 0x85, 0x01, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67,
|
|
||||||
0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12,
|
|
||||||
0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
|
|
||||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
|
||||||
0x6f, 0x6e, 0x49, 0x64, 0x12, 0x44, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73,
|
|
||||||
0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
|
||||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x43,
|
|
||||||
0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f,
|
|
||||||
0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7e, 0x0a, 0x11, 0x43, 0x72,
|
|
||||||
0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12,
|
|
||||||
0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18,
|
|
||||||
0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
|
||||||
0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64,
|
|
||||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49,
|
|
||||||
0x64, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, 0x65, 0x72,
|
|
||||||
0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72,
|
|
||||||
0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, 0x42, 0x65,
|
|
||||||
0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64,
|
|
||||||
0x79, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65,
|
|
||||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x18, 0x0a, 0x16, 0x52, 0x6f, 0x6c,
|
|
||||||
0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42,
|
|
||||||
0x6f, 0x64, 0x79, 0x22, 0x4c, 0x0a, 0x0e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
|
||||||
0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3a, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
|
||||||
0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
|
||||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
|
|
||||||
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
|
||||||
0x73, 0x22, 0x17, 0x0a, 0x15, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d, 0x65, 0x73,
|
|
||||||
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x8d, 0x01, 0x0a, 0x13, 0x49,
|
|
||||||
0x6e, 0x73, 0x65, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64,
|
|
||||||
0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
|
||||||
0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
|
|
||||||
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69,
|
|
||||||
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69,
|
|
||||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61,
|
|
||||||
0x67, 0x65, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67,
|
|
||||||
0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0a,
|
|
||||||
0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xcd, 0x01, 0x0a, 0x1a, 0x50,
|
|
||||||
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41,
|
|
||||||
0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72,
|
|
||||||
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
|
|
||||||
0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04,
|
|
||||||
0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73,
|
|
||||||
0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18,
|
|
||||||
0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a,
|
|
||||||
0x65, 0x12, 0x57, 0x0a, 0x12, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x73, 0x73,
|
|
||||||
0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e,
|
|
||||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73,
|
|
||||||
0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73,
|
|
||||||
0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x11, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74,
|
|
||||||
0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x32, 0x0a, 0x11, 0x53, 0x65,
|
|
||||||
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12,
|
|
||||||
0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
|
||||||
0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x3a,
|
|
||||||
0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48,
|
|
||||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
|
|
||||||
0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f,
|
|
||||||
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x14, 0x0a, 0x12, 0x46, 0x6c,
|
|
||||||
0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
|
||||||
0x22, 0x1c, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e,
|
|
||||||
0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x5a,
|
|
||||||
0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73,
|
|
||||||
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f,
|
|
||||||
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
|
|
||||||
0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12,
|
|
||||||
0x19, 0x0a, 0x08, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28,
|
|
||||||
0x04, 0x52, 0x07, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x54, 0x73, 0x22, 0x69, 0x0a, 0x1d, 0x43, 0x72,
|
|
||||||
0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65,
|
|
||||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63,
|
|
||||||
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
|
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
|
||||||
0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
|
0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
|
||||||
0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
|
0x12, 0x44, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03,
|
||||||
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
|
||||||
0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42, 0x0a, 0x1b, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c,
|
0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74,
|
||||||
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65,
|
0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x73, 0x65,
|
||||||
0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7e, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65,
|
||||||
0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c,
|
0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x70,
|
||||||
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43, 0x72, 0x65,
|
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||||
0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73,
|
0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d,
|
||||||
0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c,
|
0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
|
||||||
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,
|
0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x27, 0x0a,
|
||||||
0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21,
|
0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e,
|
||||||
0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02,
|
0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56,
|
||||||
0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49,
|
0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54,
|
||||||
0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x16, 0x0a,
|
||||||
0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12,
|
0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
||||||
0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
|
0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x18, 0x0a, 0x16, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63,
|
||||||
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22,
|
||||||
0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
|
0x4c, 0x0a, 0x0e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64,
|
||||||
0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74,
|
0x79, 0x12, 0x3a, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20,
|
||||||
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67, 0x69, 0x6e,
|
0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
||||||
0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73,
|
||||||
0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69,
|
0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x22, 0x17, 0x0a,
|
||||||
0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,
|
0x15, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69,
|
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x8d, 0x01, 0x0a, 0x13, 0x49, 0x6e, 0x73, 0x65, 0x72,
|
||||||
0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x69,
|
0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23,
|
||||||
0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
|
0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18,
|
||||||
0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e,
|
0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f,
|
||||||
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x12, 0x0a,
|
0x6e, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,
|
||||||
0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
|
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
||||||
0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61,
|
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
|
||||||
0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x40, 0x0a, 0x19, 0x53, 0x63, 0x68, 0x65,
|
0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74,
|
||||||
0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48,
|
0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74,
|
||||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
|
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xcd, 0x01, 0x0a, 0x1a, 0x50, 0x61, 0x72, 0x74, 0x69,
|
||||||
0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f,
|
0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67,
|
||||||
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x75, 0x0a, 0x17, 0x53, 0x63,
|
0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
||||||
0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72,
|
||||||
0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18,
|
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73,
|
||||||
0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x1f, 0x0a, 0x0b,
|
||||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c,
|
0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28,
|
||||||
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63,
|
0x04, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x57, 0x0a,
|
||||||
0x68, 0x65, 0x6d, 0x61, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, 0x74,
|
0x12, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d,
|
||||||
0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x54,
|
0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
||||||
0x73, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68,
|
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a,
|
0x73, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d,
|
||||||
0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03,
|
0x65, 0x6e, 0x74, 0x52, 0x11, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69,
|
||||||
0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a,
|
0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x32, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e,
|
||||||
0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06,
|
0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73,
|
||||||
0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78,
|
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||||
0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65,
|
0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x3a, 0x0a, 0x13, 0x44, 0x65,
|
||||||
0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20,
|
0x6c, 0x65, 0x74, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
|
||||||
0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69,
|
0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
|
||||||
0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52,
|
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
||||||
0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12,
|
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x5a, 0x0a, 0x12, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d,
|
||||||
0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c,
|
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d,
|
||||||
0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f,
|
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
||||||
0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e,
|
0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49,
|
||||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73,
|
0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73,
|
||||||
0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49,
|
||||||
0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65,
|
0x64, 0x73, 0x22, 0x62, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d,
|
||||||
0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69,
|
0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||||
0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73,
|
0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69,
|
||||||
0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
|
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
|
||||||
0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
|
0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74,
|
||||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,
|
0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d,
|
||||||
0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48,
|
0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c,
|
||||||
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
|
0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64,
|
||||||
0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f,
|
0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
||||||
0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61,
|
0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
|
||||||
0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68,
|
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x6c, 0x75, 0x73, 0x68,
|
||||||
0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
|
0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x66, 0x6c, 0x75, 0x73, 0x68,
|
||||||
0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e,
|
0x54, 0x73, 0x22, 0x69, 0x0a, 0x1d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c,
|
||||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73,
|
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61,
|
||||||
0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
|
0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f,
|
||||||
0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22,
|
0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c,
|
||||||
0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d,
|
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74,
|
||||||
0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25,
|
0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52,
|
||||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65,
|
0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42, 0x0a,
|
||||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44,
|
0x1b, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d,
|
||||||
0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a,
|
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d,
|
||||||
0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22,
|
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
||||||
0x63, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12,
|
0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49,
|
||||||
0x13, 0x0a, 0x05, 0x65, 0x7a, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04,
|
0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69,
|
||||||
0x65, 0x7a, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79,
|
0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
|
||||||
0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12,
|
0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
|
||||||
0x23, 0x0a, 0x0d, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73,
|
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
||||||
0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42,
|
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
|
||||||
0x79, 0x74, 0x65, 0x73, 0x2a, 0x9a, 0x02, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61,
|
||||||
0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10,
|
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72, 0x6f,
|
||||||
0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12,
|
0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
||||||
0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44,
|
0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
|
||||||
0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68,
|
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c,
|
||||||
0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c,
|
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c,
|
||||||
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70,
|
0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
|
||||||
0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f,
|
0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22,
|
||||||
0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10,
|
0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61,
|
||||||
0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
|
0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70,
|
||||||
0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c,
|
0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e,
|
||||||
0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53,
|
0x64, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c,
|
||||||
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f,
|
0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22,
|
||||||
0x72, 0x74, 0x10, 0x0b, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68,
|
0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73,
|
||||||
0x61, 0x6e, 0x67, 0x65, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54,
|
0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c,
|
||||||
0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54,
|
0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48,
|
||||||
0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63,
|
0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x12, 0x0a, 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73,
|
||||||
0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7,
|
0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70,
|
||||||
0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e,
|
0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72,
|
||||||
0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c,
|
0x22, 0x40, 0x0a, 0x19, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65,
|
||||||
0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b,
|
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a,
|
||||||
0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a,
|
0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01,
|
||||||
0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10,
|
0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
||||||
0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04,
|
0x49, 0x64, 0x22, 0x75, 0x0a, 0x17, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e,
|
||||||
0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63,
|
0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3d, 0x0a,
|
||||||
0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61,
|
0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e,
|
||||||
0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
|
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68,
|
||||||
0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f,
|
0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63,
|
||||||
0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77,
|
0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1b, 0x0a, 0x09,
|
||||||
0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44,
|
0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52,
|
||||||
0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44,
|
0x08, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x54, 0x73, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, 0x6e,
|
||||||
0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f,
|
0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, 0x73,
|
||||||
0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61,
|
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74,
|
||||||
0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63,
|
0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d,
|
||||||
0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c,
|
0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e,
|
||||||
0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01,
|
||||||
0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f,
|
0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, 0x6b,
|
||||||
0x74, 0x6f, 0x33,
|
0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65,
|
||||||
|
0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65,
|
||||||
|
0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e,
|
||||||
|
0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
|
||||||
|
0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f,
|
||||||
|
0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61,
|
||||||
|
0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18,
|
||||||
|
0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||||
|
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x4d,
|
||||||
|
0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, 0x50,
|
||||||
|
0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a,
|
||||||
|
0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72,
|
||||||
|
0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a,
|
||||||
|
0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12,
|
||||||
|
0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05,
|
||||||
|
0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, 0x72,
|
||||||
|
0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a,
|
||||||
|
0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
|
||||||
|
0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x64,
|
||||||
|
0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20,
|
||||||
|
0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x47,
|
||||||
|
0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18,
|
||||||
|
0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||||
|
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65,
|
||||||
|
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75,
|
||||||
|
0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75,
|
||||||
|
0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e,
|
||||||
|
0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
||||||
|
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52,
|
||||||
|
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x64,
|
||||||
|
0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01,
|
||||||
|
0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65,
|
||||||
|
0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x13, 0x0a, 0x05, 0x65, 0x7a, 0x5f, 0x69, 0x64,
|
||||||
|
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x65, 0x7a, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08,
|
||||||
|
0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07,
|
||||||
|
0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x79, 0x6c, 0x6f,
|
||||||
|
0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c,
|
||||||
|
0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x79, 0x74, 0x65, 0x73, 0x2a, 0x9a, 0x02, 0x0a,
|
||||||
|
0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07,
|
||||||
|
0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d,
|
||||||
|
0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72,
|
||||||
|
0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12,
|
||||||
|
0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72,
|
||||||
|
0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05,
|
||||||
|
0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||||
|
0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61,
|
||||||
|
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f,
|
||||||
|
0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b,
|
||||||
|
0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a,
|
||||||
|
0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a,
|
||||||
|
0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x10, 0x0b, 0x12, 0x10, 0x0a, 0x0c,
|
||||||
|
0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x10, 0x0c, 0x12, 0x0d,
|
||||||
|
0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a,
|
||||||
|
0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a,
|
||||||
|
0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12,
|
||||||
|
0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78,
|
||||||
|
0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b,
|
||||||
|
0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67,
|
||||||
|
0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69,
|
||||||
|
0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f,
|
||||||
|
0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d,
|
||||||
|
0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f,
|
||||||
|
0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54,
|
||||||
|
0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c,
|
||||||
|
0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e,
|
||||||
|
0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61,
|
||||||
|
0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52,
|
||||||
|
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70,
|
||||||
|
0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65,
|
||||||
|
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c,
|
||||||
|
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33,
|
||||||
|
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
||||||
|
0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f,
|
||||||
|
0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
|
||||||
|
0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||||
}
|
}
|
||||||
|
|
||||||
var (
|
var (
|
||||||
|
|||||||
@ -5,8 +5,10 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/walimplstest"
|
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/walimplstest"
|
||||||
)
|
)
|
||||||
@ -19,6 +21,7 @@ func TestMutableBuilder(t *testing.T) {
|
|||||||
MustBuildMutable()
|
MustBuildMutable()
|
||||||
assert.True(t, b.IsPersisted())
|
assert.True(t, b.IsPersisted())
|
||||||
assert.Equal(t, b.VChannel(), "")
|
assert.Equal(t, b.VChannel(), "")
|
||||||
|
log.Info("test", zap.Object("msg", b))
|
||||||
|
|
||||||
b = message.NewTimeTickMessageBuilderV1().
|
b = message.NewTimeTickMessageBuilderV1().
|
||||||
WithHeader(&message.TimeTickMessageHeader{}).
|
WithHeader(&message.TimeTickMessageHeader{}).
|
||||||
@ -28,6 +31,7 @@ func TestMutableBuilder(t *testing.T) {
|
|||||||
MustBuildMutable()
|
MustBuildMutable()
|
||||||
assert.False(t, b.IsPersisted())
|
assert.False(t, b.IsPersisted())
|
||||||
assert.Equal(t, b.VChannel(), "v1")
|
assert.Equal(t, b.VChannel(), "v1")
|
||||||
|
log.Info("test", zap.Object("msg", b))
|
||||||
|
|
||||||
assert.Panics(t, func() {
|
assert.Panics(t, func() {
|
||||||
message.NewCreateCollectionMessageBuilderV1().WithNotPersisted()
|
message.NewCreateCollectionMessageBuilderV1().WithNotPersisted()
|
||||||
@ -58,7 +62,9 @@ func TestImmutableTxnBuilder(t *testing.T) {
|
|||||||
WithBody(&msgpb.InsertRequest{}).
|
WithBody(&msgpb.InsertRequest{}).
|
||||||
WithVChannel("v1").
|
WithVChannel("v1").
|
||||||
MustBuildMutable()
|
MustBuildMutable()
|
||||||
immutableMsg := msg.WithTimeTick(2).WithTxnContext(txnCtx).WithLastConfirmed(msgID).IntoImmutableMessage(msgID)
|
mutableMsg := msg.WithTimeTick(2).WithTxnContext(txnCtx).WithLastConfirmed(msgID)
|
||||||
|
log.Info("test", zap.Object("msg", mutableMsg))
|
||||||
|
immutableMsg := mutableMsg.IntoImmutableMessage(msgID)
|
||||||
b.Add(immutableMsg)
|
b.Add(immutableMsg)
|
||||||
|
|
||||||
commit := message.NewCommitTxnMessageBuilderV2().
|
commit := message.NewCommitTxnMessageBuilderV2().
|
||||||
@ -67,10 +73,13 @@ func TestImmutableTxnBuilder(t *testing.T) {
|
|||||||
WithVChannel("v1").
|
WithVChannel("v1").
|
||||||
MustBuildMutable()
|
MustBuildMutable()
|
||||||
immutableCommit := commit.WithTimeTick(3).WithTxnContext(txnCtx).WithLastConfirmed(msgID).IntoImmutableMessage(msgID)
|
immutableCommit := commit.WithTimeTick(3).WithTxnContext(txnCtx).WithLastConfirmed(msgID).IntoImmutableMessage(msgID)
|
||||||
|
log.Info("test", zap.Object("msg", immutableCommit))
|
||||||
|
|
||||||
assert.NotZero(t, b.EstimateSize())
|
assert.NotZero(t, b.EstimateSize())
|
||||||
beginMsg, msgs := b.Messages()
|
beginMsg, msgs := b.Messages()
|
||||||
assert.NotEmpty(t, beginMsg)
|
assert.NotEmpty(t, beginMsg)
|
||||||
assert.Len(t, msgs, 1)
|
assert.Len(t, msgs, 1)
|
||||||
b.Build(message.MustAsImmutableCommitTxnMessageV2(immutableCommit))
|
immutableTxnMsg, err := b.Build(message.MustAsImmutableCommitTxnMessageV2(immutableCommit))
|
||||||
|
assert.NoError(t, err)
|
||||||
|
log.Info("test", zap.Object("msg", immutableTxnMsg))
|
||||||
}
|
}
|
||||||
|
|||||||
117
pkg/streaming/util/message/marshal_log_object.go
Normal file
117
pkg/streaming/util/message/marshal_log_object.go
Normal file
@ -0,0 +1,117 @@
|
|||||||
|
package message
|
||||||
|
|
||||||
|
import (
|
||||||
|
"reflect"
|
||||||
|
"strconv"
|
||||||
|
"strings"
|
||||||
|
|
||||||
|
"go.uber.org/zap/zapcore"
|
||||||
|
"google.golang.org/protobuf/proto"
|
||||||
|
)
|
||||||
|
|
||||||
|
// MarshalLogObject encodes the message into zap log object.
|
||||||
|
func (m *messageImpl) MarshalLogObject(enc zapcore.ObjectEncoder) error {
|
||||||
|
if m == nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
enc.AddString("type", m.MessageType().String())
|
||||||
|
enc.AddString("vchannel", m.VChannel())
|
||||||
|
if m.properties.Exist(messageTimeTick) {
|
||||||
|
enc.AddUint64("timetick", m.TimeTick())
|
||||||
|
}
|
||||||
|
if txn := m.TxnContext(); txn != nil {
|
||||||
|
enc.AddInt64("txnID", int64(txn.TxnID))
|
||||||
|
}
|
||||||
|
if broadcast := m.BroadcastHeader(); broadcast != nil {
|
||||||
|
enc.AddInt64("broadcastID", int64(broadcast.BroadcastID))
|
||||||
|
}
|
||||||
|
enc.AddInt("size", len(m.payload))
|
||||||
|
marshalSpecializedHeader(m.MessageType(), m.properties[messageHeader], enc)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// MarshalLogObject encodes the immutable message into zap log object.
|
||||||
|
func (m *immutableMessageImpl) MarshalLogObject(enc zapcore.ObjectEncoder) error {
|
||||||
|
if m == nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
enc.AddString("type", m.MessageType().String())
|
||||||
|
enc.AddString("vchannel", m.VChannel())
|
||||||
|
enc.AddUint64("timetick", m.TimeTick())
|
||||||
|
enc.AddString("messageID", m.MessageID().String())
|
||||||
|
enc.AddString("lastConfirmed", m.LastConfirmedMessageID().String())
|
||||||
|
if txn := m.TxnContext(); txn != nil {
|
||||||
|
enc.AddInt64("txnID", int64(txn.TxnID))
|
||||||
|
}
|
||||||
|
if broadcast := m.BroadcastHeader(); broadcast != nil {
|
||||||
|
enc.AddInt64("broadcastID", int64(broadcast.BroadcastID))
|
||||||
|
}
|
||||||
|
enc.AddInt("size", len(m.payload))
|
||||||
|
marshalSpecializedHeader(m.MessageType(), m.properties[messageHeader], enc)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *immutableTxnMessageImpl) MarshalLogObject(enc zapcore.ObjectEncoder) error {
|
||||||
|
if m == nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
enc.AddArray("txn", zapcore.ArrayMarshalerFunc(func(enc zapcore.ArrayEncoder) error {
|
||||||
|
txnMessage := AsImmutableTxnMessage(m)
|
||||||
|
txnMessage.RangeOver(func(im ImmutableMessage) error {
|
||||||
|
enc.AppendObject(im)
|
||||||
|
return nil
|
||||||
|
})
|
||||||
|
return nil
|
||||||
|
}))
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// marshalSpecializedHeader marshals the specialized header of the message.
|
||||||
|
func marshalSpecializedHeader(t MessageType, h string, enc zapcore.ObjectEncoder) {
|
||||||
|
typ := messageTypeToCustomHeaderMap[t]
|
||||||
|
// must be a proto type.
|
||||||
|
header := reflect.New(typ.Elem()).Interface().(proto.Message)
|
||||||
|
if err := DecodeProto(h, header); err != nil {
|
||||||
|
enc.AddString("headerDecodeError", err.Error())
|
||||||
|
return
|
||||||
|
}
|
||||||
|
switch header := header.(type) {
|
||||||
|
case *InsertMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
segmentIDs := make([]string, 0, len(header.GetPartitions()))
|
||||||
|
for _, partition := range header.GetPartitions() {
|
||||||
|
segmentIDs = append(segmentIDs, strconv.FormatInt(partition.GetSegmentAssignment().GetSegmentId(), 10))
|
||||||
|
}
|
||||||
|
enc.AddString("segmentIDs", strings.Join(segmentIDs, "|"))
|
||||||
|
case *DeleteMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
case *CreateCollectionMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
case *DropCollectionMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
case *CreatePartitionMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
enc.AddInt64("partitionID", header.GetPartitionId())
|
||||||
|
case *DropPartitionMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
enc.AddInt64("partitionID", header.GetPartitionId())
|
||||||
|
case *CreateSegmentMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
segmentIDs := make([]string, 0, len(header.GetSegmentIds()))
|
||||||
|
for _, segmentID := range header.GetSegmentIds() {
|
||||||
|
segmentIDs = append(segmentIDs, strconv.FormatInt(segmentID, 10))
|
||||||
|
}
|
||||||
|
enc.AddString("segmentIDs", strings.Join(segmentIDs, "|"))
|
||||||
|
case *FlushMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
segmentIDs := make([]string, 0, len(header.GetSegmentIds()))
|
||||||
|
for _, segmentID := range header.GetSegmentIds() {
|
||||||
|
segmentIDs = append(segmentIDs, strconv.FormatInt(segmentID, 10))
|
||||||
|
}
|
||||||
|
enc.AddString("segmentIDs", strings.Join(segmentIDs, "|"))
|
||||||
|
case *ManualFlushMessageHeader:
|
||||||
|
enc.AddInt64("collectionID", header.GetCollectionId())
|
||||||
|
case *SchemaChangeMessageHeader:
|
||||||
|
case *ImportMessageHeader:
|
||||||
|
}
|
||||||
|
}
|
||||||
@ -1,6 +1,7 @@
|
|||||||
package message
|
package message
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"go.uber.org/zap/zapcore"
|
||||||
"google.golang.org/protobuf/proto"
|
"google.golang.org/protobuf/proto"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -13,6 +14,8 @@ var (
|
|||||||
|
|
||||||
// BasicMessage is the basic interface of message.
|
// BasicMessage is the basic interface of message.
|
||||||
type BasicMessage interface {
|
type BasicMessage interface {
|
||||||
|
zapcore.ObjectMarshaler
|
||||||
|
|
||||||
// MessageType returns the type of message.
|
// MessageType returns the type of message.
|
||||||
MessageType() MessageType
|
MessageType() MessageType
|
||||||
|
|
||||||
|
|||||||
@ -67,6 +67,26 @@ var messageTypeMap = map[reflect.Type]MessageType{
|
|||||||
reflect.TypeOf(&SchemaChangeMessageHeader{}): MessageTypeSchemaChange,
|
reflect.TypeOf(&SchemaChangeMessageHeader{}): MessageTypeSchemaChange,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// messageTypeToCustomHeaderMap maps the message type to the proto message type.
|
||||||
|
var messageTypeToCustomHeaderMap = map[MessageType]reflect.Type{
|
||||||
|
MessageTypeTimeTick: reflect.TypeOf(&TimeTickMessageHeader{}),
|
||||||
|
MessageTypeInsert: reflect.TypeOf(&InsertMessageHeader{}),
|
||||||
|
MessageTypeDelete: reflect.TypeOf(&DeleteMessageHeader{}),
|
||||||
|
MessageTypeCreateCollection: reflect.TypeOf(&CreateCollectionMessageHeader{}),
|
||||||
|
MessageTypeDropCollection: reflect.TypeOf(&DropCollectionMessageHeader{}),
|
||||||
|
MessageTypeCreatePartition: reflect.TypeOf(&CreatePartitionMessageHeader{}),
|
||||||
|
MessageTypeDropPartition: reflect.TypeOf(&DropPartitionMessageHeader{}),
|
||||||
|
MessageTypeCreateSegment: reflect.TypeOf(&CreateSegmentMessageHeader{}),
|
||||||
|
MessageTypeFlush: reflect.TypeOf(&FlushMessageHeader{}),
|
||||||
|
MessageTypeManualFlush: reflect.TypeOf(&ManualFlushMessageHeader{}),
|
||||||
|
MessageTypeBeginTxn: reflect.TypeOf(&BeginTxnMessageHeader{}),
|
||||||
|
MessageTypeCommitTxn: reflect.TypeOf(&CommitTxnMessageHeader{}),
|
||||||
|
MessageTypeRollbackTxn: reflect.TypeOf(&RollbackTxnMessageHeader{}),
|
||||||
|
MessageTypeTxn: reflect.TypeOf(&TxnMessageHeader{}),
|
||||||
|
MessageTypeImport: reflect.TypeOf(&ImportMessageHeader{}),
|
||||||
|
MessageTypeSchemaChange: reflect.TypeOf(&SchemaChangeMessageHeader{}),
|
||||||
|
}
|
||||||
|
|
||||||
// A system preserved message, should not allowed to provide outside of the streaming system.
|
// A system preserved message, should not allowed to provide outside of the streaming system.
|
||||||
var systemMessageType = map[MessageType]struct{}{
|
var systemMessageType = map[MessageType]struct{}{
|
||||||
MessageTypeTimeTick: {},
|
MessageTypeTimeTick: {},
|
||||||
@ -259,7 +279,6 @@ func asSpecializedImmutableMessage[H proto.Message, B proto.Message](msg Immutab
|
|||||||
// Decode the specialized header.
|
// Decode the specialized header.
|
||||||
// Must be pointer type.
|
// Must be pointer type.
|
||||||
t := reflect.TypeOf(header)
|
t := reflect.TypeOf(header)
|
||||||
t.Elem()
|
|
||||||
header = reflect.New(t.Elem()).Interface().(H)
|
header = reflect.New(t.Elem()).Interface().(H)
|
||||||
|
|
||||||
// must be a pointer to a proto message
|
// must be a pointer to a proto message
|
||||||
|
|||||||
@ -21,14 +21,14 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
walName = "woodpecker"
|
WALName = "woodpecker"
|
||||||
)
|
)
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
// register the builder to the wal registry.
|
// register the builder to the wal registry.
|
||||||
registry.RegisterBuilder(&builderImpl{})
|
registry.RegisterBuilder(&builderImpl{})
|
||||||
// register the unmarshaler to the message registry.
|
// register the unmarshaler to the message registry.
|
||||||
message.RegisterMessageIDUnmsarshaler(walName, UnmarshalMessageID)
|
message.RegisterMessageIDUnmsarshaler(WALName, UnmarshalMessageID)
|
||||||
}
|
}
|
||||||
|
|
||||||
// builderImpl is the builder for woodpecker opener.
|
// builderImpl is the builder for woodpecker opener.
|
||||||
@ -36,7 +36,7 @@ type builderImpl struct{}
|
|||||||
|
|
||||||
// Name of the wal builder, should be a lowercase string.
|
// Name of the wal builder, should be a lowercase string.
|
||||||
func (b *builderImpl) Name() string {
|
func (b *builderImpl) Name() string {
|
||||||
return walName
|
return WALName
|
||||||
}
|
}
|
||||||
|
|
||||||
// Build build a wal instance.
|
// Build build a wal instance.
|
||||||
|
|||||||
@ -51,7 +51,7 @@ func (id wpID) WoodpeckerMsgId() *wp.LogMessageId {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (id wpID) WALName() string {
|
func (id wpID) WALName() string {
|
||||||
return walName
|
return WALName
|
||||||
}
|
}
|
||||||
|
|
||||||
func (id wpID) LT(other message.MessageID) bool {
|
func (id wpID) LT(other message.MessageID) bool {
|
||||||
|
|||||||
@ -14,7 +14,7 @@ func TestMessageID(t *testing.T) {
|
|||||||
wpId := message.MessageID(newMessageIDOfWoodpecker(1, 2)).(interface {
|
wpId := message.MessageID(newMessageIDOfWoodpecker(1, 2)).(interface {
|
||||||
WoodpeckerID() *woodpecker.LogMessageId
|
WoodpeckerID() *woodpecker.LogMessageId
|
||||||
}).WoodpeckerID()
|
}).WoodpeckerID()
|
||||||
assert.Equal(t, walName, newMessageIDOfWoodpecker(1, 2).WALName())
|
assert.Equal(t, WALName, newMessageIDOfWoodpecker(1, 2).WALName())
|
||||||
|
|
||||||
assert.Equal(t, int64(1), wpId.SegmentId)
|
assert.Equal(t, int64(1), wpId.SegmentId)
|
||||||
assert.Equal(t, int64(2), wpId.EntryId)
|
assert.Equal(t, int64(2), wpId.EntryId)
|
||||||
|
|||||||
@ -21,7 +21,7 @@ type walImpl struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (w *walImpl) WALName() string {
|
func (w *walImpl) WALName() string {
|
||||||
return walName
|
return WALName
|
||||||
}
|
}
|
||||||
|
|
||||||
func (w *walImpl) Append(ctx context.Context, msg message.MutableMessage) (message.MessageID, error) {
|
func (w *walImpl) Append(ctx context.Context, msg message.MutableMessage) (message.MessageID, error) {
|
||||||
|
|||||||
@ -17,11 +17,11 @@ func TestMain(m *testing.M) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func TestRegistry(t *testing.T) {
|
func TestRegistry(t *testing.T) {
|
||||||
registeredB := registry.MustGetBuilder(walName)
|
registeredB := registry.MustGetBuilder(WALName)
|
||||||
assert.NotNil(t, registeredB)
|
assert.NotNil(t, registeredB)
|
||||||
assert.Equal(t, walName, registeredB.Name())
|
assert.Equal(t, WALName, registeredB.Name())
|
||||||
|
|
||||||
id, err := message.UnmarshalMessageID(walName, newMessageIDOfWoodpecker(1, 2).Marshal())
|
id, err := message.UnmarshalMessageID(WALName, newMessageIDOfWoodpecker(1, 2).Marshal())
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.True(t, id.EQ(newMessageIDOfWoodpecker(1, 2)))
|
assert.True(t, id.EQ(newMessageIDOfWoodpecker(1, 2)))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -5245,6 +5245,9 @@ type streamingConfig struct {
|
|||||||
// write ahead buffer
|
// write ahead buffer
|
||||||
WALWriteAheadBufferCapacity ParamItem `refreshable:"true"`
|
WALWriteAheadBufferCapacity ParamItem `refreshable:"true"`
|
||||||
WALWriteAheadBufferKeepalive ParamItem `refreshable:"true"`
|
WALWriteAheadBufferKeepalive ParamItem `refreshable:"true"`
|
||||||
|
|
||||||
|
// logging
|
||||||
|
LoggingAppendSlowThreshold ParamItem `refreshable:"true"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *streamingConfig) init(base *BaseTable) {
|
func (p *streamingConfig) init(base *BaseTable) {
|
||||||
@ -5372,6 +5375,16 @@ it also determine the depth of depth first search method that is used to find th
|
|||||||
Export: true,
|
Export: true,
|
||||||
}
|
}
|
||||||
p.WALWriteAheadBufferKeepalive.Init(base.mgr)
|
p.WALWriteAheadBufferKeepalive.Init(base.mgr)
|
||||||
|
|
||||||
|
p.LoggingAppendSlowThreshold = ParamItem{
|
||||||
|
Key: "streaming.logging.appendSlowThreshold",
|
||||||
|
Version: "2.6.0",
|
||||||
|
Doc: `The threshold of slow log, 1s by default.
|
||||||
|
If the wal implementation is woodpecker, the minimum threshold is 3s`,
|
||||||
|
DefaultValue: "1s",
|
||||||
|
Export: true,
|
||||||
|
}
|
||||||
|
p.LoggingAppendSlowThreshold.Init(base.mgr)
|
||||||
}
|
}
|
||||||
|
|
||||||
// runtimeConfig is just a private environment value table.
|
// runtimeConfig is just a private environment value table.
|
||||||
|
|||||||
@ -626,6 +626,7 @@ func TestComponentParam(t *testing.T) {
|
|||||||
assert.Equal(t, 10*time.Second, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse())
|
assert.Equal(t, 10*time.Second, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse())
|
||||||
assert.Equal(t, 30*time.Second, params.StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse())
|
assert.Equal(t, 30*time.Second, params.StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse())
|
||||||
assert.Equal(t, int64(64*1024*1024), params.StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
|
assert.Equal(t, int64(64*1024*1024), params.StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
|
||||||
|
assert.Equal(t, 1*time.Second, params.StreamingCfg.LoggingAppendSlowThreshold.GetAsDurationByParse())
|
||||||
params.Save(params.StreamingCfg.WALBalancerTriggerInterval.Key, "50s")
|
params.Save(params.StreamingCfg.WALBalancerTriggerInterval.Key, "50s")
|
||||||
params.Save(params.StreamingCfg.WALBalancerBackoffInitialInterval.Key, "50s")
|
params.Save(params.StreamingCfg.WALBalancerBackoffInitialInterval.Key, "50s")
|
||||||
params.Save(params.StreamingCfg.WALBalancerBackoffMultiplier.Key, "3.5")
|
params.Save(params.StreamingCfg.WALBalancerBackoffMultiplier.Key, "3.5")
|
||||||
@ -639,6 +640,7 @@ func TestComponentParam(t *testing.T) {
|
|||||||
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.Key, "0.02")
|
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.Key, "0.02")
|
||||||
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceTolerance.Key, "0.02")
|
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceTolerance.Key, "0.02")
|
||||||
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceMaxStep.Key, "4")
|
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceMaxStep.Key, "4")
|
||||||
|
params.Save(params.StreamingCfg.LoggingAppendSlowThreshold.Key, "3s")
|
||||||
assert.Equal(t, 50*time.Second, params.StreamingCfg.WALBalancerTriggerInterval.GetAsDurationByParse())
|
assert.Equal(t, 50*time.Second, params.StreamingCfg.WALBalancerTriggerInterval.GetAsDurationByParse())
|
||||||
assert.Equal(t, 50*time.Second, params.StreamingCfg.WALBalancerBackoffInitialInterval.GetAsDurationByParse())
|
assert.Equal(t, 50*time.Second, params.StreamingCfg.WALBalancerBackoffInitialInterval.GetAsDurationByParse())
|
||||||
assert.Equal(t, 3.5, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat())
|
assert.Equal(t, 3.5, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat())
|
||||||
@ -652,6 +654,7 @@ func TestComponentParam(t *testing.T) {
|
|||||||
assert.Equal(t, 3500*time.Millisecond, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse())
|
assert.Equal(t, 3500*time.Millisecond, params.StreamingCfg.TxnDefaultKeepaliveTimeout.GetAsDurationByParse())
|
||||||
assert.Equal(t, 10*time.Second, params.StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse())
|
assert.Equal(t, 10*time.Second, params.StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse())
|
||||||
assert.Equal(t, int64(128*1024), params.StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
|
assert.Equal(t, int64(128*1024), params.StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize())
|
||||||
|
assert.Equal(t, 3*time.Second, params.StreamingCfg.LoggingAppendSlowThreshold.GetAsDurationByParse())
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("channel config priority", func(t *testing.T) {
|
t.Run("channel config priority", func(t *testing.T) {
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user