From e7521afed592ed8304416fb74b643ae850f22b20 Mon Sep 17 00:00:00 2001 From: sunby <bingyi.sun@zilliz.com> Date: Fri, 4 Jun 2021 11:45:45 +0800 Subject: [PATCH] Change SaveBinlogPath (#5576) Signed-off-by: sunby <bingyi.sun@zilliz.com> --- internal/dataservice/binlog_helper.go | 47 +- internal/dataservice/datanode_helper.go | 17 +- internal/dataservice/grpc_services.go | 24 +- internal/dataservice/meta.go | 41 +- internal/dataservice/meta_test.go | 4 +- internal/dataservice/segment_manager.go | 9 +- internal/dataservice/server_test.go | 59 ++- internal/masterservice/master_service.go | 2 +- internal/masterservice/master_service_test.go | 4 +- internal/proto/data_service.proto | 21 +- internal/proto/datapb/data_service.pb.go | 436 +++++++----------- internal/proxynode/impl.go | 2 +- internal/proxynode/task.go | 4 +- 13 files changed, 295 insertions(+), 375 deletions(-) diff --git a/internal/dataservice/binlog_helper.go b/internal/dataservice/binlog_helper.go index 0259bdd00..4d299e408 100644 --- a/internal/dataservice/binlog_helper.go +++ b/internal/dataservice/binlog_helper.go @@ -3,10 +3,10 @@ package dataservice import ( "errors" "path" - "sort" "strconv" "github.com/golang/protobuf/proto" + "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" ) @@ -174,42 +174,39 @@ func (s *Server) getDDLBinlogMeta(collID UniqueID) (metas []*datapb.DDLBinlogMet } // GetVChanPositions get vchannel latest postitions with provided dml channel names -func (s *Server) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelPair, error) { +func (s *Server) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelInfo, error) { if s.kvClient == nil { return nil, errNilKvClient } - pairs := make([]*datapb.VchannelPair, 0, len(vchans)) + pairs := make([]*datapb.VchannelInfo, 0, len(vchans)) for _, vchan := range vchans { segments := s.meta.GetSegmentsByChannel(vchan.DmlChannel) - sort.Slice(segments, func(i, j int) bool { - return segments[i].ID < segments[j].ID - }) - - dmlPos := &datapb.PositionPair{} - ddlPos := &datapb.PositionPair{} + flushedSegmentIDs := make([]UniqueID, 0) + unflushedCheckpoints := make([]*datapb.CheckPoint, 0) + for _, s := range segments { + if s.State == commonpb.SegmentState_Flushing || s.State == commonpb.SegmentState_Flushed { + flushedSegmentIDs = append(flushedSegmentIDs, s.ID) + continue + } - zp := zeroPos(vchan.DmlChannel) - dmlPos.StartPosition = &zp - dmlPos.EndPosition = &zp - ddlPos.StartPosition = &zp - ddlPos.EndPosition = &zp + if s.DmlPosition == nil { + continue + } - // find the last segment with not-nil position - for i := 0; i < len(segments); i++ { - if segments[i].DmlPosition == nil { - break + cp := &datapb.CheckPoint{ + SegmentID: s.ID, + Position: s.DmlPosition, + NumOfRows: s.NumOfRows, } - dmlPos = segments[i].DmlPosition - ddlPos = segments[i].DdlPosition + unflushedCheckpoints = append(unflushedCheckpoints, cp) } - pairs = append(pairs, &datapb.VchannelPair{ + pairs = append(pairs, &datapb.VchannelInfo{ CollectionID: vchan.CollectionID, - DmlVchannelName: vchan.DmlChannel, - DdlVchannelName: vchan.DdlChannel, - DdlPosition: ddlPos, - DmlPosition: dmlPos, + ChannelName: vchan.DmlChannel, + FlushedSegments: flushedSegmentIDs, + CheckPoints: unflushedCheckpoints, }) } return pairs, nil diff --git a/internal/dataservice/datanode_helper.go b/internal/dataservice/datanode_helper.go index 171364980..25ae0444c 100644 --- a/internal/dataservice/datanode_helper.go +++ b/internal/dataservice/datanode_helper.go @@ -23,24 +23,21 @@ type vchannel struct { // positionProvider provides vchannel pair related position pairs type positionProvider interface { - GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelPair, error) + GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelInfo, error) GetDdlChannel() string } type dummyPosProvider struct{} //GetVChanPositions implements positionProvider -func (dp dummyPosProvider) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelPair, error) { - pairs := make([]*datapb.VchannelPair, len(vchans)) +func (dp dummyPosProvider) GetVChanPositions(vchans []vchannel) ([]*datapb.VchannelInfo, error) { + pairs := make([]*datapb.VchannelInfo, len(vchans)) for _, vchan := range vchans { - dmlPos := &datapb.PositionPair{} - ddlPos := &datapb.PositionPair{} - pairs = append(pairs, &datapb.VchannelPair{ + pairs = append(pairs, &datapb.VchannelInfo{ CollectionID: vchan.CollectionID, - DmlVchannelName: vchan.DmlChannel, - DdlVchannelName: vchan.DmlChannel, - DdlPosition: ddlPos, - DmlPosition: dmlPos, + ChannelName: vchan.DmlChannel, + FlushedSegments: []int64{}, + CheckPoints: []*datapb.CheckPoint{}, }) } return pairs, nil diff --git a/internal/dataservice/grpc_services.go b/internal/dataservice/grpc_services.go index e5d0a6bce..2be2d11f3 100644 --- a/internal/dataservice/grpc_services.go +++ b/internal/dataservice/grpc_services.go @@ -165,8 +165,8 @@ func (s *Server) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentSta state.Status.ErrorCode = commonpb.ErrorCode_Success state.State = segmentInfo.State if segmentInfo.DmlPosition != nil { - state.StartPosition = segmentInfo.DmlPosition.StartPosition - state.EndPosition = segmentInfo.DmlPosition.EndPosition + state.StartPosition = segmentInfo.DmlPosition + // FIXME no need this rpc } else { } @@ -295,10 +295,6 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath resp.Reason = "server is closed" return resp, nil } - if s.flushMsgStream == nil { - resp.Reason = "flush msg stream nil" - return resp, nil - } // check segment id & collection id matched _, err := s.meta.GetCollection(req.GetCollectionID()) @@ -308,26 +304,26 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath return resp, nil } - meta, err := s.prepareBinlog(req) + binlogs, err := s.prepareBinlog(req) if err != nil { log.Error("prepare binlog meta failed", zap.Error(err)) resp.Reason = err.Error() return resp, nil } - // set segment to SegmentState_Flushing - err = s.meta.FlushSegmentWithBinlogAndPos(req.SegmentID, nil, - nil, meta) + // set segment to SegmentState_Flushing and save binlogs and checkpoints + err = s.meta.SaveBinlogAndCheckPoints(req.SegmentID, req.Flushed, binlogs, req.CheckPoints) if err != nil { resp.Reason = err.Error() return resp, nil } log.Debug("flush segment with meta", zap.Int64("id", req.SegmentID), - zap.Any("meta", meta)) + zap.Any("meta", binlogs)) - s.segmentManager.DropSegment(ctx, req.SegmentID) - - s.flushCh <- req.SegmentID + if req.Flushed { + s.segmentManager.DropSegment(ctx, req.SegmentID) + s.flushCh <- req.SegmentID + } resp.ErrorCode = commonpb.ErrorCode_Success return resp, nil } diff --git a/internal/dataservice/meta.go b/internal/dataservice/meta.go index 5ea5d7943..08363dd10 100644 --- a/internal/dataservice/meta.go +++ b/internal/dataservice/meta.go @@ -16,6 +16,8 @@ import ( "github.com/golang/protobuf/proto" "github.com/milvus-io/milvus/internal/kv" + "github.com/milvus-io/milvus/internal/log" + "go.uber.org/zap" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -141,7 +143,7 @@ func (m *meta) GetNumRowsOfCollection(collectionID UniqueID) (int64, error) { var ret int64 = 0 for _, info := range m.segments { if info.CollectionID == collectionID { - ret += info.NumRows + ret += info.NumOfRows } } return ret, nil @@ -167,7 +169,7 @@ func (m *meta) UpdateSegmentStatistic(stats *internalpb.SegmentStatisticsUpdates if !ok { return newErrSegmentNotFound(stats.SegmentID) } - seg.NumRows = stats.NumRows + seg.NumOfRows = stats.NumRows if err := m.saveSegmentInfo(seg); err != nil { return err } @@ -231,27 +233,34 @@ func (m *meta) SealSegment(segID UniqueID) error { return nil } -func (m *meta) FlushSegmentWithBinlogAndPos(segID UniqueID, - dmlPositionPair *datapb.PositionPair, - ddlPositionPair *datapb.PositionPair, - binlogMeta map[string]string) error { +func (m *meta) SaveBinlogAndCheckPoints(segID UniqueID, flushed bool, + binlogs map[string]string, checkpoints []*datapb.CheckPoint) error { m.Lock() defer m.Unlock() - segInfo, ok := m.segments[segID] if !ok { return newErrSegmentNotFound(segID) } kv := make(map[string]string) - for k, v := range binlogMeta { + for k, v := range binlogs { kv[k] = v } - segInfo.State = commonpb.SegmentState_Flushing - segInfo.DmlPosition = dmlPositionPair - segInfo.DdlPosition = ddlPositionPair - segBytes := proto.MarshalTextString(segInfo) - key := m.prepareSegmentPath(segInfo) - kv[key] = segBytes + if flushed { + segInfo.State = commonpb.SegmentState_Flushing + } + + for _, cp := range checkpoints { + segment, ok := m.segments[cp.SegmentID] + if !ok { + log.Warn("Failed to find segment", zap.Int64("id", cp.SegmentID)) + continue + } + segment.DmlPosition = cp.Position + segment.NumOfRows = cp.NumOfRows + segBytes := proto.MarshalTextString(segInfo) + key := m.prepareSegmentPath(segInfo) + kv[key] = segBytes + } if err := m.saveKvTxn(kv); err != nil { return err @@ -382,7 +391,7 @@ func (m *meta) GetNumRowsOfPartition(collectionID UniqueID, partitionID UniqueID var ret int64 = 0 for _, info := range m.segments { if info.CollectionID == collectionID && info.PartitionID == partitionID { - ret += info.NumRows + ret += info.NumOfRows } } return ret, nil @@ -440,7 +449,7 @@ func BuildSegment(collectionID UniqueID, partitionID UniqueID, segmentID UniqueI CollectionID: collectionID, PartitionID: partitionID, InsertChannel: channelName, - NumRows: 0, + NumOfRows: 0, State: commonpb.SegmentState_Growing, }, nil } diff --git a/internal/dataservice/meta_test.go b/internal/dataservice/meta_test.go index 4deb77940..92d1edde8 100644 --- a/internal/dataservice/meta_test.go +++ b/internal/dataservice/meta_test.go @@ -204,7 +204,7 @@ func TestMeta_Basic(t *testing.T) { assert.Nil(t, err) segInfo0, err := BuildSegment(collID, partID0, segID0, channelName) assert.Nil(t, err) - segInfo0.NumRows = rowCount0 + segInfo0.NumOfRows = rowCount0 err = meta.AddSegment(segInfo0) assert.Nil(t, err) @@ -213,7 +213,7 @@ func TestMeta_Basic(t *testing.T) { assert.Nil(t, err) segInfo1, err := BuildSegment(collID, partID0, segID1, channelName) assert.Nil(t, err) - segInfo1.NumRows = rowCount1 + segInfo1.NumOfRows = rowCount1 err = meta.AddSegment(segInfo1) assert.Nil(t, err) diff --git a/internal/dataservice/segment_manager.go b/internal/dataservice/segment_manager.go index fc5dbe190..1b07f5684 100644 --- a/internal/dataservice/segment_manager.go +++ b/internal/dataservice/segment_manager.go @@ -286,19 +286,18 @@ func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID Unique lastExpireTime: 0, currentRows: 0, } - s.stats[id] = segStatus + segmentInfo := &datapb.SegmentInfo{ ID: id, CollectionID: collectionID, PartitionID: partitionID, InsertChannel: channelName, - NumRows: 0, + NumOfRows: 0, State: commonpb.SegmentState_Growing, MaxRowNum: int64(totalRows), LastExpireTime: 0, } - if err := s.meta.AddSegment(segmentInfo); err != nil { return nil, err } @@ -386,7 +385,7 @@ func (s *SegmentManager) tryToSealSegment() error { if segStatus.sealed { continue } - sealed, err := s.checkSegmentSealed(segStatus) + sealed, err := s.shouldSeal(segStatus) if err != nil { return err } @@ -402,7 +401,7 @@ func (s *SegmentManager) tryToSealSegment() error { return nil } -func (s *SegmentManager) checkSegmentSealed(segStatus *segmentStatus) (bool, error) { +func (s *SegmentManager) shouldSeal(segStatus *segmentStatus) (bool, error) { var allocSize int64 for _, allocation := range segStatus.allocations { allocSize += allocation.rowNums diff --git a/internal/dataservice/server_test.go b/internal/dataservice/server_test.go index 02700cd31..d54f8fef8 100644 --- a/internal/dataservice/server_test.go +++ b/internal/dataservice/server_test.go @@ -241,7 +241,7 @@ func TestGetSegmentStates(t *testing.T) { CollectionID: 100, PartitionID: 0, InsertChannel: "", - NumRows: 0, + NumOfRows: 0, State: commonpb.SegmentState_Growing, }) assert.Nil(t, err) @@ -496,6 +496,19 @@ func TestSaveBinlogPaths(t *testing.T) { TsBinlogPath: "/by-dev/test/0/ts/Allo8", }, }, + CheckPoints: []*datapb.CheckPoint{ + { + SegmentID: 0, + Position: &internalpb.MsgPosition{ + ChannelName: "ch1", + MsgID: []byte{1, 2, 3}, + MsgGroup: "", + Timestamp: 0, + }, + NumOfRows: 10, + }, + }, + Flushed: false, }) assert.Nil(t, err) assert.EqualValues(t, resp.ErrorCode, commonpb.ErrorCode_Success) @@ -527,6 +540,11 @@ func TestSaveBinlogPaths(t *testing.T) { assert.EqualValues(t, "/by-dev/test/0/ddl/Allo9", collMetas[1].DdlBinlogPath) } + segmentInfo, err := svr.meta.GetSegment(0) + assert.Nil(t, err) + assert.EqualValues(t, segmentInfo.DmlPosition.ChannelName, "ch1") + assert.EqualValues(t, segmentInfo.DmlPosition.MsgID, []byte{1, 2, 3}) + assert.EqualValues(t, segmentInfo.NumOfRows, 10) }) t.Run("Abnormal SaveRequest", func(t *testing.T) { ctx := context.Background() @@ -660,6 +678,7 @@ func TestGetVChannelPos(t *testing.T) { CollectionID: 0, PartitionID: 0, InsertChannel: "ch1", + State: commonpb.SegmentState_Flushed, }) assert.Nil(t, err) err = svr.meta.AddSegment(&datapb.SegmentInfo{ @@ -667,32 +686,35 @@ func TestGetVChannelPos(t *testing.T) { CollectionID: 0, PartitionID: 0, InsertChannel: "ch1", + State: commonpb.SegmentState_Growing, + DmlPosition: &internalpb.MsgPosition{ + ChannelName: "ch1", + MsgID: []byte{1, 2, 3}, + MsgGroup: "", + Timestamp: 0, + }, }) assert.Nil(t, err) - req := &datapb.SaveBinlogPathsRequest{ - SegmentID: 1, - CollectionID: 0, - Field2BinlogPaths: []*datapb.ID2PathList{}, - DdlBinlogPaths: []*datapb.DDLBinlogMeta{}, - } - status, err := svr.SaveBinlogPaths(context.TODO(), req) + err = svr.meta.AddSegment(&datapb.SegmentInfo{ + ID: 3, + CollectionID: 0, + PartitionID: 0, + InsertChannel: "ch1", + State: commonpb.SegmentState_Growing, + }) assert.Nil(t, err) - assert.EqualValues(t, commonpb.ErrorCode_Success, status.ErrorCode) t.Run("get unexisted channel", func(t *testing.T) { pair, err := svr.GetVChanPositions([]vchannel{ { CollectionID: 0, DmlChannel: "chx1", - DdlChannel: "chx2", }, }) assert.Nil(t, err) assert.EqualValues(t, 1, len(pair)) - assert.Nil(t, pair[0].DmlPosition.StartPosition.MsgID) - assert.Nil(t, pair[0].DmlPosition.EndPosition.MsgID) - assert.Nil(t, pair[0].DdlPosition.StartPosition.MsgID) - assert.Nil(t, pair[0].DdlPosition.EndPosition.MsgID) + assert.Empty(t, pair[0].CheckPoints) + assert.Empty(t, pair[0].FlushedSegments) }) t.Run("get existed channel", func(t *testing.T) { @@ -706,10 +728,11 @@ func TestGetVChannelPos(t *testing.T) { assert.Nil(t, err) assert.EqualValues(t, 1, len(pair)) assert.EqualValues(t, 0, pair[0].CollectionID) - // assert.EqualValues(t, []byte{1, 2, 3}, pair[0].DmlPosition.StartPosition.MsgID) - // assert.EqualValues(t, []byte{3, 4, 5}, pair[0].DmlPosition.EndPosition.MsgID) - // assert.EqualValues(t, []byte{1, 2, 3}, pair[0].DdlPosition.StartPosition.MsgID) - // assert.EqualValues(t, []byte{3, 4, 5}, pair[0].DdlPosition.EndPosition.MsgID) + assert.EqualValues(t, 1, len(pair[0].FlushedSegments)) + assert.EqualValues(t, 1, pair[0].FlushedSegments[0]) + assert.EqualValues(t, 1, len(pair[0].CheckPoints)) + assert.EqualValues(t, 2, pair[0].CheckPoints[0].SegmentID) + assert.EqualValues(t, []byte{1, 2, 3}, pair[0].CheckPoints[0].Position.MsgID) }) } diff --git a/internal/masterservice/master_service.go b/internal/masterservice/master_service.go index e36ae41dd..eade11de8 100644 --- a/internal/masterservice/master_service.go +++ b/internal/masterservice/master_service.go @@ -771,7 +771,7 @@ func (c *Core) SetDataService(ctx context.Context, s types.DataService) error { retErr = nil return } - retRows = segInfo.Infos[0].NumRows + retRows = segInfo.Infos[0].NumOfRows retErr = nil return } diff --git a/internal/masterservice/master_service_test.go b/internal/masterservice/master_service_test.go index d6e2cfd29..d84ee09f7 100644 --- a/internal/masterservice/master_service_test.go +++ b/internal/masterservice/master_service_test.go @@ -99,8 +99,8 @@ func (d *dataMock) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInf }, Infos: []*datapb.SegmentInfo{ { - NumRows: Params.MinSegmentSizeToEnableIndex, - State: commonpb.SegmentState_Flushed, + NumOfRows: Params.MinSegmentSizeToEnableIndex, + State: commonpb.SegmentState_Flushed, }, }, }, nil diff --git a/internal/proto/data_service.proto b/internal/proto/data_service.proto index 8beb8b460..5481bb2b9 100644 --- a/internal/proto/data_service.proto +++ b/internal/proto/data_service.proto @@ -169,13 +169,6 @@ message GetPartitionStatisticsResponse { message GetSegmentInfoChannelRequest { } -message VchannelPair { - int64 collectionID = 1; - string dml_vchannel_name = 2; - string ddl_vchannel_name = 3; - PositionPair ddl_position = 4; - PositionPair dml_position = 5; -} message VchannelInfo { int64 collectionID = 1; @@ -239,12 +232,11 @@ message SegmentInfo { int64 collectionID = 2; int64 partitionID = 3; string insert_channel = 4; - int64 num_rows = 5; + int64 num_of_rows = 5; common.SegmentState state = 6; - PositionPair dmlPosition = 7; - PositionPair ddlPosition = 8; - int64 max_row_num = 9; - uint64 last_expire_time = 10; + internal.MsgPosition dml_position = 7; + int64 max_row_num = 8; + uint64 last_expire_time = 9; } message ID2PathList { @@ -269,9 +261,8 @@ message SaveBinlogPathsRequest { message CheckPoint { int64 segmentID = 1; - int64 collectionID = 2; - internal.MsgPosition position = 3; - int64 rows = 4; + internal.MsgPosition position = 2; + int64 num_of_rows = 3; } message DataNodeTtMsg { diff --git a/internal/proto/datapb/data_service.pb.go b/internal/proto/datapb/data_service.pb.go index 8d2bf2e7f..30d4e2129 100644 --- a/internal/proto/datapb/data_service.pb.go +++ b/internal/proto/datapb/data_service.pb.go @@ -1223,77 +1223,6 @@ func (m *GetSegmentInfoChannelRequest) XXX_DiscardUnknown() { var xxx_messageInfo_GetSegmentInfoChannelRequest proto.InternalMessageInfo -type VchannelPair struct { - CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - DmlVchannelName string `protobuf:"bytes,2,opt,name=dml_vchannel_name,json=dmlVchannelName,proto3" json:"dml_vchannel_name,omitempty"` - DdlVchannelName string `protobuf:"bytes,3,opt,name=ddl_vchannel_name,json=ddlVchannelName,proto3" json:"ddl_vchannel_name,omitempty"` - DdlPosition *PositionPair `protobuf:"bytes,4,opt,name=ddl_position,json=ddlPosition,proto3" json:"ddl_position,omitempty"` - DmlPosition *PositionPair `protobuf:"bytes,5,opt,name=dml_position,json=dmlPosition,proto3" json:"dml_position,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *VchannelPair) Reset() { *m = VchannelPair{} } -func (m *VchannelPair) String() string { return proto.CompactTextString(m) } -func (*VchannelPair) ProtoMessage() {} -func (*VchannelPair) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{22} -} - -func (m *VchannelPair) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_VchannelPair.Unmarshal(m, b) -} -func (m *VchannelPair) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_VchannelPair.Marshal(b, m, deterministic) -} -func (m *VchannelPair) XXX_Merge(src proto.Message) { - xxx_messageInfo_VchannelPair.Merge(m, src) -} -func (m *VchannelPair) XXX_Size() int { - return xxx_messageInfo_VchannelPair.Size(m) -} -func (m *VchannelPair) XXX_DiscardUnknown() { - xxx_messageInfo_VchannelPair.DiscardUnknown(m) -} - -var xxx_messageInfo_VchannelPair proto.InternalMessageInfo - -func (m *VchannelPair) GetCollectionID() int64 { - if m != nil { - return m.CollectionID - } - return 0 -} - -func (m *VchannelPair) GetDmlVchannelName() string { - if m != nil { - return m.DmlVchannelName - } - return "" -} - -func (m *VchannelPair) GetDdlVchannelName() string { - if m != nil { - return m.DdlVchannelName - } - return "" -} - -func (m *VchannelPair) GetDdlPosition() *PositionPair { - if m != nil { - return m.DdlPosition - } - return nil -} - -func (m *VchannelPair) GetDmlPosition() *PositionPair { - if m != nil { - return m.DmlPosition - } - return nil -} - type VchannelInfo struct { CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` ChannelName string `protobuf:"bytes,2,opt,name=channelName,proto3" json:"channelName,omitempty"` @@ -1308,7 +1237,7 @@ func (m *VchannelInfo) Reset() { *m = VchannelInfo{} } func (m *VchannelInfo) String() string { return proto.CompactTextString(m) } func (*VchannelInfo) ProtoMessage() {} func (*VchannelInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{23} + return fileDescriptor_3385cd32ad6cfe64, []int{22} } func (m *VchannelInfo) XXX_Unmarshal(b []byte) error { @@ -1369,7 +1298,7 @@ func (m *WatchDmChannelsRequest) Reset() { *m = WatchDmChannelsRequest{} func (m *WatchDmChannelsRequest) String() string { return proto.CompactTextString(m) } func (*WatchDmChannelsRequest) ProtoMessage() {} func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{24} + return fileDescriptor_3385cd32ad6cfe64, []int{23} } func (m *WatchDmChannelsRequest) XXX_Unmarshal(b []byte) error { @@ -1418,7 +1347,7 @@ func (m *FlushSegmentsRequest) Reset() { *m = FlushSegmentsRequest{} } func (m *FlushSegmentsRequest) String() string { return proto.CompactTextString(m) } func (*FlushSegmentsRequest) ProtoMessage() {} func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{25} + return fileDescriptor_3385cd32ad6cfe64, []int{24} } func (m *FlushSegmentsRequest) XXX_Unmarshal(b []byte) error { @@ -1479,7 +1408,7 @@ func (m *SegmentMsg) Reset() { *m = SegmentMsg{} } func (m *SegmentMsg) String() string { return proto.CompactTextString(m) } func (*SegmentMsg) ProtoMessage() {} func (*SegmentMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{26} + return fileDescriptor_3385cd32ad6cfe64, []int{25} } func (m *SegmentMsg) XXX_Unmarshal(b []byte) error { @@ -1527,7 +1456,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{} func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) } func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{27} + return fileDescriptor_3385cd32ad6cfe64, []int{26} } func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { @@ -1575,7 +1504,7 @@ func (m *DDLBinlogMeta) Reset() { *m = DDLBinlogMeta{} } func (m *DDLBinlogMeta) String() string { return proto.CompactTextString(m) } func (*DDLBinlogMeta) ProtoMessage() {} func (*DDLBinlogMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{28} + return fileDescriptor_3385cd32ad6cfe64, []int{27} } func (m *DDLBinlogMeta) XXX_Unmarshal(b []byte) error { @@ -1622,7 +1551,7 @@ func (m *FieldFlushMeta) Reset() { *m = FieldFlushMeta{} } func (m *FieldFlushMeta) String() string { return proto.CompactTextString(m) } func (*FieldFlushMeta) ProtoMessage() {} func (*FieldFlushMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{29} + return fileDescriptor_3385cd32ad6cfe64, []int{28} } func (m *FieldFlushMeta) XXX_Unmarshal(b []byte) error { @@ -1670,7 +1599,7 @@ func (m *SegmentFlushMeta) Reset() { *m = SegmentFlushMeta{} } func (m *SegmentFlushMeta) String() string { return proto.CompactTextString(m) } func (*SegmentFlushMeta) ProtoMessage() {} func (*SegmentFlushMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{30} + return fileDescriptor_3385cd32ad6cfe64, []int{29} } func (m *SegmentFlushMeta) XXX_Unmarshal(b []byte) error { @@ -1724,7 +1653,7 @@ func (m *DDLFlushMeta) Reset() { *m = DDLFlushMeta{} } func (m *DDLFlushMeta) String() string { return proto.CompactTextString(m) } func (*DDLFlushMeta) ProtoMessage() {} func (*DDLFlushMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{31} + return fileDescriptor_3385cd32ad6cfe64, []int{30} } func (m *DDLFlushMeta) XXX_Unmarshal(b []byte) error { @@ -1772,7 +1701,7 @@ func (m *CollectionInfo) Reset() { *m = CollectionInfo{} } func (m *CollectionInfo) String() string { return proto.CompactTextString(m) } func (*CollectionInfo) ProtoMessage() {} func (*CollectionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{32} + return fileDescriptor_3385cd32ad6cfe64, []int{31} } func (m *CollectionInfo) XXX_Unmarshal(b []byte) error { @@ -1815,26 +1744,25 @@ func (m *CollectionInfo) GetPartitions() []int64 { } type SegmentInfo struct { - ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` - CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` - InsertChannel string `protobuf:"bytes,4,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` - NumRows int64 `protobuf:"varint,5,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` - State commonpb.SegmentState `protobuf:"varint,6,opt,name=state,proto3,enum=milvus.proto.common.SegmentState" json:"state,omitempty"` - DmlPosition *PositionPair `protobuf:"bytes,7,opt,name=dmlPosition,proto3" json:"dmlPosition,omitempty"` - DdlPosition *PositionPair `protobuf:"bytes,8,opt,name=ddlPosition,proto3" json:"ddlPosition,omitempty"` - MaxRowNum int64 `protobuf:"varint,9,opt,name=max_row_num,json=maxRowNum,proto3" json:"max_row_num,omitempty"` - LastExpireTime uint64 `protobuf:"varint,10,opt,name=last_expire_time,json=lastExpireTime,proto3" json:"last_expire_time,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + ID int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + PartitionID int64 `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"` + InsertChannel string `protobuf:"bytes,4,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"` + NumOfRows int64 `protobuf:"varint,5,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` + State commonpb.SegmentState `protobuf:"varint,6,opt,name=state,proto3,enum=milvus.proto.common.SegmentState" json:"state,omitempty"` + DmlPosition *internalpb.MsgPosition `protobuf:"bytes,7,opt,name=dml_position,json=dmlPosition,proto3" json:"dml_position,omitempty"` + MaxRowNum int64 `protobuf:"varint,8,opt,name=max_row_num,json=maxRowNum,proto3" json:"max_row_num,omitempty"` + LastExpireTime uint64 `protobuf:"varint,9,opt,name=last_expire_time,json=lastExpireTime,proto3" json:"last_expire_time,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SegmentInfo) Reset() { *m = SegmentInfo{} } func (m *SegmentInfo) String() string { return proto.CompactTextString(m) } func (*SegmentInfo) ProtoMessage() {} func (*SegmentInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{33} + return fileDescriptor_3385cd32ad6cfe64, []int{32} } func (m *SegmentInfo) XXX_Unmarshal(b []byte) error { @@ -1883,9 +1811,9 @@ func (m *SegmentInfo) GetInsertChannel() string { return "" } -func (m *SegmentInfo) GetNumRows() int64 { +func (m *SegmentInfo) GetNumOfRows() int64 { if m != nil { - return m.NumRows + return m.NumOfRows } return 0 } @@ -1897,20 +1825,13 @@ func (m *SegmentInfo) GetState() commonpb.SegmentState { return commonpb.SegmentState_SegmentStateNone } -func (m *SegmentInfo) GetDmlPosition() *PositionPair { +func (m *SegmentInfo) GetDmlPosition() *internalpb.MsgPosition { if m != nil { return m.DmlPosition } return nil } -func (m *SegmentInfo) GetDdlPosition() *PositionPair { - if m != nil { - return m.DdlPosition - } - return nil -} - func (m *SegmentInfo) GetMaxRowNum() int64 { if m != nil { return m.MaxRowNum @@ -1937,7 +1858,7 @@ func (m *ID2PathList) Reset() { *m = ID2PathList{} } func (m *ID2PathList) String() string { return proto.CompactTextString(m) } func (*ID2PathList) ProtoMessage() {} func (*ID2PathList) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{34} + return fileDescriptor_3385cd32ad6cfe64, []int{33} } func (m *ID2PathList) XXX_Unmarshal(b []byte) error { @@ -1984,7 +1905,7 @@ func (m *PositionPair) Reset() { *m = PositionPair{} } func (m *PositionPair) String() string { return proto.CompactTextString(m) } func (*PositionPair) ProtoMessage() {} func (*PositionPair) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{35} + return fileDescriptor_3385cd32ad6cfe64, []int{34} } func (m *PositionPair) XXX_Unmarshal(b []byte) error { @@ -2036,7 +1957,7 @@ func (m *SaveBinlogPathsRequest) Reset() { *m = SaveBinlogPathsRequest{} func (m *SaveBinlogPathsRequest) String() string { return proto.CompactTextString(m) } func (*SaveBinlogPathsRequest) ProtoMessage() {} func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{36} + return fileDescriptor_3385cd32ad6cfe64, []int{35} } func (m *SaveBinlogPathsRequest) XXX_Unmarshal(b []byte) error { @@ -2108,9 +2029,8 @@ func (m *SaveBinlogPathsRequest) GetFlushed() bool { type CheckPoint struct { SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` - CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - Position *internalpb.MsgPosition `protobuf:"bytes,3,opt,name=position,proto3" json:"position,omitempty"` - Rows int64 `protobuf:"varint,4,opt,name=rows,proto3" json:"rows,omitempty"` + Position *internalpb.MsgPosition `protobuf:"bytes,2,opt,name=position,proto3" json:"position,omitempty"` + NumOfRows int64 `protobuf:"varint,3,opt,name=num_of_rows,json=numOfRows,proto3" json:"num_of_rows,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2120,7 +2040,7 @@ func (m *CheckPoint) Reset() { *m = CheckPoint{} } func (m *CheckPoint) String() string { return proto.CompactTextString(m) } func (*CheckPoint) ProtoMessage() {} func (*CheckPoint) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{37} + return fileDescriptor_3385cd32ad6cfe64, []int{36} } func (m *CheckPoint) XXX_Unmarshal(b []byte) error { @@ -2148,13 +2068,6 @@ func (m *CheckPoint) GetSegmentID() int64 { return 0 } -func (m *CheckPoint) GetCollectionID() int64 { - if m != nil { - return m.CollectionID - } - return 0 -} - func (m *CheckPoint) GetPosition() *internalpb.MsgPosition { if m != nil { return m.Position @@ -2162,9 +2075,9 @@ func (m *CheckPoint) GetPosition() *internalpb.MsgPosition { return nil } -func (m *CheckPoint) GetRows() int64 { +func (m *CheckPoint) GetNumOfRows() int64 { if m != nil { - return m.Rows + return m.NumOfRows } return 0 } @@ -2182,7 +2095,7 @@ func (m *DataNodeTtMsg) Reset() { *m = DataNodeTtMsg{} } func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) } func (*DataNodeTtMsg) ProtoMessage() {} func (*DataNodeTtMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{38} + return fileDescriptor_3385cd32ad6cfe64, []int{37} } func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error { @@ -2236,7 +2149,7 @@ func (m *SegmentBinlogs) Reset() { *m = SegmentBinlogs{} } func (m *SegmentBinlogs) String() string { return proto.CompactTextString(m) } func (*SegmentBinlogs) ProtoMessage() {} func (*SegmentBinlogs) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{39} + return fileDescriptor_3385cd32ad6cfe64, []int{38} } func (m *SegmentBinlogs) XXX_Unmarshal(b []byte) error { @@ -2283,7 +2196,7 @@ func (m *FieldBinlog) Reset() { *m = FieldBinlog{} } func (m *FieldBinlog) String() string { return proto.CompactTextString(m) } func (*FieldBinlog) ProtoMessage() {} func (*FieldBinlog) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{40} + return fileDescriptor_3385cd32ad6cfe64, []int{39} } func (m *FieldBinlog) XXX_Unmarshal(b []byte) error { @@ -2331,7 +2244,7 @@ func (m *ChannelStatus) Reset() { *m = ChannelStatus{} } func (m *ChannelStatus) String() string { return proto.CompactTextString(m) } func (*ChannelStatus) ProtoMessage() {} func (*ChannelStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{41} + return fileDescriptor_3385cd32ad6cfe64, []int{40} } func (m *ChannelStatus) XXX_Unmarshal(b []byte) error { @@ -2386,7 +2299,7 @@ func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} } func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) } func (*DataNodeInfo) ProtoMessage() {} func (*DataNodeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{42} + return fileDescriptor_3385cd32ad6cfe64, []int{41} } func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error { @@ -2441,7 +2354,7 @@ func (m *GetRecoveryInfoResponse) Reset() { *m = GetRecoveryInfoResponse func (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) } func (*GetRecoveryInfoResponse) ProtoMessage() {} func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{43} + return fileDescriptor_3385cd32ad6cfe64, []int{42} } func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error { @@ -2496,7 +2409,7 @@ func (m *GetRecoveryInfoRequest) Reset() { *m = GetRecoveryInfoRequest{} func (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) } func (*GetRecoveryInfoRequest) ProtoMessage() {} func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_3385cd32ad6cfe64, []int{44} + return fileDescriptor_3385cd32ad6cfe64, []int{43} } func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error { @@ -2562,7 +2475,6 @@ func init() { proto.RegisterType((*GetPartitionStatisticsRequest)(nil), "milvus.proto.data.GetPartitionStatisticsRequest") proto.RegisterType((*GetPartitionStatisticsResponse)(nil), "milvus.proto.data.GetPartitionStatisticsResponse") proto.RegisterType((*GetSegmentInfoChannelRequest)(nil), "milvus.proto.data.GetSegmentInfoChannelRequest") - proto.RegisterType((*VchannelPair)(nil), "milvus.proto.data.VchannelPair") proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo") proto.RegisterType((*WatchDmChannelsRequest)(nil), "milvus.proto.data.WatchDmChannelsRequest") proto.RegisterType((*FlushSegmentsRequest)(nil), "milvus.proto.data.FlushSegmentsRequest") @@ -2590,141 +2502,137 @@ func init() { func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) } var fileDescriptor_3385cd32ad6cfe64 = []byte{ - // 2138 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x19, 0xdb, 0x6e, 0x1b, 0xc7, - 0x55, 0x4b, 0xea, 0x42, 0x1e, 0x5e, 0x24, 0x8d, 0x55, 0x85, 0xa5, 0x6d, 0x59, 0xde, 0x26, 0x8e, - 0xa2, 0xa0, 0x52, 0x4c, 0x17, 0xbd, 0xb9, 0x69, 0x61, 0x99, 0xb1, 0x4a, 0xd4, 0x32, 0xd4, 0x91, - 0xe3, 0x00, 0x0d, 0x0a, 0x62, 0xc5, 0x1d, 0x51, 0x5b, 0x73, 0x77, 0x99, 0x9d, 0xa5, 0x2c, 0x3f, - 0x39, 0x48, 0xfa, 0xd2, 0xa2, 0x68, 0xda, 0x7e, 0x41, 0x1b, 0xa0, 0x40, 0x81, 0x02, 0x45, 0xff, - 0xa0, 0x4f, 0xfd, 0x85, 0xfe, 0x4e, 0x83, 0xb9, 0xec, 0x7d, 0x48, 0xae, 0x28, 0x1b, 0x7a, 0xdb, - 0x99, 0x3d, 0xb7, 0x39, 0xf7, 0x39, 0x03, 0xc8, 0x34, 0x7c, 0xa3, 0x4b, 0x89, 0x77, 0x66, 0xf5, - 0xc8, 0xce, 0xd0, 0x73, 0x7d, 0x17, 0xad, 0xda, 0xd6, 0xe0, 0x6c, 0x44, 0xc5, 0x6a, 0x87, 0x01, - 0x34, 0xab, 0x3d, 0xd7, 0xb6, 0x5d, 0x47, 0x6c, 0x35, 0xeb, 0x96, 0xe3, 0x13, 0xcf, 0x31, 0x06, - 0x72, 0x5d, 0x8d, 0x23, 0x34, 0xab, 0xb4, 0x77, 0x4a, 0x6c, 0x43, 0xac, 0xf4, 0x57, 0x70, 0x0d, - 0x93, 0xbe, 0x45, 0x7d, 0xe2, 0x3d, 0x71, 0x4d, 0x82, 0xc9, 0x67, 0x23, 0x42, 0x7d, 0xf4, 0x01, - 0xcc, 0x1f, 0x1b, 0x94, 0x34, 0xb4, 0x4d, 0x6d, 0xab, 0xd2, 0xba, 0xb1, 0x93, 0x60, 0x29, 0x99, - 0x1d, 0xd0, 0xfe, 0x9e, 0x41, 0x09, 0xe6, 0x90, 0xe8, 0xfb, 0xb0, 0x64, 0x98, 0xa6, 0x47, 0x28, - 0x6d, 0x14, 0x26, 0x20, 0x3d, 0x10, 0x30, 0x38, 0x00, 0xd6, 0xbf, 0xd2, 0x60, 0x2d, 0x29, 0x01, - 0x1d, 0xba, 0x0e, 0x25, 0x68, 0x0f, 0x2a, 0x96, 0x63, 0xf9, 0xdd, 0xa1, 0xe1, 0x19, 0x36, 0x95, - 0x92, 0xdc, 0x4e, 0x12, 0x0d, 0x0f, 0xda, 0x71, 0x2c, 0xff, 0x90, 0x03, 0x62, 0xb0, 0xc2, 0x6f, - 0x74, 0x0f, 0x16, 0xa9, 0x6f, 0xf8, 0xa3, 0x40, 0xa6, 0xeb, 0x4a, 0x99, 0x8e, 0x38, 0x08, 0x96, - 0xa0, 0xfa, 0x39, 0x54, 0x1f, 0x0d, 0x46, 0xf4, 0x74, 0x76, 0x5d, 0x20, 0x98, 0x37, 0x8f, 0x3b, - 0x6d, 0xce, 0xb4, 0x88, 0xf9, 0x37, 0xd2, 0xa1, 0xda, 0x73, 0x07, 0x03, 0xd2, 0xf3, 0x2d, 0xd7, - 0xe9, 0xb4, 0x1b, 0xf3, 0xfc, 0x5f, 0x62, 0x4f, 0xff, 0xb3, 0x06, 0x2b, 0x47, 0xa4, 0x6f, 0x13, - 0xc7, 0xef, 0xb4, 0x03, 0xf6, 0x6b, 0xb0, 0xd0, 0x73, 0x47, 0x8e, 0xcf, 0xf9, 0xd7, 0xb0, 0x58, - 0xa0, 0xdb, 0x50, 0xed, 0x9d, 0x1a, 0x8e, 0x43, 0x06, 0x5d, 0xc7, 0xb0, 0x09, 0x67, 0x55, 0xc6, - 0x15, 0xb9, 0xf7, 0xc4, 0xb0, 0x49, 0x86, 0x63, 0x31, 0xcb, 0x11, 0x6d, 0x42, 0x65, 0x68, 0x78, - 0xbe, 0x95, 0x10, 0x2a, 0xbe, 0xa5, 0xff, 0x55, 0x83, 0xf5, 0x07, 0x94, 0x5a, 0x7d, 0x27, 0x23, - 0xd9, 0x3a, 0x2c, 0x3a, 0xae, 0x49, 0x3a, 0x6d, 0x2e, 0x5a, 0x11, 0xcb, 0x15, 0xba, 0x0e, 0xe5, - 0x21, 0x21, 0x5e, 0xd7, 0x73, 0x07, 0x81, 0x60, 0x25, 0xb6, 0x81, 0xdd, 0x01, 0x41, 0xbf, 0x84, - 0x55, 0x9a, 0x22, 0x44, 0x1b, 0xc5, 0xcd, 0xe2, 0x56, 0xa5, 0xf5, 0x9d, 0x9d, 0x8c, 0x67, 0xef, - 0xa4, 0x99, 0xe2, 0x2c, 0xb6, 0xfe, 0x79, 0x01, 0xae, 0x85, 0x70, 0x42, 0x56, 0xf6, 0xcd, 0x34, - 0x47, 0x49, 0x3f, 0x14, 0x4f, 0x2c, 0xf2, 0x68, 0x2e, 0x54, 0x79, 0x31, 0xae, 0xf2, 0x1c, 0x16, - 0x4c, 0xeb, 0x73, 0x21, 0xa3, 0x4f, 0x74, 0x0b, 0x2a, 0xe4, 0x7c, 0x68, 0x79, 0xa4, 0xeb, 0x5b, - 0x36, 0x69, 0x2c, 0x6e, 0x6a, 0x5b, 0xf3, 0x18, 0xc4, 0xd6, 0x53, 0xcb, 0x26, 0x31, 0x9f, 0x5d, - 0xca, 0xef, 0xb3, 0x5f, 0x6b, 0xf0, 0x56, 0xc6, 0x4a, 0x32, 0x90, 0x30, 0xac, 0xf0, 0x93, 0x47, - 0x9a, 0x61, 0xd1, 0xc4, 0x14, 0x7e, 0x67, 0x92, 0xc2, 0x23, 0x70, 0x9c, 0xc1, 0x9f, 0x2d, 0xb0, - 0xfe, 0xa6, 0xc1, 0xb5, 0xa3, 0x53, 0xf7, 0x85, 0x64, 0x41, 0x67, 0x0f, 0xb0, 0xb4, 0x29, 0x0a, - 0xd3, 0x4d, 0x51, 0xcc, 0x9a, 0x22, 0x08, 0xd3, 0xf9, 0x28, 0x4c, 0xf5, 0xe7, 0xb0, 0x96, 0x14, - 0x51, 0x2a, 0x71, 0x03, 0x20, 0x74, 0x3c, 0xa1, 0xbe, 0x22, 0x8e, 0xed, 0xcc, 0xa6, 0x90, 0xe7, - 0xf0, 0xd6, 0x3e, 0xf1, 0x25, 0x2f, 0xf6, 0x8f, 0x5c, 0x42, 0x27, 0x49, 0x09, 0x0b, 0x69, 0x09, - 0xf5, 0x7f, 0x17, 0xc2, 0xe4, 0xc2, 0x59, 0x75, 0x9c, 0x13, 0x17, 0xdd, 0x80, 0x72, 0x08, 0x22, - 0xc3, 0x24, 0xda, 0x40, 0x3f, 0x80, 0x05, 0x26, 0xa9, 0x88, 0x91, 0x7a, 0x3a, 0xf9, 0x06, 0x67, - 0x8a, 0xd1, 0xc4, 0x02, 0x1e, 0x75, 0xa0, 0x4e, 0x7d, 0xc3, 0xf3, 0xbb, 0x43, 0x97, 0x72, 0x6d, - 0x73, 0xf5, 0x57, 0x5a, 0xfa, 0x98, 0xf4, 0x7d, 0x40, 0xfb, 0x87, 0x12, 0x12, 0xd7, 0x38, 0x66, - 0xb0, 0x44, 0x1f, 0x41, 0x95, 0x38, 0x66, 0x44, 0x68, 0x3e, 0x37, 0xa1, 0x0a, 0x71, 0xcc, 0x90, - 0x4c, 0x64, 0x9f, 0x85, 0xfc, 0xf6, 0xf9, 0x83, 0x06, 0x8d, 0xac, 0x81, 0xa4, 0x47, 0x44, 0x14, - 0xb5, 0xdc, 0x14, 0xd1, 0x7d, 0x81, 0x44, 0x84, 0x81, 0x26, 0xa6, 0xbc, 0xd0, 0x48, 0x58, 0xa2, - 0xe8, 0x16, 0x7c, 0x2b, 0x92, 0x86, 0xff, 0x79, 0x63, 0xce, 0xf2, 0xa5, 0x06, 0xeb, 0x69, 0x5e, - 0x97, 0x39, 0xf7, 0xf7, 0x60, 0xc1, 0x72, 0x4e, 0xdc, 0xe0, 0xd8, 0x1b, 0x13, 0x12, 0x0f, 0xe3, - 0x25, 0x80, 0x75, 0x1b, 0xae, 0xef, 0x13, 0xbf, 0xe3, 0x50, 0xe2, 0xf9, 0x7b, 0x96, 0x33, 0x70, - 0xfb, 0x87, 0x86, 0x7f, 0x7a, 0x89, 0x18, 0x49, 0xb8, 0x7b, 0x21, 0xe5, 0xee, 0xfa, 0x3f, 0x34, - 0xb8, 0xa1, 0xe6, 0x27, 0x8f, 0xde, 0x84, 0xd2, 0x89, 0x45, 0x06, 0x66, 0x94, 0x02, 0xc2, 0x35, - 0x8b, 0x95, 0x21, 0x03, 0x96, 0x27, 0x1c, 0xd7, 0xa8, 0x1c, 0xf9, 0x9e, 0xe5, 0xf4, 0x1f, 0x5b, - 0xd4, 0xc7, 0x02, 0x3e, 0xa6, 0xcf, 0x62, 0x7e, 0xcf, 0xfc, 0xad, 0xf0, 0x4c, 0x21, 0xea, 0x43, - 0x51, 0xba, 0xe8, 0x9b, 0x6d, 0x58, 0x14, 0xed, 0x83, 0xfe, 0x7b, 0x0d, 0x36, 0xf6, 0x89, 0xff, - 0x30, 0xdc, 0x63, 0x62, 0x5a, 0xd4, 0xb7, 0x7a, 0x57, 0x20, 0xcc, 0x57, 0x1a, 0xdc, 0x1a, 0x2b, - 0x8c, 0xb4, 0xa0, 0xcc, 0x68, 0x41, 0x01, 0x54, 0x67, 0xb4, 0x5f, 0x90, 0x97, 0xcf, 0x8c, 0xc1, - 0x88, 0x1c, 0x1a, 0x96, 0x27, 0x32, 0xda, 0x8c, 0xf9, 0xfd, 0x9f, 0x1a, 0xdc, 0xdc, 0x27, 0xac, - 0x19, 0x15, 0x35, 0xe7, 0x0a, 0xb5, 0x93, 0xa3, 0xd3, 0xfb, 0xa3, 0x30, 0xa6, 0x52, 0xda, 0x2b, - 0x51, 0xdf, 0x06, 0x0f, 0xc7, 0x58, 0x5e, 0x90, 0x8e, 0x2e, 0x95, 0xa7, 0xff, 0xa5, 0x00, 0xd5, - 0x67, 0xb2, 0x6f, 0x63, 0xcc, 0x32, 0x7a, 0xd0, 0x14, 0x7a, 0xd8, 0x86, 0x55, 0xd3, 0x1e, 0x74, - 0xcf, 0x14, 0x3d, 0xe0, 0xb2, 0x69, 0x0f, 0x9e, 0xc5, 0xfb, 0x40, 0x06, 0x6b, 0xa6, 0x61, 0x8b, - 0x12, 0xd6, 0x4c, 0xc2, 0xee, 0x41, 0x95, 0xc1, 0xa6, 0xea, 0xd4, 0x2d, 0x45, 0xa2, 0x0b, 0x6a, - 0x12, 0xd7, 0x4f, 0xc5, 0x34, 0x07, 0x61, 0x91, 0x62, 0x34, 0xec, 0x18, 0x8d, 0x85, 0xbc, 0x34, - 0xec, 0x90, 0x86, 0xfe, 0x1f, 0x2d, 0x52, 0x0a, 0x2f, 0xf1, 0x79, 0x94, 0xb2, 0x09, 0xf1, 0xfe, - 0x57, 0xd5, 0x12, 0xff, 0x8c, 0x41, 0x90, 0xde, 0xf3, 0x43, 0xd7, 0x72, 0xc2, 0x86, 0xfd, 0xa6, - 0x42, 0xb2, 0x87, 0x21, 0x14, 0x8e, 0x63, 0xa0, 0x2d, 0x58, 0x3e, 0x61, 0xb7, 0x2a, 0x62, 0x06, - 0xbd, 0x55, 0x63, 0x9e, 0xa7, 0xd0, 0xf4, 0xb6, 0xfe, 0x3b, 0x0d, 0xd6, 0x3f, 0x31, 0xfc, 0xde, - 0x69, 0xdb, 0xbe, 0x7c, 0x66, 0xfb, 0x10, 0xca, 0x81, 0xf9, 0x82, 0xd4, 0xac, 0xd2, 0x67, 0x5c, - 0x63, 0x38, 0xc2, 0x60, 0xb7, 0x9f, 0x35, 0x7e, 0x19, 0xbc, 0x7c, 0xcf, 0x3a, 0x6b, 0xe0, 0x26, - 0x4b, 0xf5, 0x7c, 0xa6, 0x54, 0x9f, 0x03, 0x48, 0xe1, 0x0e, 0x68, 0x7f, 0x06, 0xb9, 0x7e, 0x08, - 0x4b, 0x92, 0x9a, 0x8c, 0xcd, 0x69, 0xc5, 0x39, 0x00, 0xd7, 0x8f, 0x60, 0x5d, 0xee, 0x3f, 0x62, - 0x55, 0x50, 0x54, 0xcc, 0x03, 0xe2, 0x1b, 0xa8, 0x01, 0x4b, 0xb2, 0x30, 0x4a, 0x77, 0x0b, 0x96, - 0xec, 0xfa, 0x73, 0xcc, 0xe1, 0xba, 0xac, 0xfa, 0x49, 0x4f, 0x83, 0xe3, 0xb0, 0xd8, 0xea, 0xbf, - 0x86, 0x5a, 0xbb, 0xfd, 0x38, 0x46, 0xeb, 0x0e, 0xb0, 0x58, 0xeb, 0xc6, 0xb1, 0x34, 0x8e, 0x55, - 0x33, 0xcd, 0x41, 0x54, 0xa5, 0xd1, 0xdb, 0x50, 0xf7, 0x69, 0x37, 0x4b, 0xbc, 0xea, 0xd3, 0x08, - 0x4a, 0x3f, 0x80, 0x3a, 0x17, 0x96, 0x1b, 0x75, 0x8a, 0xac, 0xb7, 0xa1, 0x1a, 0x23, 0x27, 0xdc, - 0xa7, 0x8c, 0x2b, 0x91, 0xb0, 0x94, 0x15, 0xc0, 0xa0, 0xa9, 0x8e, 0x28, 0x4e, 0x6e, 0xaa, 0x6f, - 0x02, 0x58, 0xb4, 0x2b, 0x9d, 0x9e, 0xcb, 0x58, 0xc2, 0x65, 0x8b, 0x3e, 0x12, 0x1b, 0xe8, 0x47, - 0xb0, 0xc8, 0xf9, 0xb3, 0x46, 0x55, 0x91, 0x63, 0xb9, 0x35, 0x92, 0x27, 0xc0, 0x12, 0x41, 0xff, - 0x18, 0xaa, 0xed, 0xf6, 0xe3, 0x48, 0x8e, 0x3c, 0x91, 0x9f, 0xe3, 0x8c, 0xaf, 0xa0, 0x1e, 0xd5, - 0x54, 0x9e, 0x52, 0xea, 0x50, 0x08, 0xc9, 0x15, 0x3a, 0x6d, 0xf4, 0x21, 0x2c, 0x8a, 0xa1, 0x92, - 0xf4, 0xa0, 0x77, 0x92, 0x32, 0xcb, 0x81, 0x53, 0xac, 0x30, 0xf3, 0x0d, 0x2c, 0x91, 0x98, 0x87, - 0x87, 0x75, 0x48, 0xa4, 0x96, 0x22, 0x8e, 0xed, 0xe8, 0xff, 0x2a, 0x42, 0x25, 0xe6, 0x80, 0x19, - 0xf6, 0xaf, 0xe7, 0x36, 0xf8, 0x0e, 0xd4, 0x2d, 0xde, 0x4e, 0x75, 0x65, 0xf4, 0xf3, 0x14, 0x5e, - 0xc6, 0x35, 0x2b, 0xde, 0x64, 0xa1, 0x6f, 0x43, 0xc9, 0x19, 0xd9, 0x5d, 0xcf, 0x7d, 0x41, 0xe5, - 0xf5, 0x7e, 0xc9, 0x19, 0xd9, 0xd8, 0x7d, 0x41, 0xa3, 0xeb, 0xd2, 0xe2, 0x05, 0xaf, 0x4b, 0x0f, - 0x20, 0x9e, 0xc2, 0xe5, 0xbd, 0xff, 0x42, 0x69, 0x9f, 0x93, 0x88, 0x2a, 0x49, 0xa3, 0x34, 0x43, - 0xf5, 0xd9, 0x80, 0x8a, 0x6d, 0x9c, 0xb3, 0x93, 0x75, 0x9d, 0x91, 0xdd, 0x28, 0x0b, 0xc7, 0xb5, - 0x8d, 0x73, 0xec, 0xbe, 0x78, 0x32, 0xb2, 0xd1, 0x16, 0xac, 0x0c, 0x0c, 0xea, 0x77, 0xe3, 0xe3, - 0x0b, 0xe0, 0xe3, 0x8b, 0x3a, 0xdb, 0xff, 0x28, 0x1c, 0x61, 0xe8, 0xf7, 0xa0, 0xd2, 0x69, 0xb7, - 0x98, 0xf7, 0xb0, 0x46, 0x37, 0x63, 0xaf, 0x35, 0x58, 0x38, 0x8c, 0x39, 0x9b, 0x58, 0xb0, 0x54, - 0x5b, 0x8d, 0x0b, 0xa7, 0xb8, 0x44, 0x6a, 0xaf, 0xeb, 0x12, 0x59, 0x98, 0xe9, 0x12, 0xa9, 0xff, - 0xbf, 0x00, 0xeb, 0x47, 0xc6, 0x19, 0x79, 0xf3, 0x77, 0x91, 0x5c, 0x95, 0xe1, 0x31, 0xac, 0xf2, - 0xc8, 0x6f, 0xc5, 0xe4, 0xe1, 0x05, 0x42, 0x9d, 0xc3, 0x63, 0x26, 0xc1, 0x59, 0x44, 0xf4, 0x73, - 0xa8, 0x27, 0x12, 0x6a, 0x90, 0x80, 0x36, 0x15, 0xa4, 0x12, 0x19, 0x1a, 0xa7, 0xf0, 0xd2, 0xbd, - 0xc2, 0xe2, 0x85, 0x7b, 0x05, 0x96, 0x92, 0x65, 0x7e, 0x5c, 0xe2, 0xf9, 0x31, 0x58, 0xea, 0x5f, - 0x6b, 0x00, 0x11, 0xd6, 0x94, 0x4c, 0x9b, 0x27, 0x2f, 0xfc, 0x14, 0x4a, 0x33, 0xcc, 0x28, 0x42, - 0x1c, 0x56, 0xd5, 0x79, 0x2a, 0x90, 0x33, 0x24, 0xf6, 0xad, 0x7f, 0xa1, 0x41, 0xad, 0x6d, 0xf8, - 0xc6, 0x13, 0xd7, 0x24, 0x4f, 0x67, 0xac, 0xca, 0x39, 0xa6, 0x94, 0x37, 0xa0, 0xcc, 0x62, 0x90, - 0xfa, 0x86, 0x3d, 0xe4, 0xb2, 0xcf, 0xe3, 0x68, 0x43, 0xf7, 0xa0, 0x2e, 0x53, 0x8d, 0x30, 0x0d, - 0x9d, 0xa2, 0xac, 0x3d, 0xa8, 0x9e, 0x44, 0x55, 0x7c, 0xd2, 0x45, 0x3d, 0x56, 0xec, 0x71, 0x02, - 0x47, 0x7f, 0x00, 0x95, 0xd8, 0xcf, 0x09, 0x95, 0xb5, 0x01, 0x4b, 0xc7, 0x31, 0x3e, 0x65, 0x1c, - 0x2c, 0xf5, 0x2f, 0x35, 0xa8, 0xc9, 0x54, 0x2b, 0x6e, 0x03, 0x4c, 0xc3, 0x5c, 0x03, 0xa2, 0xe8, - 0xf3, 0x6f, 0xf4, 0xe3, 0xe4, 0x60, 0xea, 0x6d, 0xa5, 0x6f, 0x71, 0x22, 0xbc, 0x91, 0x4c, 0x24, - 0xdb, 0x3c, 0x57, 0xc9, 0xcf, 0x35, 0xa8, 0x06, 0x16, 0xe4, 0x25, 0xa7, 0x11, 0xbd, 0x6e, 0x08, - 0x39, 0x82, 0x25, 0xfb, 0x73, 0x46, 0x3c, 0x1a, 0x64, 0x95, 0x22, 0x0e, 0x96, 0xe8, 0x27, 0x50, - 0x0a, 0x3b, 0xcf, 0xe2, 0xd8, 0x50, 0x4a, 0x1c, 0x16, 0x87, 0x18, 0xfa, 0x7f, 0x35, 0x3e, 0x1c, - 0xc4, 0xa4, 0xe7, 0x9e, 0x11, 0xef, 0x65, 0x62, 0x04, 0x73, 0x71, 0x77, 0xba, 0x1f, 0x93, 0x25, - 0x67, 0x17, 0x1c, 0x22, 0xa0, 0xfb, 0x91, 0xb5, 0x8a, 0x63, 0x7b, 0x92, 0xa4, 0xb3, 0x45, 0x06, - 0xfd, 0x93, 0x98, 0x24, 0x25, 0xcf, 0x71, 0xa5, 0x73, 0xdf, 0xed, 0xbb, 0xb0, 0x9a, 0xf1, 0x0e, - 0x54, 0x07, 0xf8, 0xd8, 0xe9, 0xb9, 0xf6, 0x70, 0x40, 0x7c, 0xb2, 0x32, 0x87, 0xaa, 0x50, 0x7a, - 0x18, 0xac, 0xb4, 0xd6, 0xff, 0x6a, 0x50, 0x61, 0x0e, 0x71, 0x24, 0x5e, 0xe2, 0xd0, 0x10, 0x10, - 0x1f, 0x35, 0xd8, 0x43, 0xd7, 0x09, 0x47, 0x83, 0xe8, 0x83, 0x31, 0xa9, 0x23, 0x0b, 0x2a, 0x55, - 0xd0, 0xbc, 0x33, 0x06, 0x23, 0x05, 0xae, 0xcf, 0x21, 0x9b, 0x73, 0x64, 0xe5, 0xf5, 0xa9, 0xd5, - 0x7b, 0x1e, 0x74, 0x23, 0x13, 0x38, 0xa6, 0x40, 0x03, 0x8e, 0xa9, 0x89, 0xa3, 0x5c, 0x88, 0xb1, - 0x54, 0xe0, 0x60, 0xfa, 0x1c, 0xfa, 0x0c, 0xd6, 0xd8, 0xdd, 0x3b, 0x1c, 0x01, 0x04, 0x0c, 0x5b, - 0xe3, 0x19, 0x66, 0x80, 0x2f, 0xc8, 0xd2, 0x80, 0x6a, 0xfc, 0x21, 0x10, 0xa9, 0x5e, 0x27, 0x14, - 0x6f, 0x95, 0xcd, 0x77, 0xa7, 0xc2, 0x85, 0x2c, 0xf6, 0x61, 0x81, 0xb7, 0xc7, 0x48, 0xe5, 0xfd, - 0xf1, 0x47, 0xbf, 0xe6, 0xa4, 0x01, 0x85, 0x3e, 0x87, 0x7e, 0x03, 0xcb, 0xa9, 0xe7, 0x16, 0xf4, - 0x9e, 0x82, 0xa4, 0xfa, 0xe1, 0xac, 0xb9, 0x9d, 0x07, 0x34, 0xae, 0x97, 0xf8, 0x93, 0x84, 0x52, - 0x2f, 0x8a, 0x67, 0x15, 0xa5, 0x5e, 0x54, 0x6f, 0x1b, 0xfa, 0x1c, 0xea, 0x43, 0x3d, 0x39, 0x69, - 0x41, 0x5b, 0x0a, 0x64, 0xe5, 0xf0, 0xb9, 0xf9, 0x5e, 0x0e, 0xc8, 0x90, 0x91, 0x0d, 0x2b, 0xe9, - 0x81, 0x3a, 0xda, 0x9e, 0x48, 0x20, 0x19, 0x2f, 0xef, 0xe7, 0x82, 0x0d, 0xd9, 0xbd, 0xe4, 0x5e, - 0x9c, 0x19, 0xe8, 0xa2, 0x1d, 0x35, 0x99, 0x71, 0x93, 0xe6, 0xe6, 0x6e, 0x6e, 0xf8, 0x90, 0x35, - 0x81, 0xd5, 0xcc, 0x80, 0x16, 0xbd, 0x3f, 0x89, 0x4e, 0x6a, 0xd8, 0xd1, 0x9c, 0x3e, 0x42, 0xd6, - 0xe7, 0xd0, 0x17, 0xa2, 0x4c, 0xa8, 0x86, 0x9e, 0xe8, 0xae, 0x9a, 0xdb, 0x84, 0x69, 0x6d, 0xb3, - 0x75, 0x11, 0x94, 0xf0, 0xac, 0xaf, 0x78, 0x8a, 0x57, 0x0c, 0x0e, 0xd3, 0xf9, 0x29, 0xa0, 0x37, - 0x7e, 0x22, 0xda, 0xbc, 0x7b, 0x01, 0x8c, 0x50, 0x00, 0x37, 0xfd, 0x32, 0x12, 0xa4, 0xab, 0xdd, - 0xa9, 0xce, 0x39, 0x5b, 0xae, 0xfa, 0x14, 0x96, 0x53, 0x17, 0x01, 0x65, 0xfc, 0xab, 0x2f, 0x0b, - 0x39, 0x92, 0x4b, 0xaa, 0x62, 0xa2, 0x31, 0x41, 0xa6, 0xa8, 0xaa, 0xcd, 0xed, 0x3c, 0xa0, 0xc1, - 0x41, 0x5a, 0x7f, 0x2f, 0x42, 0x29, 0xe8, 0x74, 0xae, 0xa0, 0xaa, 0x5d, 0x41, 0x99, 0xf9, 0x14, - 0x96, 0x53, 0xd3, 0x45, 0xa5, 0x76, 0xd5, 0x13, 0xc8, 0x69, 0xa6, 0xfb, 0x04, 0x6a, 0x89, 0x71, - 0x21, 0x7a, 0x77, 0x5c, 0xa1, 0x49, 0x67, 0xeb, 0xc9, 0x84, 0xf7, 0xee, 0xfd, 0xea, 0x6e, 0xdf, - 0xf2, 0x4f, 0x47, 0xc7, 0xec, 0xcf, 0xae, 0x00, 0xfd, 0xae, 0xe5, 0xca, 0xaf, 0xdd, 0x40, 0x41, - 0xbb, 0x1c, 0x7b, 0x97, 0xb1, 0x19, 0x1e, 0x1f, 0x2f, 0xf2, 0xd5, 0xbd, 0x6f, 0x02, 0x00, 0x00, - 0xff, 0xff, 0xf1, 0x4d, 0xb3, 0x0c, 0x46, 0x24, 0x00, 0x00, + // 2072 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0x5b, 0x6f, 0x1c, 0x49, + 0x15, 0x76, 0xcf, 0xf8, 0x36, 0x67, 0x2e, 0xb6, 0x2b, 0xc6, 0x3b, 0x9a, 0x24, 0x8e, 0xd3, 0xec, + 0x66, 0xbd, 0x59, 0x61, 0x6f, 0x26, 0x88, 0x5b, 0x58, 0x50, 0x9c, 0x49, 0xcc, 0x88, 0x38, 0x98, + 0x72, 0x76, 0x57, 0x62, 0x85, 0x46, 0xed, 0xe9, 0xf2, 0xb8, 0x49, 0x5f, 0x66, 0xbb, 0x7a, 0x1c, + 0xe7, 0x29, 0xab, 0x85, 0x17, 0x10, 0x62, 0xe1, 0x1f, 0x00, 0x12, 0x12, 0x12, 0x2f, 0xfc, 0x03, + 0x9e, 0xf8, 0x0b, 0xfc, 0x12, 0xde, 0x41, 0x75, 0xe9, 0x7b, 0xcd, 0x4c, 0x7b, 0x9c, 0xc8, 0x6f, + 0x53, 0x35, 0xe7, 0x56, 0xa7, 0xce, 0xf9, 0xce, 0xa9, 0xd3, 0x80, 0x4c, 0x23, 0x30, 0x7a, 0x94, + 0xf8, 0x67, 0x56, 0x9f, 0xec, 0x0c, 0x7d, 0x2f, 0xf0, 0xd0, 0x9a, 0x63, 0xd9, 0x67, 0x23, 0x2a, + 0x56, 0x3b, 0x8c, 0xa0, 0x55, 0xeb, 0x7b, 0x8e, 0xe3, 0xb9, 0x62, 0xab, 0xd5, 0xb0, 0xdc, 0x80, + 0xf8, 0xae, 0x61, 0xcb, 0x75, 0x2d, 0xc9, 0xd0, 0xaa, 0xd1, 0xfe, 0x29, 0x71, 0x0c, 0xb1, 0xd2, + 0x5f, 0xc3, 0x35, 0x4c, 0x06, 0x16, 0x0d, 0x88, 0xff, 0xcc, 0x33, 0x09, 0x26, 0x5f, 0x8c, 0x08, + 0x0d, 0xd0, 0x47, 0x30, 0x7f, 0x6c, 0x50, 0xd2, 0xd4, 0xb6, 0xb4, 0xed, 0x6a, 0xfb, 0xc6, 0x4e, + 0x4a, 0xa5, 0x54, 0x76, 0x40, 0x07, 0x7b, 0x06, 0x25, 0x98, 0x53, 0xa2, 0xef, 0xc0, 0x92, 0x61, + 0x9a, 0x3e, 0xa1, 0xb4, 0x59, 0x9a, 0xc0, 0xf4, 0x50, 0xd0, 0xe0, 0x90, 0x58, 0xff, 0x5a, 0x83, + 0xf5, 0xb4, 0x05, 0x74, 0xe8, 0xb9, 0x94, 0xa0, 0x3d, 0xa8, 0x5a, 0xae, 0x15, 0xf4, 0x86, 0x86, + 0x6f, 0x38, 0x54, 0x5a, 0x72, 0x3b, 0x2d, 0x34, 0x3a, 0x68, 0xd7, 0xb5, 0x82, 0x43, 0x4e, 0x88, + 0xc1, 0x8a, 0x7e, 0xa3, 0xfb, 0xb0, 0x48, 0x03, 0x23, 0x18, 0x85, 0x36, 0x5d, 0x57, 0xda, 0x74, + 0xc4, 0x49, 0xb0, 0x24, 0xd5, 0xcf, 0xa1, 0xf6, 0xc4, 0x1e, 0xd1, 0xd3, 0xd9, 0x7d, 0x81, 0x60, + 0xde, 0x3c, 0xee, 0x76, 0xb8, 0xd2, 0x32, 0xe6, 0xbf, 0x91, 0x0e, 0xb5, 0xbe, 0x67, 0xdb, 0xa4, + 0x1f, 0x58, 0x9e, 0xdb, 0xed, 0x34, 0xe7, 0xf9, 0x7f, 0xa9, 0x3d, 0xfd, 0x4f, 0x1a, 0xac, 0x1e, + 0x91, 0x81, 0x43, 0xdc, 0xa0, 0xdb, 0x09, 0xd5, 0xaf, 0xc3, 0x42, 0xdf, 0x1b, 0xb9, 0x01, 0xd7, + 0x5f, 0xc7, 0x62, 0x81, 0x6e, 0x43, 0xad, 0x7f, 0x6a, 0xb8, 0x2e, 0xb1, 0x7b, 0xae, 0xe1, 0x10, + 0xae, 0xaa, 0x82, 0xab, 0x72, 0xef, 0x99, 0xe1, 0x90, 0x9c, 0xc6, 0x72, 0x5e, 0x23, 0xda, 0x82, + 0xea, 0xd0, 0xf0, 0x03, 0x2b, 0x65, 0x54, 0x72, 0x4b, 0xff, 0xb3, 0x06, 0x1b, 0x0f, 0x29, 0xb5, + 0x06, 0x6e, 0xce, 0xb2, 0x0d, 0x58, 0x74, 0x3d, 0x93, 0x74, 0x3b, 0xdc, 0xb4, 0x32, 0x96, 0x2b, + 0x74, 0x1d, 0x2a, 0x43, 0x42, 0xfc, 0x9e, 0xef, 0xd9, 0xa1, 0x61, 0xcb, 0x6c, 0x03, 0x7b, 0x36, + 0x41, 0x3f, 0x87, 0x35, 0x9a, 0x11, 0x44, 0x9b, 0xe5, 0xad, 0xf2, 0x76, 0xb5, 0xfd, 0xcd, 0x9d, + 0x5c, 0x64, 0xef, 0x64, 0x95, 0xe2, 0x3c, 0xb7, 0xfe, 0x65, 0x09, 0xae, 0x45, 0x74, 0xc2, 0x56, + 0xf6, 0x9b, 0x79, 0x8e, 0x92, 0x41, 0x64, 0x9e, 0x58, 0x14, 0xf1, 0x5c, 0xe4, 0xf2, 0x72, 0xd2, + 0xe5, 0x05, 0x6e, 0x30, 0xeb, 0xcf, 0x85, 0x9c, 0x3f, 0xd1, 0x2d, 0xa8, 0x92, 0xf3, 0xa1, 0xe5, + 0x93, 0x5e, 0x60, 0x39, 0xa4, 0xb9, 0xb8, 0xa5, 0x6d, 0xcf, 0x63, 0x10, 0x5b, 0xcf, 0x2d, 0x87, + 0x24, 0x62, 0x76, 0xa9, 0x78, 0xcc, 0xfe, 0x55, 0x83, 0x77, 0x72, 0xb7, 0x24, 0x13, 0x09, 0xc3, + 0x2a, 0x3f, 0x79, 0xec, 0x19, 0x96, 0x4d, 0xcc, 0xe1, 0x77, 0x26, 0x39, 0x3c, 0x26, 0xc7, 0x39, + 0xfe, 0xd9, 0x12, 0xeb, 0x2f, 0x1a, 0x5c, 0x3b, 0x3a, 0xf5, 0x5e, 0x4a, 0x15, 0x74, 0xf6, 0x04, + 0xcb, 0x5e, 0x45, 0x69, 0xfa, 0x55, 0x94, 0xf3, 0x57, 0x11, 0xa6, 0xe9, 0x7c, 0x9c, 0xa6, 0xfa, + 0x0b, 0x58, 0x4f, 0x9b, 0x28, 0x9d, 0xb8, 0x09, 0x10, 0x05, 0x9e, 0x70, 0x5f, 0x19, 0x27, 0x76, + 0x66, 0x73, 0xc8, 0x0b, 0x78, 0x67, 0x9f, 0x04, 0x52, 0x17, 0xfb, 0x8f, 0x5c, 0xc2, 0x27, 0x69, + 0x0b, 0x4b, 0x59, 0x0b, 0xf5, 0x7f, 0x96, 0x22, 0x70, 0xe1, 0xaa, 0xba, 0xee, 0x89, 0x87, 0x6e, + 0x40, 0x25, 0x22, 0x91, 0x69, 0x12, 0x6f, 0xa0, 0xef, 0xc2, 0x02, 0xb3, 0x54, 0xe4, 0x48, 0x23, + 0x0b, 0xbe, 0xe1, 0x99, 0x12, 0x32, 0xb1, 0xa0, 0x47, 0x5d, 0x68, 0xd0, 0xc0, 0xf0, 0x83, 0xde, + 0xd0, 0xa3, 0xdc, 0xdb, 0xdc, 0xfd, 0xd5, 0xb6, 0x3e, 0x06, 0xbe, 0x0f, 0xe8, 0xe0, 0x50, 0x52, + 0xe2, 0x3a, 0xe7, 0x0c, 0x97, 0xe8, 0x31, 0xd4, 0x88, 0x6b, 0xc6, 0x82, 0xe6, 0x0b, 0x0b, 0xaa, + 0x12, 0xd7, 0x8c, 0xc4, 0xc4, 0xf7, 0xb3, 0x50, 0xfc, 0x7e, 0x7e, 0xaf, 0x41, 0x33, 0x7f, 0x41, + 0x32, 0x22, 0x62, 0x89, 0x5a, 0x61, 0x89, 0xe8, 0x81, 0x60, 0x22, 0xe2, 0x82, 0x26, 0x42, 0x5e, + 0x74, 0x49, 0x58, 0xb2, 0xe8, 0x16, 0x7c, 0x23, 0xb6, 0x86, 0xff, 0xf3, 0xd6, 0x82, 0xe5, 0xd7, + 0x1a, 0x6c, 0x64, 0x75, 0x5d, 0xe6, 0xdc, 0xdf, 0x86, 0x05, 0xcb, 0x3d, 0xf1, 0xc2, 0x63, 0x6f, + 0x4e, 0x00, 0x1e, 0xa6, 0x4b, 0x10, 0xeb, 0x0e, 0x5c, 0xdf, 0x27, 0x41, 0xd7, 0xa5, 0xc4, 0x0f, + 0xf6, 0x2c, 0xd7, 0xf6, 0x06, 0x87, 0x46, 0x70, 0x7a, 0x89, 0x1c, 0x49, 0x85, 0x7b, 0x29, 0x13, + 0xee, 0xfa, 0xdf, 0x35, 0xb8, 0xa1, 0xd6, 0x27, 0x8f, 0xde, 0x82, 0xe5, 0x13, 0x8b, 0xd8, 0x66, + 0x0c, 0x01, 0xd1, 0x9a, 0xe5, 0xca, 0x90, 0x11, 0xcb, 0x13, 0x8e, 0x6b, 0x54, 0x8e, 0x02, 0xdf, + 0x72, 0x07, 0x4f, 0x2d, 0x1a, 0x60, 0x41, 0x9f, 0xf0, 0x67, 0xb9, 0x78, 0x64, 0xfe, 0x46, 0x44, + 0xa6, 0x30, 0xf5, 0x91, 0x28, 0x5d, 0xf4, 0xed, 0x36, 0x2c, 0x8a, 0xf6, 0x41, 0xff, 0x9d, 0x06, + 0x9b, 0xfb, 0x24, 0x78, 0x14, 0xed, 0x31, 0x33, 0x2d, 0x1a, 0x58, 0xfd, 0x2b, 0x30, 0xe6, 0x6b, + 0x0d, 0x6e, 0x8d, 0x35, 0x46, 0xde, 0xa0, 0x44, 0xb4, 0xb0, 0x00, 0xaa, 0x11, 0xed, 0xa7, 0xe4, + 0xd5, 0xa7, 0x86, 0x3d, 0x22, 0x87, 0x86, 0xe5, 0x0b, 0x44, 0x9b, 0x11, 0xdf, 0xff, 0xa1, 0xc1, + 0xcd, 0x7d, 0xc2, 0x9a, 0x51, 0x51, 0x73, 0xae, 0xd0, 0x3b, 0x05, 0x3a, 0xbd, 0x3f, 0x88, 0xcb, + 0x54, 0x5a, 0x7b, 0x25, 0xee, 0xdb, 0xe4, 0xe9, 0x98, 0xc0, 0x05, 0x19, 0xe8, 0xd2, 0x79, 0xfa, + 0xbf, 0x34, 0xa8, 0x7d, 0x2a, 0xfb, 0x36, 0x5e, 0xcd, 0xb2, 0x7e, 0xd0, 0xd4, 0x7e, 0x48, 0xb4, + 0x7a, 0xaa, 0xee, 0xef, 0xc7, 0x8c, 0x82, 0xf4, 0x5f, 0x1c, 0x7a, 0x96, 0x1b, 0xf5, 0xa6, 0x37, + 0x15, 0x88, 0xf5, 0x28, 0xa2, 0xc2, 0x49, 0x0e, 0xb4, 0x0d, 0x2b, 0x27, 0xec, 0x01, 0x41, 0xcc, + 0xb0, 0x8d, 0x68, 0xce, 0x73, 0xb4, 0xc8, 0x6e, 0xeb, 0xbf, 0xd5, 0x60, 0xe3, 0x33, 0x23, 0xe8, + 0x9f, 0x76, 0x9c, 0xcb, 0x27, 0xf1, 0xc7, 0x50, 0x39, 0x93, 0xe7, 0x08, 0x51, 0xe8, 0x96, 0xc2, + 0xea, 0xa4, 0xc7, 0x70, 0xcc, 0xc1, 0x1a, 0xfd, 0x75, 0xfe, 0xee, 0xb9, 0x7c, 0x7b, 0x36, 0x6b, + 0x8c, 0xa6, 0xab, 0xd2, 0x7c, 0xae, 0x2a, 0x9d, 0x03, 0x48, 0xe3, 0x0e, 0xe8, 0x60, 0x06, 0xbb, + 0xbe, 0x07, 0x4b, 0x52, 0x9a, 0x0c, 0xc3, 0x69, 0x75, 0x28, 0x24, 0xd7, 0x8f, 0x60, 0x43, 0xee, + 0x3f, 0x61, 0x80, 0x2f, 0x8a, 0xc3, 0x01, 0x09, 0x0c, 0xd4, 0x84, 0x25, 0x59, 0x03, 0x64, 0xb8, + 0x85, 0x4b, 0xd6, 0xe9, 0x1f, 0x73, 0xba, 0x1e, 0x03, 0x7a, 0x19, 0x69, 0x70, 0x1c, 0xd5, 0x15, + 0xfd, 0x97, 0x50, 0xef, 0x74, 0x9e, 0x26, 0x64, 0xdd, 0x81, 0x15, 0xd3, 0xb4, 0x7b, 0x49, 0x2e, + 0x8d, 0x73, 0xd5, 0x4d, 0xd3, 0x8e, 0x0b, 0x12, 0x7a, 0x17, 0x1a, 0x01, 0xed, 0xe5, 0x85, 0xd7, + 0x02, 0x1a, 0x53, 0xe9, 0x07, 0xd0, 0xe0, 0xc6, 0xf2, 0x4b, 0x9d, 0x62, 0xeb, 0x6d, 0xa8, 0x25, + 0xc4, 0x89, 0xf0, 0xa9, 0xe0, 0x6a, 0x6c, 0x2c, 0x65, 0x58, 0x1f, 0xf6, 0x8f, 0xb1, 0xc4, 0xc9, + 0xfd, 0xe3, 0x4d, 0x00, 0x8b, 0xf6, 0x64, 0xd0, 0x73, 0x1b, 0x97, 0x71, 0xc5, 0xa2, 0x4f, 0xc4, + 0x06, 0xfa, 0x3e, 0x2c, 0x72, 0xfd, 0xac, 0x27, 0x53, 0xc0, 0x09, 0xbf, 0x8d, 0xf4, 0x09, 0xb0, + 0x64, 0xd0, 0x3f, 0x81, 0x5a, 0xa7, 0xf3, 0x34, 0xb6, 0xa3, 0x48, 0xe6, 0x17, 0x38, 0xe3, 0x6b, + 0x68, 0xc4, 0xe5, 0x83, 0x43, 0x4a, 0x03, 0x4a, 0x91, 0xb8, 0x52, 0xb7, 0x83, 0x3e, 0x86, 0x45, + 0x31, 0x3f, 0x91, 0x11, 0xf4, 0x5e, 0xda, 0x66, 0x39, 0x5b, 0x49, 0xd4, 0x20, 0xbe, 0x81, 0x25, + 0x13, 0x8b, 0xf0, 0x08, 0x72, 0x05, 0xb4, 0x94, 0x71, 0x62, 0x47, 0xff, 0x6f, 0x09, 0xaa, 0x89, + 0x00, 0xcc, 0xa9, 0x7f, 0x33, 0x0f, 0x9f, 0xf7, 0xa0, 0x61, 0xf1, 0xce, 0xa1, 0x27, 0xb3, 0x9f, + 0x97, 0x83, 0x0a, 0xae, 0x5b, 0xc9, 0x7e, 0x02, 0x6d, 0x42, 0xd5, 0x1d, 0x39, 0x3d, 0xef, 0xa4, + 0xe7, 0x7b, 0x2f, 0xa9, 0x7c, 0xcc, 0x56, 0xdc, 0x91, 0xf3, 0xb3, 0x13, 0xec, 0xbd, 0xa4, 0xf1, + 0xf3, 0x60, 0xf1, 0x82, 0xcf, 0x83, 0xc7, 0x50, 0x33, 0x1d, 0x3b, 0xee, 0xe9, 0x97, 0x8a, 0xf7, + 0xf4, 0xa6, 0x63, 0x47, 0x3d, 0xfd, 0x26, 0x54, 0x1d, 0xe3, 0x9c, 0x19, 0xd7, 0x73, 0x47, 0x4e, + 0x73, 0x59, 0xd8, 0xe7, 0x18, 0xe7, 0xd8, 0x7b, 0xf9, 0x6c, 0xe4, 0xa0, 0x6d, 0x58, 0xb5, 0x0d, + 0x1a, 0xf4, 0x92, 0xef, 0xed, 0x0a, 0x7f, 0x6f, 0x37, 0xd8, 0xfe, 0xe3, 0xe8, 0xcd, 0xad, 0xdf, + 0x87, 0x6a, 0xb7, 0xd3, 0x66, 0x31, 0xc0, 0x3a, 0xb3, 0x9c, 0xd7, 0xd7, 0x61, 0xe1, 0x30, 0x11, + 0x32, 0x62, 0xc1, 0x00, 0xb3, 0x16, 0xda, 0xc2, 0x6a, 0x9d, 0xe2, 0xd5, 0xa3, 0xbd, 0xa9, 0x57, + 0x4f, 0x69, 0xa6, 0x57, 0x8f, 0xfe, 0xbf, 0x12, 0x6c, 0x1c, 0x19, 0x67, 0xe4, 0xed, 0x37, 0xcf, + 0x85, 0xf0, 0xfd, 0x29, 0xac, 0xf1, 0xfc, 0x6d, 0x27, 0xec, 0xe1, 0x30, 0xaf, 0x46, 0xe2, 0xc4, + 0x95, 0xe0, 0x3c, 0x23, 0xfa, 0x09, 0x34, 0x52, 0xb0, 0x18, 0xc2, 0xc8, 0x96, 0x42, 0x54, 0x0a, + 0x67, 0x71, 0x86, 0x2f, 0x5b, 0xf1, 0x17, 0x2f, 0x5c, 0xf1, 0x19, 0xb0, 0x4a, 0x94, 0x5b, 0xe2, + 0x28, 0x17, 0x2e, 0x19, 0x6a, 0x42, 0xcc, 0x35, 0x05, 0x2f, 0x7f, 0x04, 0xcb, 0x33, 0xdc, 0x78, + 0xc4, 0x93, 0x4d, 0xd8, 0x72, 0x26, 0x61, 0xf5, 0xaf, 0x34, 0xa8, 0x77, 0x8c, 0xc0, 0x78, 0xe6, + 0x99, 0xe4, 0xf9, 0x8c, 0x35, 0xb4, 0xc0, 0xf8, 0xec, 0x06, 0x54, 0x58, 0xae, 0xd1, 0xc0, 0x70, + 0x86, 0xdc, 0x88, 0x79, 0x1c, 0x6f, 0xe8, 0x3e, 0x34, 0x24, 0x26, 0x88, 0x2b, 0xa0, 0x53, 0x9c, + 0xb2, 0x07, 0xb5, 0x93, 0xb8, 0xe6, 0x4e, 0x7a, 0x41, 0x26, 0x4a, 0x33, 0x4e, 0xf1, 0xe8, 0x0f, + 0xa1, 0x9a, 0xf8, 0x73, 0x42, 0x1d, 0x6c, 0xc2, 0xd2, 0x71, 0x42, 0x4f, 0x05, 0x87, 0x4b, 0xf6, + 0x22, 0xae, 0x4b, 0x60, 0x14, 0x6d, 0x2a, 0xeb, 0x72, 0xb8, 0x07, 0x44, 0x89, 0xe6, 0xbf, 0xd1, + 0x0f, 0xd2, 0x13, 0x93, 0x77, 0x95, 0x31, 0xc4, 0x85, 0xf0, 0xb6, 0x2f, 0x85, 0x8a, 0x45, 0xde, + 0x38, 0x5f, 0x6a, 0x50, 0x0b, 0x6f, 0x90, 0x17, 0x88, 0x66, 0x3c, 0x76, 0x17, 0x76, 0x84, 0x4b, + 0xf6, 0xcf, 0x19, 0xf1, 0x69, 0x18, 0x4b, 0x65, 0x1c, 0x2e, 0xd1, 0x0f, 0x61, 0x39, 0xea, 0x13, + 0xcb, 0x63, 0x53, 0x26, 0x75, 0x58, 0x1c, 0x71, 0xe8, 0xff, 0xd6, 0xf8, 0xd4, 0x0a, 0x93, 0xbe, + 0x77, 0x46, 0xfc, 0x57, 0xa9, 0xd9, 0xc0, 0xc5, 0xc3, 0xe9, 0x41, 0xc2, 0x96, 0x82, 0x3d, 0x6b, + 0xc4, 0x80, 0x1e, 0xc4, 0xb7, 0x55, 0x1e, 0xdb, 0x41, 0xa4, 0x83, 0x2d, 0xbe, 0xd0, 0x3f, 0x8a, + 0x11, 0x47, 0xfa, 0x1c, 0x57, 0x3a, 0x90, 0xbc, 0x7b, 0x0f, 0xd6, 0x72, 0xd1, 0x81, 0x1a, 0x00, + 0x9f, 0xb8, 0x7d, 0xcf, 0x19, 0xda, 0x24, 0x20, 0xab, 0x73, 0xa8, 0x06, 0xcb, 0x8f, 0xc2, 0x95, + 0xd6, 0xfe, 0x4f, 0x1d, 0xaa, 0x2c, 0x20, 0x8e, 0xc4, 0x27, 0x22, 0x34, 0x04, 0xc4, 0xdf, 0xc0, + 0xce, 0xd0, 0x73, 0xa3, 0x99, 0x15, 0xfa, 0x68, 0x0c, 0x8c, 0xe4, 0x49, 0xa5, 0x0b, 0x5a, 0x77, + 0xc6, 0x70, 0x64, 0xc8, 0xf5, 0x39, 0xe4, 0x70, 0x8d, 0xac, 0x8c, 0x3e, 0xb7, 0xfa, 0x2f, 0xc2, + 0xde, 0x61, 0x82, 0xc6, 0x0c, 0x69, 0xa8, 0x31, 0x33, 0x0a, 0x93, 0x0b, 0x31, 0x2f, 0x09, 0x03, + 0x4c, 0x9f, 0x43, 0x5f, 0xc0, 0x3a, 0x7b, 0x14, 0x46, 0x6f, 0xd3, 0x50, 0x61, 0x7b, 0xbc, 0xc2, + 0x1c, 0xf1, 0x05, 0x55, 0x1a, 0x50, 0x4b, 0x7e, 0xa1, 0x42, 0xaa, 0xb1, 0xb9, 0xe2, 0x23, 0x5a, + 0xeb, 0xfd, 0xa9, 0x74, 0x91, 0x8a, 0x7d, 0x58, 0xe0, 0xcd, 0x2c, 0x52, 0x45, 0x7f, 0xf2, 0x6b, + 0x54, 0x6b, 0xd2, 0xcb, 0x59, 0x9f, 0x43, 0xbf, 0x82, 0x95, 0xcc, 0x77, 0x00, 0xf4, 0x81, 0x42, + 0xa4, 0xfa, 0x8b, 0x4e, 0xeb, 0x6e, 0x11, 0xd2, 0xa4, 0x5f, 0x92, 0xb3, 0x72, 0xa5, 0x5f, 0x14, + 0xf3, 0x7e, 0xa5, 0x5f, 0x54, 0x43, 0x77, 0x7d, 0x0e, 0x0d, 0xa0, 0x91, 0x1e, 0x01, 0xa0, 0x6d, + 0x05, 0xb3, 0x72, 0x2a, 0xda, 0xfa, 0xa0, 0x00, 0x65, 0xa4, 0xc8, 0x81, 0xd5, 0xec, 0xa4, 0x17, + 0xdd, 0x9d, 0x28, 0x20, 0x9d, 0x2f, 0x1f, 0x16, 0xa2, 0x8d, 0xd4, 0xbd, 0xe2, 0x51, 0x9c, 0x9b, + 0x34, 0xa2, 0x1d, 0xb5, 0x98, 0x71, 0x23, 0xd0, 0xd6, 0x6e, 0x61, 0xfa, 0x48, 0x35, 0x81, 0xb5, + 0xdc, 0xe4, 0x10, 0x7d, 0x38, 0x49, 0x4e, 0x66, 0x34, 0xd1, 0x9a, 0x3e, 0xdb, 0xd4, 0xe7, 0xd0, + 0x57, 0xa2, 0x4c, 0xa8, 0xa6, 0x71, 0xe8, 0x9e, 0x5a, 0xdb, 0x84, 0x31, 0x62, 0xab, 0x7d, 0x11, + 0x96, 0xe8, 0xac, 0xaf, 0x39, 0xc4, 0x2b, 0x26, 0x5a, 0x59, 0x7c, 0x0a, 0xe5, 0x8d, 0x1f, 0xd5, + 0xb5, 0xee, 0x5d, 0x80, 0x23, 0x32, 0xc0, 0xcb, 0x8e, 0xec, 0x43, 0xb8, 0xda, 0x9d, 0x1a, 0x9c, + 0xb3, 0x61, 0xd5, 0xe7, 0xb0, 0x92, 0x69, 0xf8, 0x95, 0xf9, 0xaf, 0x7e, 0x14, 0x14, 0x00, 0x97, + 0x4c, 0xc5, 0x44, 0x63, 0x92, 0x4c, 0x51, 0x55, 0x5b, 0x77, 0x8b, 0x90, 0x86, 0x07, 0x69, 0xff, + 0xad, 0x0c, 0xcb, 0x61, 0xa7, 0x73, 0x05, 0x55, 0xed, 0x0a, 0xca, 0xcc, 0xe7, 0xb0, 0x92, 0x99, + 0x05, 0x2a, 0xbd, 0xab, 0x9e, 0x17, 0x4e, 0xbb, 0xba, 0xcf, 0xa0, 0x9e, 0x1a, 0xee, 0xa1, 0xf7, + 0xc7, 0x15, 0x9a, 0x2c, 0x5a, 0x4f, 0x16, 0xbc, 0x77, 0xff, 0x17, 0xf7, 0x06, 0x56, 0x70, 0x3a, + 0x3a, 0x66, 0xff, 0xec, 0x0a, 0xd2, 0x6f, 0x59, 0x9e, 0xfc, 0xb5, 0x1b, 0x3a, 0x68, 0x97, 0x73, + 0xef, 0x32, 0x35, 0xc3, 0xe3, 0xe3, 0x45, 0xbe, 0xba, 0xff, 0xff, 0x00, 0x00, 0x00, 0xff, 0xff, + 0x34, 0x34, 0xef, 0xf0, 0xdf, 0x22, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/proxynode/impl.go b/internal/proxynode/impl.go index 6a753929e..70f4cb4c2 100644 --- a/internal/proxynode/impl.go +++ b/internal/proxynode/impl.go @@ -1445,7 +1445,7 @@ func (node *ProxyNode) GetPersistentSegmentInfo(ctx context.Context, req *milvus SegmentID: info.ID, CollectionID: info.CollectionID, PartitionID: info.PartitionID, - NumRows: info.NumRows, + NumRows: info.NumOfRows, State: info.State, } } diff --git a/internal/proxynode/task.go b/internal/proxynode/task.go index ce125d6a1..ce799d726 100644 --- a/internal/proxynode/task.go +++ b/internal/proxynode/task.go @@ -2892,9 +2892,9 @@ func (gibpt *GetIndexBuildProgressTask) Execute(ctx context.Context) error { indexed := int64(0) for _, info := range infoResp.Infos { - total += info.NumRows + total += info.NumOfRows if buildFinishMap[buildIndexMap[info.ID]] { - indexed += info.NumRows + indexed += info.NumOfRows } } -- GitLab