feat: storage v2 sync (#39663)

related: #39173

Signed-off-by: shaoting-huang <shaoting.huang@zilliz.com>
pull/40373/head
sthuang 2025-03-05 11:22:15 +08:00 committed by GitHub
parent a3755cf409
commit 63a7c4570e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
32 changed files with 3105 additions and 2435 deletions

View File

@ -21,9 +21,9 @@ func (_m *MockManager) EXPECT() *MockManager_Expecter {
return &MockManager_Expecter{mock: &_m.Mock}
}
// AllocNewGrowingSegment provides a mock function with given fields: ctx, collectionID, partitionID, segmentID, channelName
func (_m *MockManager) AllocNewGrowingSegment(ctx context.Context, collectionID int64, partitionID int64, segmentID int64, channelName string) (*SegmentInfo, error) {
ret := _m.Called(ctx, collectionID, partitionID, segmentID, channelName)
// AllocNewGrowingSegment provides a mock function with given fields: ctx, collectionID, partitionID, segmentID, channelName, storageVersion
func (_m *MockManager) AllocNewGrowingSegment(ctx context.Context, collectionID int64, partitionID int64, segmentID int64, channelName string, storageVersion int64) (*SegmentInfo, error) {
ret := _m.Called(ctx, collectionID, partitionID, segmentID, channelName, storageVersion)
if len(ret) == 0 {
panic("no return value specified for AllocNewGrowingSegment")
@ -31,19 +31,19 @@ func (_m *MockManager) AllocNewGrowingSegment(ctx context.Context, collectionID
var r0 *SegmentInfo
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, int64, string) (*SegmentInfo, error)); ok {
return rf(ctx, collectionID, partitionID, segmentID, channelName)
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, int64, string, int64) (*SegmentInfo, error)); ok {
return rf(ctx, collectionID, partitionID, segmentID, channelName, storageVersion)
}
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, int64, string) *SegmentInfo); ok {
r0 = rf(ctx, collectionID, partitionID, segmentID, channelName)
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, int64, string, int64) *SegmentInfo); ok {
r0 = rf(ctx, collectionID, partitionID, segmentID, channelName, storageVersion)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*SegmentInfo)
}
}
if rf, ok := ret.Get(1).(func(context.Context, int64, int64, int64, string) error); ok {
r1 = rf(ctx, collectionID, partitionID, segmentID, channelName)
if rf, ok := ret.Get(1).(func(context.Context, int64, int64, int64, string, int64) error); ok {
r1 = rf(ctx, collectionID, partitionID, segmentID, channelName, storageVersion)
} else {
r1 = ret.Error(1)
}
@ -62,13 +62,14 @@ type MockManager_AllocNewGrowingSegment_Call struct {
// - partitionID int64
// - segmentID int64
// - channelName string
func (_e *MockManager_Expecter) AllocNewGrowingSegment(ctx interface{}, collectionID interface{}, partitionID interface{}, segmentID interface{}, channelName interface{}) *MockManager_AllocNewGrowingSegment_Call {
return &MockManager_AllocNewGrowingSegment_Call{Call: _e.mock.On("AllocNewGrowingSegment", ctx, collectionID, partitionID, segmentID, channelName)}
// - storageVersion int64
func (_e *MockManager_Expecter) AllocNewGrowingSegment(ctx interface{}, collectionID interface{}, partitionID interface{}, segmentID interface{}, channelName interface{}, storageVersion interface{}) *MockManager_AllocNewGrowingSegment_Call {
return &MockManager_AllocNewGrowingSegment_Call{Call: _e.mock.On("AllocNewGrowingSegment", ctx, collectionID, partitionID, segmentID, channelName, storageVersion)}
}
func (_c *MockManager_AllocNewGrowingSegment_Call) Run(run func(ctx context.Context, collectionID int64, partitionID int64, segmentID int64, channelName string)) *MockManager_AllocNewGrowingSegment_Call {
func (_c *MockManager_AllocNewGrowingSegment_Call) Run(run func(ctx context.Context, collectionID int64, partitionID int64, segmentID int64, channelName string, storageVersion int64)) *MockManager_AllocNewGrowingSegment_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(int64), args[2].(int64), args[3].(int64), args[4].(string))
run(args[0].(context.Context), args[1].(int64), args[2].(int64), args[3].(int64), args[4].(string), args[5].(int64))
})
return _c
}
@ -78,14 +79,14 @@ func (_c *MockManager_AllocNewGrowingSegment_Call) Return(_a0 *SegmentInfo, _a1
return _c
}
func (_c *MockManager_AllocNewGrowingSegment_Call) RunAndReturn(run func(context.Context, int64, int64, int64, string) (*SegmentInfo, error)) *MockManager_AllocNewGrowingSegment_Call {
func (_c *MockManager_AllocNewGrowingSegment_Call) RunAndReturn(run func(context.Context, int64, int64, int64, string, int64) (*SegmentInfo, error)) *MockManager_AllocNewGrowingSegment_Call {
_c.Call.Return(run)
return _c
}
// AllocSegment provides a mock function with given fields: ctx, collectionID, partitionID, channelName, requestRows
func (_m *MockManager) AllocSegment(ctx context.Context, collectionID int64, partitionID int64, channelName string, requestRows int64) ([]*Allocation, error) {
ret := _m.Called(ctx, collectionID, partitionID, channelName, requestRows)
// AllocSegment provides a mock function with given fields: ctx, collectionID, partitionID, channelName, requestRows, storageVersion
func (_m *MockManager) AllocSegment(ctx context.Context, collectionID int64, partitionID int64, channelName string, requestRows int64, storageVersion int64) ([]*Allocation, error) {
ret := _m.Called(ctx, collectionID, partitionID, channelName, requestRows, storageVersion)
if len(ret) == 0 {
panic("no return value specified for AllocSegment")
@ -93,19 +94,19 @@ func (_m *MockManager) AllocSegment(ctx context.Context, collectionID int64, par
var r0 []*Allocation
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, string, int64) ([]*Allocation, error)); ok {
return rf(ctx, collectionID, partitionID, channelName, requestRows)
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, string, int64, int64) ([]*Allocation, error)); ok {
return rf(ctx, collectionID, partitionID, channelName, requestRows, storageVersion)
}
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, string, int64) []*Allocation); ok {
r0 = rf(ctx, collectionID, partitionID, channelName, requestRows)
if rf, ok := ret.Get(0).(func(context.Context, int64, int64, string, int64, int64) []*Allocation); ok {
r0 = rf(ctx, collectionID, partitionID, channelName, requestRows, storageVersion)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]*Allocation)
}
}
if rf, ok := ret.Get(1).(func(context.Context, int64, int64, string, int64) error); ok {
r1 = rf(ctx, collectionID, partitionID, channelName, requestRows)
if rf, ok := ret.Get(1).(func(context.Context, int64, int64, string, int64, int64) error); ok {
r1 = rf(ctx, collectionID, partitionID, channelName, requestRows, storageVersion)
} else {
r1 = ret.Error(1)
}
@ -124,13 +125,14 @@ type MockManager_AllocSegment_Call struct {
// - partitionID int64
// - channelName string
// - requestRows int64
func (_e *MockManager_Expecter) AllocSegment(ctx interface{}, collectionID interface{}, partitionID interface{}, channelName interface{}, requestRows interface{}) *MockManager_AllocSegment_Call {
return &MockManager_AllocSegment_Call{Call: _e.mock.On("AllocSegment", ctx, collectionID, partitionID, channelName, requestRows)}
// - storageVersion int64
func (_e *MockManager_Expecter) AllocSegment(ctx interface{}, collectionID interface{}, partitionID interface{}, channelName interface{}, requestRows interface{}, storageVersion interface{}) *MockManager_AllocSegment_Call {
return &MockManager_AllocSegment_Call{Call: _e.mock.On("AllocSegment", ctx, collectionID, partitionID, channelName, requestRows, storageVersion)}
}
func (_c *MockManager_AllocSegment_Call) Run(run func(ctx context.Context, collectionID int64, partitionID int64, channelName string, requestRows int64)) *MockManager_AllocSegment_Call {
func (_c *MockManager_AllocSegment_Call) Run(run func(ctx context.Context, collectionID int64, partitionID int64, channelName string, requestRows int64, storageVersion int64)) *MockManager_AllocSegment_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(int64), args[2].(int64), args[3].(string), args[4].(int64))
run(args[0].(context.Context), args[1].(int64), args[2].(int64), args[3].(string), args[4].(int64), args[5].(int64))
})
return _c
}
@ -140,7 +142,7 @@ func (_c *MockManager_AllocSegment_Call) Return(_a0 []*Allocation, _a1 error) *M
return _c
}
func (_c *MockManager_AllocSegment_Call) RunAndReturn(run func(context.Context, int64, int64, string, int64) ([]*Allocation, error)) *MockManager_AllocSegment_Call {
func (_c *MockManager_AllocSegment_Call) RunAndReturn(run func(context.Context, int64, int64, string, int64, int64) ([]*Allocation, error)) *MockManager_AllocSegment_Call {
_c.Call.Return(run)
return _c
}

View File

@ -74,10 +74,10 @@ type Manager interface {
// CreateSegment create new segment when segment not exist
// Deprecated: AllocSegment allocates rows and record the allocation, will be deprecated after enabling streamingnode.
AllocSegment(ctx context.Context, collectionID, partitionID UniqueID, channelName string, requestRows int64) ([]*Allocation, error)
AllocSegment(ctx context.Context, collectionID, partitionID UniqueID, channelName string, requestRows int64, storageVersion int64) ([]*Allocation, error)
// AllocNewGrowingSegment allocates segment for streaming node.
AllocNewGrowingSegment(ctx context.Context, collectionID, partitionID, segmentID UniqueID, channelName string) (*SegmentInfo, error)
AllocNewGrowingSegment(ctx context.Context, collectionID, partitionID, segmentID UniqueID, channelName string, storageVersion int64) (*SegmentInfo, error)
// DropSegment drops the segment from manager.
DropSegment(ctx context.Context, channel string, segmentID UniqueID)
@ -287,7 +287,7 @@ func (s *SegmentManager) genLastExpireTsForSegments() (Timestamp, error) {
// AllocSegment allocate segment per request collcation, partication, channel and rows
func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID,
partitionID UniqueID, channelName string, requestRows int64,
partitionID UniqueID, channelName string, requestRows int64, storageVersion int64,
) ([]*Allocation, error) {
log := log.Ctx(ctx).
With(zap.Int64("collectionID", collectionID)).
@ -331,7 +331,7 @@ func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID
return nil, err
}
for _, allocation := range newSegmentAllocations {
segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName)
segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName, storageVersion)
if err != nil {
log.Error("Failed to open new segment for segment allocation")
return nil, err
@ -367,13 +367,13 @@ func (s *SegmentManager) genExpireTs(ctx context.Context) (Timestamp, error) {
}
// AllocNewGrowingSegment allocates segment for streaming node.
func (s *SegmentManager) AllocNewGrowingSegment(ctx context.Context, collectionID, partitionID, segmentID UniqueID, channelName string) (*SegmentInfo, error) {
func (s *SegmentManager) AllocNewGrowingSegment(ctx context.Context, collectionID, partitionID, segmentID UniqueID, channelName string, storageVersion int64) (*SegmentInfo, error) {
s.channelLock.Lock(channelName)
defer s.channelLock.Unlock(channelName)
return s.openNewSegmentWithGivenSegmentID(ctx, collectionID, partitionID, segmentID, channelName)
return s.openNewSegmentWithGivenSegmentID(ctx, collectionID, partitionID, segmentID, channelName, storageVersion)
}
func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID UniqueID, partitionID UniqueID, channelName string) (*SegmentInfo, error) {
func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID UniqueID, partitionID UniqueID, channelName string, storageVersion int64) (*SegmentInfo, error) {
log := log.Ctx(ctx)
ctx, sp := otel.Tracer(typeutil.DataCoordRole).Start(ctx, "open-Segment")
defer sp.End()
@ -382,10 +382,10 @@ func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID Unique
log.Error("failed to open new segment while AllocID", zap.Error(err))
return nil, err
}
return s.openNewSegmentWithGivenSegmentID(ctx, collectionID, partitionID, id, channelName)
return s.openNewSegmentWithGivenSegmentID(ctx, collectionID, partitionID, id, channelName, storageVersion)
}
func (s *SegmentManager) openNewSegmentWithGivenSegmentID(ctx context.Context, collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, channelName string) (*SegmentInfo, error) {
func (s *SegmentManager) openNewSegmentWithGivenSegmentID(ctx context.Context, collectionID UniqueID, partitionID UniqueID, segmentID UniqueID, channelName string, storageVersion int64) (*SegmentInfo, error) {
maxNumOfRows, err := s.estimateMaxNumOfRows(collectionID)
if err != nil {
log.Error("failed to open new segment while estimateMaxNumOfRows", zap.Error(err))
@ -402,6 +402,7 @@ func (s *SegmentManager) openNewSegmentWithGivenSegmentID(ctx context.Context, c
MaxRowNum: int64(maxNumOfRows),
Level: datapb.SegmentLevel_L1,
LastExpireTime: 0,
StorageVersion: storageVersion,
}
segment := NewSegmentInfo(segmentInfo)
if err := s.meta.AddSegment(ctx, segment); err != nil {

View File

@ -37,6 +37,7 @@ import (
mockkv "github.com/milvus-io/milvus/internal/kv/mocks"
"github.com/milvus-io/milvus/internal/metastore/kv/datacoord"
"github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/rootcoordpb"
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
@ -121,7 +122,7 @@ func TestAllocSegment(t *testing.T) {
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
t.Run("normal allocation", func(t *testing.T) {
allocations, err := segmentManager.AllocSegment(ctx, collID, 100, "c1", 100)
allocations, err := segmentManager.AllocSegment(ctx, collID, 100, "c1", 100, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
assert.EqualValues(t, 100, allocations[0].NumOfRows)
@ -139,7 +140,7 @@ func TestAllocSegment(t *testing.T) {
// }
segmentManager, err := newSegmentManager(meta, failsAllocator)
assert.NoError(t, err)
_, err = segmentManager.AllocSegment(ctx, collID, 100, "c2", 100)
_, err = segmentManager.AllocSegment(ctx, collID, 100, "c2", 100, storage.StorageV1)
assert.Error(t, err)
})
@ -155,7 +156,7 @@ func TestAllocSegment(t *testing.T) {
t.Run("alloc clear unhealthy segment", func(t *testing.T) {
vchannel := "c1"
partitionID := int64(100)
allocations1, err := segmentManager.AllocSegment(ctx, collID, partitionID, vchannel, 100)
allocations1, err := segmentManager.AllocSegment(ctx, collID, partitionID, vchannel, 100, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations1))
segments, ok := segmentManager.channel2Growing.Get(vchannel)
@ -165,7 +166,7 @@ func TestAllocSegment(t *testing.T) {
err = meta.SetState(context.TODO(), allocations1[0].SegmentID, commonpb.SegmentState_Dropped)
assert.NoError(t, err)
allocations2, err := segmentManager.AllocSegment(ctx, collID, partitionID, vchannel, 100)
allocations2, err := segmentManager.AllocSegment(ctx, collID, partitionID, vchannel, 100, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations2))
// clear old healthy and alloc new
@ -232,13 +233,13 @@ func TestLastExpireReset(t *testing.T) {
segmentManager, _ := newSegmentManager(meta, mockAllocator)
initSegment.SegmentInfo.State = commonpb.SegmentState_Dropped
meta.segments.SetSegment(1, initSegment)
allocs, _ := segmentManager.AllocSegment(context.Background(), collID, 0, channelName, bigRows)
allocs, _ := segmentManager.AllocSegment(context.Background(), collID, 0, channelName, bigRows, storage.StorageV1)
segmentID1, expire1 := allocs[0].SegmentID, allocs[0].ExpireTime
time.Sleep(100 * time.Millisecond)
allocs, _ = segmentManager.AllocSegment(context.Background(), collID, 0, channelName, bigRows)
allocs, _ = segmentManager.AllocSegment(context.Background(), collID, 0, channelName, bigRows, storage.StorageV1)
segmentID2, expire2 := allocs[0].SegmentID, allocs[0].ExpireTime
time.Sleep(100 * time.Millisecond)
allocs, _ = segmentManager.AllocSegment(context.Background(), collID, 0, channelName, smallRows)
allocs, _ = segmentManager.AllocSegment(context.Background(), collID, 0, channelName, smallRows, storage.StorageV1)
segmentID3, expire3 := allocs[0].SegmentID, allocs[0].ExpireTime
// simulate handleTimeTick op on dataCoord
@ -282,7 +283,7 @@ func TestLastExpireReset(t *testing.T) {
assert.True(t, segment3.GetLastExpireTime() > expire3)
flushableSegIDs, _ := newSegmentManager.GetFlushableSegments(context.Background(), channelName, expire3)
assert.ElementsMatch(t, []UniqueID{segmentID1, segmentID2}, flushableSegIDs) // segment1 and segment2 can be flushed
newAlloc, err := newSegmentManager.AllocSegment(context.Background(), collID, 0, channelName, 2000)
newAlloc, err := newSegmentManager.AllocSegment(context.Background(), collID, 0, channelName, 2000, storage.StorageV1)
assert.Nil(t, err)
assert.Equal(t, segmentID3, newAlloc[0].SegmentID) // segment3 still can be used to allocate
}
@ -357,7 +358,7 @@ func TestSaveSegmentsToMeta(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator)
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 0, "c1", 1000)
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 0, "c1", 1000, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
_, err = segmentManager.SealAllSegments(context.Background(), "c1", nil)
@ -379,7 +380,7 @@ func TestSaveSegmentsToMetaWithSpecificSegments(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator)
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 0, "c1", 1000)
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 0, "c1", 1000, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
_, err = segmentManager.SealAllSegments(context.Background(), "c1", []int64{allocations[0].SegmentID})
@ -401,7 +402,7 @@ func TestDropSegment(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator)
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 100, "c1", 1000)
allocations, err := segmentManager.AllocSegment(context.Background(), collID, 100, "c1", 1000, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
segID := allocations[0].SegmentID
@ -428,7 +429,7 @@ func TestAllocRowsLargerThanOneSegment(t *testing.T) {
return 1, nil
}
segmentManager, _ := newSegmentManager(meta, mockAllocator, withCalUpperLimitPolicy(mockPolicy))
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 2, len(allocations))
assert.EqualValues(t, 1, allocations[0].NumOfRows)
@ -454,7 +455,7 @@ func TestExpireAllocation(t *testing.T) {
var maxts Timestamp
var id int64 = -1
for i := 0; i < 100; i++ {
allocs, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "ch1", 100)
allocs, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "ch1", 100, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocs))
if id == -1 {
@ -488,7 +489,7 @@ func TestGetFlushableSegments(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
@ -552,7 +553,7 @@ func TestTryToSealSegment(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator, withSegmentSealPolices(sealL1SegmentByLifetime())) // always seal
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
@ -579,7 +580,7 @@ func TestTryToSealSegment(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator, withChannelSealPolices(getChannelOpenSegCapacityPolicy(-1))) // always seal
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
@ -606,7 +607,7 @@ func TestTryToSealSegment(t *testing.T) {
segmentManager, _ := newSegmentManager(meta, mockAllocator,
withSegmentSealPolices(sealL1SegmentByLifetime()),
withChannelSealPolices(getChannelOpenSegCapacityPolicy(-1))) // always seal
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
@ -631,7 +632,7 @@ func TestTryToSealSegment(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
@ -716,7 +717,7 @@ func TestTryToSealSegment(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator, withSegmentSealPolices(alwaysSealPolicy())) // always seal
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))
@ -747,7 +748,7 @@ func TestTryToSealSegment(t *testing.T) {
assert.NoError(t, err)
meta.AddCollection(&collectionInfo{ID: collID, Schema: schema})
segmentManager, _ := newSegmentManager(meta, mockAllocator, withChannelSealPolices(getChannelOpenSegCapacityPolicy(-1))) // always seal
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2)
allocations, err := segmentManager.AllocSegment(context.TODO(), collID, 0, "c1", 2, storage.StorageV1)
assert.NoError(t, err)
assert.EqualValues(t, 1, len(allocations))

View File

@ -223,7 +223,7 @@ func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
// Have segment manager allocate and return the segment allocation info.
segmentAllocations, err := s.segmentManager.AllocSegment(ctx,
r.CollectionID, r.PartitionID, r.ChannelName, int64(r.Count))
r.CollectionID, r.PartitionID, r.ChannelName, int64(r.Count), r.GetStorageVersion())
if err != nil {
log.Warn("failed to alloc segment", zap.Any("request", r), zap.Error(err))
assigns = append(assigns, &datapb.SegmentIDAssignment{
@ -273,7 +273,7 @@ func (s *Server) AllocSegment(ctx context.Context, req *datapb.AllocSegmentReque
}
// Alloc new growing segment and return the segment info.
segmentInfo, err := s.segmentManager.AllocNewGrowingSegment(ctx, req.GetCollectionId(), req.GetPartitionId(), req.GetSegmentId(), req.GetVchannel())
segmentInfo, err := s.segmentManager.AllocNewGrowingSegment(ctx, req.GetCollectionId(), req.GetPartitionId(), req.GetSegmentId(), req.GetVchannel(), req.GetStorageVersion())
if err != nil {
return &datapb.AllocSegmentResponse{Status: merr.Status(err)}, nil
}

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/metastore/model"
mocks2 "github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
@ -502,13 +503,13 @@ func (s *ServerSuite) TestFlush_NormalCase() {
schema := newTestSchema()
s.testServer.meta.AddCollection(&collectionInfo{ID: 0, Schema: schema, Partitions: []int64{}, VChannelNames: []string{"channel-1"}})
allocations, err := s.testServer.segmentManager.AllocSegment(context.TODO(), 0, 1, "channel-1", 1)
allocations, err := s.testServer.segmentManager.AllocSegment(context.TODO(), 0, 1, "channel-1", 1, storage.StorageV1)
s.NoError(err)
s.EqualValues(1, len(allocations))
expireTs := allocations[0].ExpireTime
segID := allocations[0].SegmentID
info, err := s.testServer.segmentManager.AllocNewGrowingSegment(context.TODO(), 0, 1, 1, "channel-1")
info, err := s.testServer.segmentManager.AllocNewGrowingSegment(context.TODO(), 0, 1, 1, "channel-1", storage.StorageV1)
s.NoError(err)
s.NotNil(info)

View File

@ -38,6 +38,7 @@ type SegmentInfo struct {
bm25stats *SegmentBM25Stats
level datapb.SegmentLevel
syncingTasks int32
storageVersion int64
}
func (s *SegmentInfo) SegmentID() int64 {
@ -95,6 +96,10 @@ func (s *SegmentInfo) SyncingRows() int64 {
return s.syncingRows
}
func (s *SegmentInfo) GetStorageVersion() int64 {
return s.storageVersion
}
func (s *SegmentInfo) Clone() *SegmentInfo {
return &SegmentInfo{
segmentID: s.segmentID,
@ -129,5 +134,6 @@ func NewSegmentInfo(info *datapb.SegmentInfo, bfs pkoracle.PkStat, bm25Stats *Se
level: level,
bfs: bfs,
bm25stats: bm25Stats,
storageVersion: info.GetStorageVersion(),
}
}

View File

@ -66,3 +66,13 @@ func (t *SyncTask) WithFailureCallback(callback func(error)) *SyncTask {
t.failureCallback = callback
return t
}
func (t *SyncTask) WithSyncBufferSize(size int64) *SyncTask {
t.syncBufferSize = size
return t
}
func (t *SyncTask) WithMultiPartUploadSize(size int64) *SyncTask {
t.multiPartUploadSize = size
return t
}

View File

@ -0,0 +1,227 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package syncmgr
import (
"context"
"fmt"
"math"
"github.com/apache/arrow/go/v17/arrow"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/apache/arrow/go/v17/arrow/memory"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/flushcommon/metacache"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/internal/storagev2/packed"
iTypeutil "github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
"github.com/milvus-io/milvus/pkg/v2/util/retry"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type BulkPackWriterV2 struct {
*BulkPackWriter
arrowSchema *arrow.Schema
bufferSize int64
multiPartUploadSize int64
}
func NewBulkPackWriterV2(metaCache metacache.MetaCache, chunkManager storage.ChunkManager,
allocator allocator.Interface, bufferSize, multiPartUploadSize int64, writeRetryOpts ...retry.Option,
) *BulkPackWriterV2 {
arrowSchema, err := storage.ConvertToArrowSchema(metaCache.Schema().Fields)
if err != nil {
return nil
}
return &BulkPackWriterV2{
BulkPackWriter: &BulkPackWriter{
metaCache: metaCache,
chunkManager: chunkManager,
allocator: allocator,
writeRetryOpts: writeRetryOpts,
},
arrowSchema: arrowSchema,
bufferSize: bufferSize,
multiPartUploadSize: multiPartUploadSize,
}
}
func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
inserts map[int64]*datapb.FieldBinlog,
deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog,
size int64,
err error,
) {
err = bw.prefetchIDs(pack)
if err != nil {
log.Warn("failed allocate ids for sync task", zap.Error(err))
return
}
if inserts, err = bw.writeInserts(ctx, pack); err != nil {
log.Error("failed to write insert data", zap.Error(err))
return
}
if stats, err = bw.writeStats(ctx, pack); err != nil {
log.Error("failed to process stats blob", zap.Error(err))
return
}
if deltas, err = bw.writeDelta(ctx, pack); err != nil {
log.Error("failed to process delta blob", zap.Error(err))
return
}
if bm25Stats, err = bw.writeBM25Stasts(ctx, pack); err != nil {
log.Error("failed to process bm25 stats blob", zap.Error(err))
return
}
size = bw.sizeWritten
return
}
func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, error) {
if len(pack.insertData) == 0 {
return make(map[int64]*datapb.FieldBinlog), nil
}
columnGroups, err := bw.splitInsertData(pack.insertData, packed.ColumnGroupSizeThreshold)
if err != nil {
return nil, err
}
rec, err := bw.serializeBinlog(ctx, pack)
if err != nil {
return nil, err
}
logs := make(map[int64]*datapb.FieldBinlog)
paths := make([]string, 0)
for columnGroup := range columnGroups {
columnGroupID := typeutil.UniqueID(columnGroup)
path := metautil.BuildInsertLogPath(bw.chunkManager.RootPath(), pack.collectionID, pack.partitionID, pack.segmentID, columnGroupID, bw.nextID())
paths = append(paths, path)
}
tsArray := rec.Column(common.TimeStampField).(*array.Int64)
rows := rec.Len()
var tsFrom uint64 = math.MaxUint64
var tsTo uint64 = 0
for i := 0; i < rows; i++ {
ts := typeutil.Timestamp(tsArray.Value(i))
if ts < tsFrom {
tsFrom = ts
}
if ts > tsTo {
tsTo = ts
}
}
w, err := storage.NewPackedRecordWriter(paths, bw.arrowSchema, bw.bufferSize, bw.multiPartUploadSize, columnGroups)
if err != nil {
return nil, err
}
if err = w.Write(rec); err != nil {
return nil, err
}
for columnGroup := range columnGroups {
columnGroupID := typeutil.UniqueID(columnGroup)
logs[columnGroupID] = &datapb.FieldBinlog{
FieldID: columnGroupID,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup)),
MemorySize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup)),
LogPath: w.GetWrittenPaths()[columnGroupID],
EntriesNum: w.GetWrittenRowNum(),
TimestampFrom: tsFrom,
TimestampTo: tsTo,
},
},
}
}
if err = w.Close(); err != nil {
return nil, err
}
return logs, nil
}
// split by row average size
func (bw *BulkPackWriterV2) splitInsertData(insertData []*storage.InsertData, splitThresHold int64) ([]storagecommon.ColumnGroup, error) {
groups := make([]storagecommon.ColumnGroup, 0)
shortColumnGroup := storagecommon.ColumnGroup{Columns: make([]int, 0)}
memorySizes := make(map[storage.FieldID]int64, len(insertData[0].Data))
rowNums := make(map[storage.FieldID]int64, len(insertData[0].Data))
for _, data := range insertData {
for fieldID, fieldData := range data.Data {
if _, ok := memorySizes[fieldID]; !ok {
memorySizes[fieldID] = 0
rowNums[fieldID] = 0
}
memorySizes[fieldID] += int64(fieldData.GetMemorySize())
rowNums[fieldID] += int64(fieldData.RowNum())
}
}
uniqueRows := lo.Uniq(lo.Values(rowNums))
if len(uniqueRows) != 1 || uniqueRows[0] == 0 {
return nil, fmt.Errorf("row num is not equal for each field")
}
for i, field := range bw.metaCache.Schema().GetFields() {
if _, ok := memorySizes[field.FieldID]; !ok {
return nil, fmt.Errorf("field %d not found in insert data", field.FieldID)
}
if rowNums[field.FieldID] != 0 && memorySizes[field.FieldID]/rowNums[field.FieldID] >= splitThresHold {
groups = append(groups, storagecommon.ColumnGroup{Columns: []int{i}})
} else {
shortColumnGroup.Columns = append(shortColumnGroup.Columns, i)
}
}
if len(shortColumnGroup.Columns) > 0 {
groups = append(groups, shortColumnGroup)
}
return groups, nil
}
func (bw *BulkPackWriterV2) serializeBinlog(ctx context.Context, pack *SyncPack) (storage.Record, error) {
if len(pack.insertData) == 0 {
return nil, nil
}
builder := array.NewRecordBuilder(memory.DefaultAllocator, bw.arrowSchema)
defer builder.Release()
for _, chunk := range pack.insertData {
if err := iTypeutil.BuildRecord(builder, chunk, bw.metaCache.Schema().GetFields()); err != nil {
return nil, err
}
}
rec := builder.NewRecord()
field2Col := make(map[storage.FieldID]int, len(bw.metaCache.Schema().GetFields()))
for c, field := range bw.metaCache.Schema().GetFields() {
field2Col[field.FieldID] = c
}
return storage.NewSimpleArrowRecord(rec, field2Col), nil
}

View File

@ -0,0 +1,228 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package syncmgr
import (
"context"
"fmt"
"math"
"math/rand"
"sync/atomic"
"testing"
"github.com/samber/lo"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/flushcommon/metacache"
"github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle"
"github.com/milvus-io/milvus/internal/mocks/flushcommon/mock_util"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/internal/util/initcore"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
func TestPackWriterV2Suite(t *testing.T) {
suite.Run(t, new(PackWriterV2Suite))
}
type PackWriterV2Suite struct {
suite.Suite
ctx context.Context
mockID atomic.Int64
logIDAlloc allocator.Interface
mockBinlogIO *mock_util.MockBinlogIO
schema *schemapb.CollectionSchema
cm storage.ChunkManager
rootPath string
maxRowNum int64
chunkSize uint64
}
func (s *PackWriterV2Suite) SetupTest() {
s.ctx = context.Background()
s.logIDAlloc = allocator.NewLocalAllocator(1, math.MaxInt64)
s.rootPath = "/tmp"
initcore.InitLocalArrowFileSystem(s.rootPath)
paramtable.Get().Init(paramtable.NewBaseTable())
s.schema = &schemapb.CollectionSchema{
Name: "sync_task_test_col",
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, DataType: schemapb.DataType_Int64},
{
FieldID: 100,
Name: "pk",
DataType: schemapb.DataType_Int64,
IsPrimaryKey: true,
},
{
FieldID: 101,
Name: "vector",
DataType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "128"},
},
},
},
}
s.cm = storage.NewLocalChunkManager(storage.RootPath(s.rootPath))
}
func (s *PackWriterV2Suite) TestPackWriterV2_Write() {
collectionID := int64(123)
partitionID := int64(456)
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
rows := 10
bfs := pkoracle.NewBloomFilterSet()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{}, bfs, nil)
metacache.UpdateNumOfRows(1000)(seg)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Collection().Return(collectionID).Maybe()
mc.EXPECT().Schema().Return(s.schema).Maybe()
mc.EXPECT().GetSegmentByID(segmentID).Return(seg, true).Maybe()
mc.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg}).Maybe()
mc.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {
action(seg)
}).Return().Maybe()
deletes := &storage.DeleteData{}
for i := 0; i < rows; i++ {
pk := storage.NewInt64PrimaryKey(int64(i + 1))
ts := uint64(100 + i)
deletes.Append(pk, ts)
}
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema)).WithDeleteData(deletes)
bw := NewBulkPackWriterV2(mc, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0)
gotInserts, _, _, _, _, err := bw.Write(context.Background(), pack)
s.NoError(err)
s.Equal(gotInserts[0].Binlogs[0].GetEntriesNum(), int64(rows))
s.Equal(gotInserts[0].Binlogs[0].GetLogPath(), "/tmp/insert_log/123/456/789/0/1")
}
func (s *PackWriterV2Suite) TestWriteEmptyInsertData() {
s.logIDAlloc = allocator.NewLocalAllocator(1, 1)
collectionID := int64(123)
partitionID := int64(456)
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName)
bw := NewBulkPackWriterV2(mc, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
s.NoError(err)
}
func (s *PackWriterV2Suite) TestNoPkField() {
s.schema = &schemapb.CollectionSchema{
Name: "no pk field",
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, DataType: schemapb.DataType_Int64},
},
}
s.logIDAlloc = allocator.NewLocalAllocator(1, math.MaxInt64)
collectionID := int64(123)
partitionID := int64(456)
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
buf, _ := storage.NewInsertData(s.schema)
data := make(map[storage.FieldID]any)
data[common.RowIDField] = int64(1)
data[common.TimeStampField] = int64(1)
buf.Append(data)
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
bw := NewBulkPackWriterV2(mc, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err)
}
func (s *PackWriterV2Suite) TestAllocIDExhausedError() {
s.logIDAlloc = allocator.NewLocalAllocator(1, 1)
collectionID := int64(123)
partitionID := int64(456)
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
rows := 10
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema))
bw := NewBulkPackWriterV2(mc, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err)
}
func (s *PackWriterV2Suite) TestWriteInsertDataError() {
s.logIDAlloc = allocator.NewLocalAllocator(1, math.MaxInt64)
collectionID := int64(123)
partitionID := int64(456)
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
buf, _ := storage.NewInsertData(s.schema)
data := make(map[storage.FieldID]any)
data[common.RowIDField] = int64(1)
buf.Append(data)
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
bw := NewBulkPackWriterV2(mc, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0)
_, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err)
}
func genInsertData(size int, schema *schemapb.CollectionSchema) []*storage.InsertData {
buf, _ := storage.NewInsertData(schema)
for i := 0; i < size; i++ {
data := make(map[storage.FieldID]any)
data[common.RowIDField] = int64(i + 1)
data[common.TimeStampField] = int64(i + 1)
data[100] = int64(i + 1)
vector := lo.RepeatBy(128, func(_ int) float32 {
return rand.Float32()
})
data[101] = vector
buf.Append(data)
}
return []*storage.InsertData{buf}
}

View File

@ -76,8 +76,10 @@ type SyncTask struct {
tr *timerecord.TimeRecorder
flushedSize int64
execTime time.Duration
flushedSize int64
execTime time.Duration
multiPartUploadSize int64
syncBufferSize int64
}
func (t *SyncTask) getLogger() *log.MLogger {
@ -111,7 +113,7 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
}
}()
_, has := t.metacache.GetSegmentByID(t.segmentID)
segmentInfo, has := t.metacache.GetSegmentByID(t.segmentID)
if !has {
if t.pack.isDrop {
log.Info("segment dropped, discard sync task")
@ -122,11 +124,21 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
return err
}
writer := NewBulkPackWriter(t.metacache, t.chunkManager, t.allocator, t.writeRetryOpts...)
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.flushedSize, err = writer.Write(ctx, t.pack)
if err != nil {
log.Warn("failed to write sync data", zap.Error(err))
return err
switch segmentInfo.GetStorageVersion() {
case storage.StorageV2:
writer := NewBulkPackWriterV2(t.metacache, t.chunkManager, t.allocator, t.syncBufferSize, t.multiPartUploadSize, t.writeRetryOpts...)
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.flushedSize, err = writer.Write(ctx, t.pack)
if err != nil {
log.Warn("failed to write sync data with storage v2 format", zap.Error(err))
return err
}
default:
writer := NewBulkPackWriter(t.metacache, t.chunkManager, t.allocator, t.writeRetryOpts...)
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.flushedSize, err = writer.Write(ctx, t.pack)
if err != nil {
log.Warn("failed to write sync data", zap.Error(err))
return err
}
}
metrics.DataNodeFlushedSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.dataSource, t.level.String()).Add(float64(t.flushedSize))

View File

@ -38,6 +38,8 @@ import (
"github.com/milvus-io/milvus/internal/json"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/internal/util/initcore"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
@ -101,13 +103,15 @@ func (s *SyncTaskSuite) SetupTest() {
}
s.chunkManager = mocks.NewChunkManager(s.T())
s.chunkManager.EXPECT().RootPath().Return("files").Maybe()
s.chunkManager.EXPECT().RootPath().Return("/tmp").Maybe()
s.chunkManager.EXPECT().Write(mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe()
s.broker = broker.NewMockBroker(s.T())
s.metacache = metacache.NewMockMetaCache(s.T())
s.metacache.EXPECT().Collection().Return(s.collectionID).Maybe()
s.metacache.EXPECT().Schema().Return(s.schema).Maybe()
initcore.InitLocalArrowFileSystem("/tmp")
}
func (s *SyncTaskSuite) getEmptyInsertBuffer() *storage.InsertData {
@ -159,10 +163,7 @@ func (s *SyncTaskSuite) getSuiteSyncTask(pack *SyncPack) *SyncTask {
return task
}
func (s *SyncTaskSuite) TestRunNormal() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
s.broker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil)
func (s *SyncTaskSuite) createSegment(storageVersion int64) *metacache.SegmentInfo {
bfs := pkoracle.NewBloomFilterSet()
fd, err := storage.NewFieldData(schemapb.DataType_Int64, &schemapb.FieldSchema{
FieldID: 101,
@ -179,9 +180,29 @@ func (s *SyncTaskSuite) TestRunNormal() {
}
bfs.UpdatePKRange(fd)
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{}, bfs, nil)
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{StorageVersion: storageVersion}, bfs, nil)
metacache.UpdateNumOfRows(1000)(seg)
seg.GetBloomFilterSet().Roll()
return seg
}
func (s *SyncTaskSuite) TestRunNormal() {
s.runTestRunNormal(storage.StorageV1)
}
func (s *SyncTaskSuite) TestRunNormalWithStorageV2() {
s.runTestRunNormal(storage.StorageV2)
}
func (s *SyncTaskSuite) runTestRunNormal(storageVersion int64) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
s.broker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil)
seg := s.createSegment(storageVersion)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {
@ -196,6 +217,10 @@ func (s *SyncTaskSuite) TestRunNormal() {
Timestamp: 100,
}))
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
if storageVersion == storage.StorageV2 {
task.WithMultiPartUploadSize(0)
task.WithSyncBufferSize(packed.DefaultWriteBufferSize)
}
err := task.Run(ctx)
s.NoError(err)
@ -211,6 +236,10 @@ func (s *SyncTaskSuite) TestRunNormal() {
Timestamp: 100,
}))
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
if storageVersion == storage.StorageV2 {
task.WithMultiPartUploadSize(0)
task.WithSyncBufferSize(packed.DefaultWriteBufferSize)
}
err := task.Run(ctx)
s.NoError(err)
@ -227,6 +256,10 @@ func (s *SyncTaskSuite) TestRunNormal() {
Timestamp: 100,
}))
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
if storageVersion == storage.StorageV2 {
task.WithMultiPartUploadSize(0)
task.WithSyncBufferSize(packed.DefaultWriteBufferSize)
}
err := task.Run(ctx)
s.NoError(err)
})
@ -242,6 +275,10 @@ func (s *SyncTaskSuite) TestRunNormal() {
Timestamp: 100,
}))
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
if storageVersion == storage.StorageV2 {
task.WithMultiPartUploadSize(0)
task.WithSyncBufferSize(packed.DefaultWriteBufferSize)
}
err := task.Run(ctx)
s.NoError(err)
})
@ -251,13 +288,34 @@ func (s *SyncTaskSuite) TestRunL0Segment() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
s.broker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil)
bfs := pkoracle.NewBloomFilterSet()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{Level: datapb.SegmentLevel_L0}, bfs, nil)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.Run("pure_delete_l0_flush", func() {
bfs := pkoracle.NewBloomFilterSet()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{Level: datapb.SegmentLevel_L0}, bfs, nil)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
task := s.getSuiteSyncTask(new(SyncPack).
WithDeleteData(s.getDeleteBuffer()).
WithFlush().
WithCheckpoint(&msgpb.MsgPosition{
ChannelName: s.channelName,
MsgID: []byte{1, 2, 3, 4},
Timestamp: 100,
}))
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
err := task.Run(ctx)
s.NoError(err)
})
s.Run("pure_delete_l0_no_flush_storage_v2", func() {
// should not affect l0 segment with storage v2
bfs := pkoracle.NewBloomFilterSet()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{Level: datapb.SegmentLevel_L0, StorageVersion: storage.StorageV2}, bfs, nil)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
task := s.getSuiteSyncTask(new(SyncPack).
WithDeleteData(s.getDeleteBuffer()).
WithFlush().
@ -338,6 +396,22 @@ func (s *SyncTaskSuite) TestRunError() {
})
}
func (s *SyncTaskSuite) TestRunErrorWithStorageV2() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
s.Run("storage v2 allocate_id_fail", func() {
mockAllocator := allocator.NewMockAllocator(s.T())
mockAllocator.EXPECT().Alloc(mock.Anything).Return(0, 0, errors.New("mocked"))
segV2 := metacache.NewSegmentInfo(&datapb.SegmentInfo{Level: datapb.SegmentLevel_L0, StorageVersion: storage.StorageV2}, pkoracle.NewBloomFilterSet(), nil)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(segV2, true)
task := s.getSuiteSyncTask(new(SyncPack).WithFlush()).WithAllocator(mockAllocator)
err := task.Run(ctx)
s.Error(err)
})
}
func (s *SyncTaskSuite) TestSyncTask_MarshalJSON() {
t := &SyncTask{
segmentID: 12345,

View File

@ -2,6 +2,7 @@ package writebuffer
import (
"context"
"fmt"
"github.com/samber/lo"
"go.uber.org/zap"
@ -16,6 +17,7 @@ import (
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/metrics"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/conc"
@ -203,6 +205,22 @@ func (wb *l0WriteBuffer) BufferData(insertData []*InsertData, deleteMsgs []*msgs
return nil
}
// bufferInsert function InsertMsg into bufferred InsertData and returns primary key field data for future usage.
func (wb *l0WriteBuffer) bufferInsert(inData *InsertData, startPos, endPos *msgpb.MsgPosition) error {
wb.CreateNewGrowingSegment(inData.partitionID, inData.segmentID, startPos, storage.StorageV1)
segBuf := wb.getOrCreateBuffer(inData.segmentID)
totalMemSize := segBuf.insertBuffer.Buffer(inData, startPos, endPos)
wb.metaCache.UpdateSegments(metacache.SegmentActions(
metacache.UpdateBufferedRows(segBuf.insertBuffer.rows),
metacache.SetStartPositionIfNil(startPos),
), metacache.WithSegmentIDs(inData.segmentID))
metrics.DataNodeFlowGraphBufferDataSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), fmt.Sprint(wb.collectionID)).Add(float64(totalMemSize))
return nil
}
func (wb *l0WriteBuffer) getL0SegmentID(partitionID int64, startPos *msgpb.MsgPosition) int64 {
log := wb.logger
segmentID, ok := wb.l0Segments[partitionID]

View File

@ -26,7 +26,7 @@ type BufferManager interface {
// Register adds a WriteBuffer with provided schema & options.
Register(channel string, metacache metacache.MetaCache, opts ...WriteBufferOption) error
// CreateNewGrowingSegment notifies writeBuffer to create a new growing segment.
CreateNewGrowingSegment(ctx context.Context, channel string, partition int64, segmentID int64) error
CreateNewGrowingSegment(ctx context.Context, channel string, partition int64, segmentID int64, storageVersion int64) error
// SealSegments notifies writeBuffer corresponding to provided channel to seal segments.
// which will cause segment start flush procedure.
SealSegments(ctx context.Context, channel string, segmentIDs []int64) error
@ -176,7 +176,7 @@ func (m *bufferManager) Register(channel string, metacache metacache.MetaCache,
}
// CreateNewGrowingSegment notifies writeBuffer to create a new growing segment.
func (m *bufferManager) CreateNewGrowingSegment(ctx context.Context, channel string, partitionID int64, segmentID int64) error {
func (m *bufferManager) CreateNewGrowingSegment(ctx context.Context, channel string, partitionID int64, segmentID int64, storageVersion int64) error {
buf, loaded := m.buffers.Get(channel)
if !loaded {
log.Ctx(ctx).Warn("write buffer not found when create new growing segment",
@ -185,7 +185,7 @@ func (m *bufferManager) CreateNewGrowingSegment(ctx context.Context, channel str
zap.Int64("segmentID", segmentID))
return merr.WrapErrChannelNotFound(channel)
}
buf.CreateNewGrowingSegment(partitionID, segmentID, nil)
buf.CreateNewGrowingSegment(partitionID, segmentID, nil, storageVersion)
return nil
}

View File

@ -15,6 +15,7 @@ import (
"github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/flushcommon/metacache"
"github.com/milvus-io/milvus/internal/flushcommon/syncmgr"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/util/hardware"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
@ -106,7 +107,7 @@ func (s *ManagerSuite) TestFlushSegments() {
func (s *ManagerSuite) TestCreateNewGrowingSegment() {
manager := s.manager
err := manager.CreateNewGrowingSegment(context.Background(), s.channelName, 1, 1)
err := manager.CreateNewGrowingSegment(context.Background(), s.channelName, 1, 1, storage.StorageV2)
s.Error(err)
s.metacache.EXPECT().GetSegmentByID(mock.Anything).Return(nil, false).Once()
@ -118,7 +119,7 @@ func (s *ManagerSuite) TestCreateNewGrowingSegment() {
s.NoError(err)
s.manager.buffers.Insert(s.channelName, wb)
err = manager.CreateNewGrowingSegment(context.Background(), s.channelName, 1, 1)
err = manager.CreateNewGrowingSegment(context.Background(), s.channelName, 1, 1, storage.StorageV2)
s.NoError(err)
}

View File

@ -76,17 +76,17 @@ func (_c *MockBufferManager_BufferData_Call) RunAndReturn(run func(string, []*In
return _c
}
// CreateNewGrowingSegment provides a mock function with given fields: ctx, channel, partition, segmentID
func (_m *MockBufferManager) CreateNewGrowingSegment(ctx context.Context, channel string, partition int64, segmentID int64) error {
ret := _m.Called(ctx, channel, partition, segmentID)
// CreateNewGrowingSegment provides a mock function with given fields: ctx, channel, partition, segmentID, storageVersion
func (_m *MockBufferManager) CreateNewGrowingSegment(ctx context.Context, channel string, partition int64, segmentID int64, storageVersion int64) error {
ret := _m.Called(ctx, channel, partition, segmentID, storageVersion)
if len(ret) == 0 {
panic("no return value specified for CreateNewGrowingSegment")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, string, int64, int64) error); ok {
r0 = rf(ctx, channel, partition, segmentID)
if rf, ok := ret.Get(0).(func(context.Context, string, int64, int64, int64) error); ok {
r0 = rf(ctx, channel, partition, segmentID, storageVersion)
} else {
r0 = ret.Error(0)
}
@ -104,13 +104,14 @@ type MockBufferManager_CreateNewGrowingSegment_Call struct {
// - channel string
// - partition int64
// - segmentID int64
func (_e *MockBufferManager_Expecter) CreateNewGrowingSegment(ctx interface{}, channel interface{}, partition interface{}, segmentID interface{}) *MockBufferManager_CreateNewGrowingSegment_Call {
return &MockBufferManager_CreateNewGrowingSegment_Call{Call: _e.mock.On("CreateNewGrowingSegment", ctx, channel, partition, segmentID)}
// - storageVersion int64
func (_e *MockBufferManager_Expecter) CreateNewGrowingSegment(ctx interface{}, channel interface{}, partition interface{}, segmentID interface{}, storageVersion interface{}) *MockBufferManager_CreateNewGrowingSegment_Call {
return &MockBufferManager_CreateNewGrowingSegment_Call{Call: _e.mock.On("CreateNewGrowingSegment", ctx, channel, partition, segmentID, storageVersion)}
}
func (_c *MockBufferManager_CreateNewGrowingSegment_Call) Run(run func(ctx context.Context, channel string, partition int64, segmentID int64)) *MockBufferManager_CreateNewGrowingSegment_Call {
func (_c *MockBufferManager_CreateNewGrowingSegment_Call) Run(run func(ctx context.Context, channel string, partition int64, segmentID int64, storageVersion int64)) *MockBufferManager_CreateNewGrowingSegment_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(string), args[2].(int64), args[3].(int64))
run(args[0].(context.Context), args[1].(string), args[2].(int64), args[3].(int64), args[4].(int64))
})
return _c
}
@ -120,7 +121,7 @@ func (_c *MockBufferManager_CreateNewGrowingSegment_Call) Return(_a0 error) *Moc
return _c
}
func (_c *MockBufferManager_CreateNewGrowingSegment_Call) RunAndReturn(run func(context.Context, string, int64, int64) error) *MockBufferManager_CreateNewGrowingSegment_Call {
func (_c *MockBufferManager_CreateNewGrowingSegment_Call) RunAndReturn(run func(context.Context, string, int64, int64, int64) error) *MockBufferManager_CreateNewGrowingSegment_Call {
_c.Call.Return(run)
return _c
}

View File

@ -107,9 +107,9 @@ func (_c *MockWriteBuffer_Close_Call) RunAndReturn(run func(context.Context, boo
return _c
}
// CreateNewGrowingSegment provides a mock function with given fields: partitionID, segmentID, startPos
func (_m *MockWriteBuffer) CreateNewGrowingSegment(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition) {
_m.Called(partitionID, segmentID, startPos)
// CreateNewGrowingSegment provides a mock function with given fields: partitionID, segmentID, startPos, storageVersion
func (_m *MockWriteBuffer) CreateNewGrowingSegment(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition, storageVersion int64) {
_m.Called(partitionID, segmentID, startPos, storageVersion)
}
// MockWriteBuffer_CreateNewGrowingSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateNewGrowingSegment'
@ -121,13 +121,14 @@ type MockWriteBuffer_CreateNewGrowingSegment_Call struct {
// - partitionID int64
// - segmentID int64
// - startPos *msgpb.MsgPosition
func (_e *MockWriteBuffer_Expecter) CreateNewGrowingSegment(partitionID interface{}, segmentID interface{}, startPos interface{}) *MockWriteBuffer_CreateNewGrowingSegment_Call {
return &MockWriteBuffer_CreateNewGrowingSegment_Call{Call: _e.mock.On("CreateNewGrowingSegment", partitionID, segmentID, startPos)}
// - storageVersion int64
func (_e *MockWriteBuffer_Expecter) CreateNewGrowingSegment(partitionID interface{}, segmentID interface{}, startPos interface{}, storageVersion interface{}) *MockWriteBuffer_CreateNewGrowingSegment_Call {
return &MockWriteBuffer_CreateNewGrowingSegment_Call{Call: _e.mock.On("CreateNewGrowingSegment", partitionID, segmentID, startPos, storageVersion)}
}
func (_c *MockWriteBuffer_CreateNewGrowingSegment_Call) Run(run func(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition)) *MockWriteBuffer_CreateNewGrowingSegment_Call {
func (_c *MockWriteBuffer_CreateNewGrowingSegment_Call) Run(run func(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition, storageVersion int64)) *MockWriteBuffer_CreateNewGrowingSegment_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64), args[1].(int64), args[2].(*msgpb.MsgPosition))
run(args[0].(int64), args[1].(int64), args[2].(*msgpb.MsgPosition), args[3].(int64))
})
return _c
}
@ -137,7 +138,7 @@ func (_c *MockWriteBuffer_CreateNewGrowingSegment_Call) Return() *MockWriteBuffe
return _c
}
func (_c *MockWriteBuffer_CreateNewGrowingSegment_Call) RunAndReturn(run func(int64, int64, *msgpb.MsgPosition)) *MockWriteBuffer_CreateNewGrowingSegment_Call {
func (_c *MockWriteBuffer_CreateNewGrowingSegment_Call) RunAndReturn(run func(int64, int64, *msgpb.MsgPosition, int64)) *MockWriteBuffer_CreateNewGrowingSegment_Call {
_c.Call.Return(run)
return _c
}

View File

@ -21,6 +21,7 @@ type writeBufferOption struct {
metaWriter syncmgr.MetaWriter
errorHandler func(error)
taskObserverCallback TaskObserverCallback
storageVersion int64
}
func defaultWBOption(metacache metacache.MetaCache) *writeBufferOption {

View File

@ -39,7 +39,7 @@ type WriteBuffer interface {
// HasSegment checks whether certain segment exists in this buffer.
HasSegment(segmentID int64) bool
// CreateNewGrowingSegment creates a new growing segment in the buffer.
CreateNewGrowingSegment(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition)
CreateNewGrowingSegment(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition, storageVersion int64)
// BufferData is the method to buffer dml data msgs.
BufferData(insertMsgs []*InsertData, deleteMsgs []*msgstream.DeleteMsg, startPos, endPos *msgpb.MsgPosition) error
// FlushTimestamp set flush timestamp for write buffer
@ -500,40 +500,26 @@ func (id *InsertData) batchPkExists(pks []storage.PrimaryKey, tss []uint64, hits
return hits
}
func (wb *writeBufferBase) CreateNewGrowingSegment(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition) {
func (wb *writeBufferBase) CreateNewGrowingSegment(partitionID int64, segmentID int64, startPos *msgpb.MsgPosition, storageVersion int64) {
_, ok := wb.metaCache.GetSegmentByID(segmentID)
// new segment
if !ok {
wb.metaCache.AddSegment(&datapb.SegmentInfo{
ID: segmentID,
PartitionID: partitionID,
CollectionID: wb.collectionID,
InsertChannel: wb.channelName,
StartPosition: startPos,
State: commonpb.SegmentState_Growing,
}, func(_ *datapb.SegmentInfo) pkoracle.PkStat {
segmentInfo := &datapb.SegmentInfo{
ID: segmentID,
PartitionID: partitionID,
CollectionID: wb.collectionID,
InsertChannel: wb.channelName,
StartPosition: startPos,
State: commonpb.SegmentState_Growing,
StorageVersion: storageVersion,
}
wb.metaCache.AddSegment(segmentInfo, func(_ *datapb.SegmentInfo) pkoracle.PkStat {
return pkoracle.NewBloomFilterSetWithBatchSize(wb.getEstBatchSize())
}, metacache.NewBM25StatsFactory, metacache.SetStartPosRecorded(false))
log.Info("add growing segment", zap.Int64("segmentID", segmentID), zap.String("channel", wb.channelName))
}
}
// bufferInsert function InsertMsg into bufferred InsertData and returns primary key field data for future usage.
func (wb *writeBufferBase) bufferInsert(inData *InsertData, startPos, endPos *msgpb.MsgPosition) error {
wb.CreateNewGrowingSegment(inData.partitionID, inData.segmentID, startPos)
segBuf := wb.getOrCreateBuffer(inData.segmentID)
totalMemSize := segBuf.insertBuffer.Buffer(inData, startPos, endPos)
wb.metaCache.UpdateSegments(metacache.SegmentActions(
metacache.UpdateBufferedRows(segBuf.insertBuffer.rows),
metacache.SetStartPositionIfNil(startPos),
), metacache.WithSegmentIDs(inData.segmentID))
metrics.DataNodeFlowGraphBufferDataSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), fmt.Sprint(wb.collectionID)).Add(float64(totalMemSize))
return nil
}
// bufferDelete buffers DeleteMsg into DeleteData.
func (wb *writeBufferBase) bufferDelete(segmentID int64, pks []storage.PrimaryKey, tss []typeutil.Timestamp, startPos, endPos *msgpb.MsgPosition) {
segBuf := wb.getOrCreateBuffer(segmentID)

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
)
@ -46,10 +47,10 @@ type rwOptions struct {
type RwOption func(*rwOptions)
func defaultRwOptions() *rwOptions {
func DefaultRwOptions() *rwOptions {
return &rwOptions{
bufferSize: 32 * 1024 * 1024,
multiPartUploadSize: 10 * 1024 * 1024,
bufferSize: packed.DefaultWriteBufferSize,
multiPartUploadSize: packed.DefaultMultiPartUploadSize,
}
}
@ -90,7 +91,7 @@ func WithColumnGroups(columnGroups []storagecommon.ColumnGroup) RwOption {
}
func NewBinlogRecordReader(ctx context.Context, binlogs []*datapb.FieldBinlog, schema *schemapb.CollectionSchema, option ...RwOption) (RecordReader, error) {
rwOptions := defaultRwOptions()
rwOptions := DefaultRwOptions()
for _, opt := range option {
opt(rwOptions)
}
@ -140,7 +141,7 @@ func NewBinlogRecordWriter(ctx context.Context, collectionID, partitionID, segme
schema *schemapb.CollectionSchema, allocator allocator.Interface, chunkSize uint64, rootPath string, maxRowNum int64,
option ...RwOption,
) (BinlogRecordWriter, error) {
rwOptions := defaultRwOptions()
rwOptions := DefaultRwOptions()
for _, opt := range option {
opt(rwOptions)
}

View File

@ -224,6 +224,10 @@ func (pw *packedRecordWriter) GetWrittenUncompressed() uint64 {
return pw.writtenUncompressed
}
func (pw *packedRecordWriter) GetColumnGroupWrittenUncompressed(columnGroup int) uint64 {
return pw.columnGroupUncompressed[columnGroup]
}
func (pw *packedRecordWriter) GetWrittenPaths() []string {
return pw.paths
}

View File

@ -0,0 +1,24 @@
// Copyright 2023 Zilliz
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package packed
const (
// ColumnGroupSizeThreshold is the threshold of column group size per row.
ColumnGroupSizeThreshold = 1024 // 1KB
// DefaultBufferSize is the default buffer size for writing data to storage.
DefaultWriteBufferSize = 32 * 1024 * 1024 // 32MB
// DefaultMultiPartUploadSize is the default size of each part of a multipart upload.
DefaultMultiPartUploadSize = 10 * 1024 * 1024 // 10MB
)

View File

@ -53,7 +53,7 @@ func (impl *msgHandlerImpl) HandleCreateSegment(ctx context.Context, vchannel st
return errors.Wrap(err, "failed to get create segment message body")
}
for _, segmentInfo := range body.GetSegments() {
if err := impl.wbMgr.CreateNewGrowingSegment(ctx, vchannel, segmentInfo.GetPartitionId(), segmentInfo.GetSegmentId()); err != nil {
if err := impl.wbMgr.CreateNewGrowingSegment(ctx, vchannel, segmentInfo.GetPartitionId(), segmentInfo.GetSegmentId(), segmentInfo.StorageVersion); err != nil {
log.Warn("fail to create new growing segment",
zap.String("vchannel", vchannel),
zap.Int64("partition_id", segmentInfo.GetPartitionId()),
@ -63,7 +63,8 @@ func (impl *msgHandlerImpl) HandleCreateSegment(ctx context.Context, vchannel st
log.Info("create new growing segment",
zap.String("vchannel", vchannel),
zap.Int64("partition_id", segmentInfo.GetPartitionId()),
zap.Int64("segment_id", segmentInfo.GetSegmentId()))
zap.Int64("segment_id", segmentInfo.GetSegmentId()),
zap.Int64("storage_version", segmentInfo.GetStorageVersion()))
}
return nil
}

View File

@ -8,12 +8,14 @@ import (
"github.com/milvus-io/milvus/internal/streamingnode/client/handler/registry"
"github.com/milvus-io/milvus/internal/streamingnode/server/service"
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
"github.com/milvus-io/milvus/internal/util/initcore"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
_ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/kafka"
_ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/pulsar"
_ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/rmq"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
// Server is the streamingnode server.
@ -39,6 +41,12 @@ func (s *Server) Init(ctx context.Context) (err error) {
// init all service.
s.initService(ctx)
log.Info("streamingnode server initialized")
// init storage v2 file system.
err = initcore.InitStorageV2FileSystem(paramtable.Get())
if err != nil {
return err
}
return nil
}

View File

@ -8,6 +8,7 @@ import (
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/policy"
@ -19,6 +20,7 @@ import (
"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/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/syncutil"
)
@ -226,11 +228,13 @@ func (m *partitionSegmentManager) allocNewGrowingSegment(ctx context.Context) (*
if err != nil {
return nil, err
}
resp, err := dc.AllocSegment(ctx, &datapb.AllocSegmentRequest{
CollectionId: pendingSegment.GetCollectionID(),
PartitionId: pendingSegment.GetPartitionID(),
SegmentId: pendingSegment.GetSegmentID(),
Vchannel: pendingSegment.GetVChannel(),
CollectionId: pendingSegment.GetCollectionID(),
PartitionId: pendingSegment.GetPartitionID(),
SegmentId: pendingSegment.GetSegmentID(),
Vchannel: pendingSegment.GetVChannel(),
StorageVersion: pendingSegment.GetStorageVersion(),
})
if err := merr.CheckRPCCall(resp, err); err != nil {
return nil, errors.Wrap(err, "failed to alloc growing segment at datacoord")
@ -244,8 +248,9 @@ func (m *partitionSegmentManager) allocNewGrowingSegment(ctx context.Context) (*
// We only execute one segment creation operation at a time.
// But in future, we need to modify the segment creation operation to support batch creation.
// Because the partition-key based collection may create huge amount of segments at the same time.
PartitionId: pendingSegment.GetPartitionID(),
SegmentId: pendingSegment.GetSegmentID(),
PartitionId: pendingSegment.GetPartitionID(),
SegmentId: pendingSegment.GetSegmentID(),
StorageVersion: pendingSegment.GetStorageVersion(),
}},
}).BuildMutable()
if err != nil {
@ -300,7 +305,11 @@ func (m *partitionSegmentManager) createNewPendingSegment(ctx context.Context) (
if err != nil {
return nil, errors.Wrap(err, "failed to allocate segment id")
}
meta := newSegmentAllocManager(m.pchannel, m.collectionID, m.paritionID, int64(segmentID), m.vchannel, m.metrics)
storageVersion := storage.StorageV1
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
storageVersion = storage.StorageV2
}
meta := newSegmentAllocManager(m.pchannel, m.collectionID, m.paritionID, int64(segmentID), m.vchannel, m.metrics, storageVersion)
tx := meta.BeginModification()
tx.IntoPending()
if err := tx.Commit(ctx); err != nil {

View File

@ -299,6 +299,7 @@ func initializeTestState(t *testing.T) {
paramtable.Get().DataCoordCfg.SegmentSealProportion.SwapTempValue("1.0")
paramtable.Get().DataCoordCfg.SegmentSealProportionJitter.SwapTempValue("0.0")
paramtable.Get().DataCoordCfg.SegmentMaxSize.SwapTempValue("1")
paramtable.Get().Save(paramtable.Get().CommonCfg.EnableStorageV2.Key, "true")
streamingNodeCatalog := mock_metastore.NewMockStreamingNodeCataLog(t)
dataCoordClient := mocks.NewMockDataCoordClient(t)

View File

@ -66,16 +66,18 @@ func newSegmentAllocManager(
segmentID int64,
vchannel string,
metrics *metricsutil.SegmentAssignMetrics,
storageVersion int64,
) *segmentAllocManager {
return &segmentAllocManager{
pchannel: pchannel,
inner: &streamingpb.SegmentAssignmentMeta{
CollectionId: collectionID,
PartitionId: partitionID,
SegmentId: segmentID,
Vchannel: vchannel,
State: streamingpb.SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_UNKNOWN,
Stat: nil,
CollectionId: collectionID,
PartitionId: partitionID,
SegmentId: segmentID,
Vchannel: vchannel,
State: streamingpb.SegmentAssignmentState_SEGMENT_ASSIGNMENT_STATE_UNKNOWN,
Stat: nil,
StorageVersion: storageVersion,
},
immutableStat: nil, // immutable stat can be seen after sealed.
ackSem: atomic.NewInt32(0),
@ -136,6 +138,10 @@ func (s *segmentAllocManager) GetSegmentID() int64 {
return s.inner.GetSegmentId()
}
func (s *segmentAllocManager) GetStorageVersion() int64 {
return s.inner.GetStorageVersion()
}
// GetVChannel returns the vchannel of the segment assignment meta.
func (s *segmentAllocManager) GetVChannel() string {
return s.inner.GetVchannel()

View File

@ -172,6 +172,7 @@ message SegmentIDRequest {
bool isImport = 5; // deprecated
int64 importTaskID = 6; // deprecated
SegmentLevel level = 7; // deprecated
int64 storage_version = 8;
}
message AllocSegmentRequest {
@ -179,6 +180,7 @@ message AllocSegmentRequest {
int64 partition_id = 2;
int64 segment_id = 3; // segment id must be allocate from rootcoord idalloc service.
string vchannel = 4;
int64 storage_version = 5;
}
message AllocSegmentResponse {

File diff suppressed because it is too large Load Diff

View File

@ -92,6 +92,7 @@ message CreateSegmentMessageBody {
message CreateSegmentInfo {
int64 partition_id = 1;
int64 segment_id = 2;
int64 storage_version = 3;
}
// BeginTxnMessageBody is the body of begin transaction message.

View File

@ -570,8 +570,9 @@ type CreateSegmentInfo struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
PartitionId int64 `protobuf:"varint,1,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
SegmentId int64 `protobuf:"varint,2,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
PartitionId int64 `protobuf:"varint,1,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
SegmentId int64 `protobuf:"varint,2,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
StorageVersion int64 `protobuf:"varint,3,opt,name=storage_version,json=storageVersion,proto3" json:"storage_version,omitempty"`
}
func (x *CreateSegmentInfo) Reset() {
@ -620,6 +621,13 @@ func (x *CreateSegmentInfo) GetSegmentId() int64 {
return 0
}
func (x *CreateSegmentInfo) GetStorageVersion() int64 {
if x != nil {
return x.StorageVersion
}
return 0
}
// BeginTxnMessageBody is the body of begin transaction message.
// Just do nothing now.
type BeginTxnMessageBody struct {
@ -2129,197 +2137,200 @@ var file_messages_proto_rawDesc = []byte{
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, 0x55, 0x0a, 0x11,
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, 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,
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, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72,
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03,
0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42,
0x0a, 0x1b, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
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, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x49, 0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74,
0x69, 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, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69,
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72,
0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 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, 0x28, 0x03, 0x52,
0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a,
0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
0x22, 0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73,
0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65,
0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f,
0x6e, 0x64, 0x73, 0x18, 0x01, 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, 0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f,
0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x12, 0x0a, 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d,
0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
0x72, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68,
0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a,
0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03,
0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a,
0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06,
0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78,
0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, 0x6b, 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,
0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65,
0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b,
0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 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, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42, 0x0a, 0x1b, 0x44, 0x72, 0x6f, 0x70, 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, 0x28, 0x03, 0x52, 0x0c, 0x63,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43,
0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 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,
0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
0x6e, 0x49, 0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69,
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, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61,
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67,
0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64,
0x65, 0x72, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f,
0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 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, 0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d,
0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61,
0x64, 0x65, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54,
0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22,
0x12, 0x0a, 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61,
0x64, 0x65, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61,
0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e,
0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67,
0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f,
0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18,
0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16,
0x6b, 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, 0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f,
0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f,
0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 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, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c,
0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65,
0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 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, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73,
0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f,
0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74,
0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45,
0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01,
0x20, 0x01, 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, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b,
0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b,
0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65,
0x18, 0x02, 0x20, 0x01, 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, 0x42,
0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07,
0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64,
0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41,
0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 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,
0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a,
0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 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, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
0x79, 0x2a, 0x88, 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, 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,
0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f,
0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 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, 0x0b, 0x72,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x2a, 0x88, 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, 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 (

View File

@ -494,6 +494,7 @@ message SegmentAssignmentMeta {
string vchannel = 4;
SegmentAssignmentState state = 5;
SegmentAssignmentStat stat = 6;
int64 storage_version = 7;
}
// SegmentAssignmentState is the state of segment assignment.

View File

@ -3373,12 +3373,13 @@ type SegmentAssignmentMeta struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
SegmentId int64 `protobuf:"varint,3,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
Vchannel string `protobuf:"bytes,4,opt,name=vchannel,proto3" json:"vchannel,omitempty"`
State SegmentAssignmentState `protobuf:"varint,5,opt,name=state,proto3,enum=milvus.proto.streaming.SegmentAssignmentState" json:"state,omitempty"`
Stat *SegmentAssignmentStat `protobuf:"bytes,6,opt,name=stat,proto3" json:"stat,omitempty"`
CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
PartitionId int64 `protobuf:"varint,2,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"`
SegmentId int64 `protobuf:"varint,3,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
Vchannel string `protobuf:"bytes,4,opt,name=vchannel,proto3" json:"vchannel,omitempty"`
State SegmentAssignmentState `protobuf:"varint,5,opt,name=state,proto3,enum=milvus.proto.streaming.SegmentAssignmentState" json:"state,omitempty"`
Stat *SegmentAssignmentStat `protobuf:"bytes,6,opt,name=stat,proto3" json:"stat,omitempty"`
StorageVersion int64 `protobuf:"varint,7,opt,name=storage_version,json=storageVersion,proto3" json:"storage_version,omitempty"`
}
func (x *SegmentAssignmentMeta) Reset() {
@ -3455,6 +3456,13 @@ func (x *SegmentAssignmentMeta) GetStat() *SegmentAssignmentStat {
return nil
}
func (x *SegmentAssignmentMeta) GetStorageVersion() int64 {
if x != nil {
return x.StorageVersion
}
return 0
}
// SegmentAssignmentStat is the stat of segment assignment.
type SegmentAssignmentStat struct {
state protoimpl.MessageState
@ -4043,7 +4051,7 @@ var file_streaming_proto_rawDesc = []byte{
0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e,
0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x52, 0x11, 0x62, 0x61,
0x6c, 0x61, 0x6e, 0x63, 0x65, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22,
0xa3, 0x02, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67,
0xcc, 0x02, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67,
0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 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, 0x21,
@ -4061,185 +4069,187 @@ var file_streaming_proto_rawDesc = []byte{
0x32, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e,
0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x52,
0x04, 0x73, 0x74, 0x61, 0x74, 0x22, 0x87, 0x03, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e,
0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x12,
0x26, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69,
0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x42, 0x69, 0x6e,
0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72,
0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c,
0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x30, 0x0a, 0x14,
0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f,
0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x69, 0x6e, 0x73, 0x65,
0x72, 0x74, 0x65, 0x64, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x40,
0x0a, 0x1c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x04,
0x20, 0x01, 0x28, 0x03, 0x52, 0x1a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65,
0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73,
0x12, 0x4d, 0x0a, 0x23, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65,
0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f,
0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x6c,
0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73,
0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12,
0x25, 0x0a, 0x0e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65,
0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x43,
0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x37, 0x0a, 0x18, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65,
0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69,
0x63, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x15, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65,
0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22,
0x4f, 0x0a, 0x0d, 0x57, 0x41, 0x4c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74,
0x12, 0x3e, 0x0a, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x20, 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, 0x49, 0x44, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44,
0x2a, 0xc5, 0x01, 0x0a, 0x11, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x74,
0x61, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e,
0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e,
0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x43, 0x48, 0x41, 0x4e,
0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55,
0x4e, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21,
0x0a, 0x1d, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f,
0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x49, 0x4e, 0x47, 0x10,
0x02, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45,
0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x45,
0x44, 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f,
0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x41, 0x56, 0x41,
0x49, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x9a, 0x01, 0x0a, 0x12, 0x42, 0x72, 0x6f,
0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12,
0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53,
0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10,
0x00, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54,
0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e,
0x47, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54,
0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x4f, 0x4e, 0x45,
0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f,
0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x41, 0x49, 0x54, 0x5f,
0x41, 0x43, 0x4b, 0x10, 0x03, 0x2a, 0x82, 0x04, 0x0a, 0x0d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,
0x69, 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x52, 0x45, 0x41,
0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4b, 0x10, 0x00, 0x12, 0x24,
0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45,
0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49,
0x53, 0x54, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e,
0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x46,
0x45, 0x4e, 0x43, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x54, 0x52, 0x45, 0x41,
0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x48, 0x55,
0x54, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41,
0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49,
0x44, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x53, 0x45, 0x51, 0x10, 0x04, 0x12,
0x29, 0x0a, 0x25, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44,
0x45, 0x5f, 0x55, 0x4e, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x41, 0x4e,
0x4e, 0x45, 0x4c, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x05, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54,
0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e,
0x4f, 0x52, 0x45, 0x44, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06,
0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f,
0x44, 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45, 0x52, 0x10, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x54,
0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56,
0x41, 0x49, 0x4c, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x08, 0x12,
0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44,
0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x58,
0x50, 0x49, 0x52, 0x45, 0x44, 0x10, 0x09, 0x12, 0x2c, 0x0a, 0x28, 0x53, 0x54, 0x52, 0x45, 0x41,
0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49,
0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54,
0x41, 0x54, 0x45, 0x10, 0x0a, 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49,
0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45,
0x52, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41,
0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52,
0x43, 0x45, 0x5f, 0x41, 0x43, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x10, 0x0c, 0x12, 0x1b, 0x0a,
0x16, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f,
0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0xe7, 0x07, 0x2a, 0xd5, 0x01, 0x0a, 0x16, 0x53,
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74,
0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54,
0x04, 0x73, 0x74, 0x61, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65,
0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e,
0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x87,
0x03, 0x0a, 0x15, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f,
0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
0x04, 0x52, 0x0d, 0x6d, 0x61, 0x78, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65,
0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77,
0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65,
0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65,
0x64, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20,
0x01, 0x28, 0x04, 0x52, 0x12, 0x69, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x65, 0x64, 0x42, 0x69, 0x6e,
0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x40, 0x0a, 0x1c, 0x63, 0x72, 0x65, 0x61, 0x74,
0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f,
0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x1a, 0x63,
0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61,
0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x4d, 0x0a, 0x23, 0x6c, 0x61, 0x73,
0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73,
0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73,
0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x20, 0x6c, 0x61, 0x73, 0x74, 0x4d, 0x6f, 0x64, 0x69,
0x66, 0x69, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x61, 0x6e,
0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x69, 0x6e, 0x6c,
0x6f, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04,
0x52, 0x0d, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12,
0x37, 0x0a, 0x18, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e,
0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28,
0x04, 0x52, 0x15, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74,
0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x22, 0x4f, 0x0a, 0x0d, 0x57, 0x41, 0x4c, 0x43,
0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x3e, 0x0a, 0x09, 0x6d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 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, 0x49, 0x44, 0x52, 0x09,
0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x2a, 0xc5, 0x01, 0x0a, 0x11, 0x50, 0x43,
0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12,
0x1f, 0x0a, 0x1b, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41,
0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00,
0x12, 0x25, 0x0a, 0x21, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54,
0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41,
0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x43, 0x48, 0x41, 0x4e,
0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41,
0x53, 0x53, 0x49, 0x47, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x43,
0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54,
0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x45, 0x44, 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f,
0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54,
0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x41, 0x56, 0x41, 0x49, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10,
0x04, 0x2a, 0x9a, 0x01, 0x0a, 0x12, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54,
0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41,
0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45,
0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52,
0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41,
0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19,
0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53,
0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x4f, 0x4e, 0x45, 0x10, 0x02, 0x12, 0x21, 0x0a, 0x1d, 0x42,
0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54,
0x41, 0x54, 0x45, 0x5f, 0x57, 0x41, 0x49, 0x54, 0x5f, 0x41, 0x43, 0x4b, 0x10, 0x03, 0x2a, 0x82,
0x04, 0x0a, 0x0d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65,
0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f,
0x44, 0x45, 0x5f, 0x4f, 0x4b, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41,
0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45,
0x4c, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x10, 0x01, 0x12, 0x21, 0x0a,
0x1d, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f,
0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x46, 0x45, 0x4e, 0x43, 0x45, 0x44, 0x10, 0x02,
0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f,
0x44, 0x45, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x48, 0x55, 0x54, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03,
0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f,
0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45,
0x53, 0x54, 0x5f, 0x53, 0x45, 0x51, 0x10, 0x04, 0x12, 0x29, 0x0a, 0x25, 0x53, 0x54, 0x52, 0x45,
0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x4d, 0x41, 0x54,
0x43, 0x48, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x54, 0x45, 0x52,
0x4d, 0x10, 0x05, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47,
0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x44, 0x5f, 0x4f, 0x50,
0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52,
0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45,
0x52, 0x10, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47,
0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x49, 0x4c, 0x44, 0x5f, 0x41, 0x52,
0x47, 0x55, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x08, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45,
0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53,
0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x58, 0x50, 0x49, 0x52, 0x45, 0x44, 0x10, 0x09,
0x12, 0x2c, 0x0a, 0x28, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f,
0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53,
0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x10, 0x0a, 0x12, 0x20,
0x0a, 0x1c, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45,
0x5f, 0x55, 0x4e, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0b,
0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f,
0x44, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x41, 0x43, 0x51, 0x55,
0x49, 0x52, 0x45, 0x44, 0x10, 0x0c, 0x12, 0x1b, 0x0a, 0x16, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d,
0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e,
0x10, 0xe7, 0x07, 0x2a, 0xd5, 0x01, 0x0a, 0x16, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41,
0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x24,
0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e,
0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f,
0x57, 0x4e, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f,
0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45,
0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45,
0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54,
0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, 0x02,
0x12, 0x23, 0x0a, 0x1f, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49,
0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x45, 0x41,
0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54,
0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54,
0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53,
0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e,
0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10,
0x01, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53,
0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x52,
0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x45, 0x47, 0x4d, 0x45,
0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54,
0x41, 0x54, 0x45, 0x5f, 0x53, 0x45, 0x41, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20,
0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45,
0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x46, 0x4c, 0x55, 0x53, 0x48, 0x45, 0x44,
0x10, 0x04, 0x32, 0x89, 0x01, 0x0a, 0x19, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67,
0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65,
0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74,
0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74,
0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d,
0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x32, 0xe8,
0x01, 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72,
0x64, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
0x65, 0x12, 0x62, 0x0a, 0x09, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x28,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74,
0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73,
0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e,
0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x03, 0x41, 0x63, 0x6b, 0x12, 0x2b, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65,
0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41,
0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69,
0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, 0x6b, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xa5, 0x01, 0x0a, 0x1f, 0x53, 0x74,
0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x41, 0x73, 0x73, 0x69,
0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01,
0x0a, 0x12, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63,
0x6f, 0x76, 0x65, 0x72, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73,
0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x45, 0x5f, 0x46, 0x4c, 0x55, 0x53, 0x48, 0x45, 0x44, 0x10, 0x04, 0x32, 0x89, 0x01, 0x0a, 0x19,
0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61,
0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74,
0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12,
0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65,
0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53,
0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x32, 0xe8, 0x01, 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65,
0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63,
0x61, 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x62, 0x0a, 0x09, 0x42, 0x72,
0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67,
0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f,
0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30,
0x01, 0x32, 0xe1, 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e,
0x6f, 0x64, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
0x65, 0x12, 0x60, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65,
0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72,
0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28,
0x01, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x07, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x26,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74,
0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e,
0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xbe, 0x03, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,
0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x53, 0x65,
0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,
0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73,
0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69,
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61,
0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f,
0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x52, 0x65,
0x6d, 0x6f, 0x76, 0x65, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64,
0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62,
0x0a, 0x03, 0x41, 0x63, 0x6b, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42,
0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61,
0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x00, 0x32, 0xa5, 0x01, 0x0a, 0x1f, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67,
0x43, 0x6f, 0x6f, 0x72, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53,
0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x12, 0x41, 0x73, 0x73, 0x69, 0x67,
0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x31, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72,
0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e,
0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xe1, 0x01, 0x0a, 0x1b, 0x53,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x48, 0x61, 0x6e, 0x64,
0x6c, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x60, 0x0a, 0x07, 0x50, 0x72,
0x6f, 0x64, 0x75, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50,
0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72,
0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x07,
0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67,
0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xbe,
0x03, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65,
0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81,
0x01, 0x0a, 0x06, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69,
0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65,
0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74,
0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67,
0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69,
0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d,
0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x96, 0x01,
0x0a, 0x0d, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12,
0x40, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69,
0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c,
0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 0x41, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61,
0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x36, 0x5a, 0x34, 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, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x70, 0x62, 0x62, 0x06,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x00, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x39, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72,
0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67,
0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76,
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e,
0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d,
0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x0d, 0x43, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x40, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e,
0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d,
0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61,
0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, 0x2e, 0x6d, 0x69, 0x6c,
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d,
0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64,
0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53,
0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42,
0x36, 0x5a, 0x34, 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, 0x73, 0x74, 0x72, 0x65,
0x61, 0x6d, 0x69, 0x6e, 0x67, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (