diff --git a/internal/proto/query_coord.proto b/internal/proto/query_coord.proto index 726fd44e1a..c941b5ca31 100644 --- a/internal/proto/query_coord.proto +++ b/internal/proto/query_coord.proto @@ -236,10 +236,12 @@ message SegmentLoadInfo { message LoadSegmentsRequest { common.MsgBase base = 1; - int64 nodeID = 2; + int64 dst_nodeID = 2; repeated SegmentLoadInfo infos = 3; schema.CollectionSchema schema = 4; - TriggerCondition load_condition = 5; + TriggerCondition load_condition = 5; // deprecated + int64 source_nodeID = 6; + int64 collectionID = 7; } message ReleaseSegmentsRequest { @@ -275,6 +277,8 @@ message QueryChannelInfo { int64 collectionID = 1; string query_channelID = 2; string query_result_channelID = 3; + repeated SegmentInfo global_sealed_segments = 4; + internal.MsgPosition seek_position = 5; } message CollectionInfo { diff --git a/internal/proto/querypb/query_coord.pb.go b/internal/proto/querypb/query_coord.pb.go index bd1f3cb45b..d66b3ced7c 100644 --- a/internal/proto/querypb/query_coord.pb.go +++ b/internal/proto/querypb/query_coord.pb.go @@ -1498,10 +1498,12 @@ func (m *SegmentLoadInfo) GetDeltalogs() []*datapb.DeltaLogInfo { type LoadSegmentsRequest struct { Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + DstNodeID int64 `protobuf:"varint,2,opt,name=dst_nodeID,json=dstNodeID,proto3" json:"dst_nodeID,omitempty"` Infos []*SegmentLoadInfo `protobuf:"bytes,3,rep,name=infos,proto3" json:"infos,omitempty"` Schema *schemapb.CollectionSchema `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"` LoadCondition TriggerCondition `protobuf:"varint,5,opt,name=load_condition,json=loadCondition,proto3,enum=milvus.proto.query.TriggerCondition" json:"load_condition,omitempty"` + SourceNodeID int64 `protobuf:"varint,6,opt,name=source_nodeID,json=sourceNodeID,proto3" json:"source_nodeID,omitempty"` + CollectionID int64 `protobuf:"varint,7,opt,name=collectionID,proto3" json:"collectionID,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1539,9 +1541,9 @@ func (m *LoadSegmentsRequest) GetBase() *commonpb.MsgBase { return nil } -func (m *LoadSegmentsRequest) GetNodeID() int64 { +func (m *LoadSegmentsRequest) GetDstNodeID() int64 { if m != nil { - return m.NodeID + return m.DstNodeID } return 0 } @@ -1567,6 +1569,20 @@ func (m *LoadSegmentsRequest) GetLoadCondition() TriggerCondition { return TriggerCondition_handoff } +func (m *LoadSegmentsRequest) GetSourceNodeID() int64 { + if m != nil { + return m.SourceNodeID + } + return 0 +} + +func (m *LoadSegmentsRequest) GetCollectionID() int64 { + if m != nil { + return m.CollectionID + } + return 0 +} + type ReleaseSegmentsRequest struct { Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` @@ -1695,12 +1711,14 @@ func (m *DmChannelInfo) GetChannelIDs() []string { } type QueryChannelInfo struct { - CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - QueryChannelID string `protobuf:"bytes,2,opt,name=query_channelID,json=queryChannelID,proto3" json:"query_channelID,omitempty"` - QueryResultChannelID string `protobuf:"bytes,3,opt,name=query_result_channelID,json=queryResultChannelID,proto3" json:"query_result_channelID,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + QueryChannelID string `protobuf:"bytes,2,opt,name=query_channelID,json=queryChannelID,proto3" json:"query_channelID,omitempty"` + QueryResultChannelID string `protobuf:"bytes,3,opt,name=query_result_channelID,json=queryResultChannelID,proto3" json:"query_result_channelID,omitempty"` + GlobalSealedSegments []*SegmentInfo `protobuf:"bytes,4,rep,name=global_sealed_segments,json=globalSealedSegments,proto3" json:"global_sealed_segments,omitempty"` + SeekPosition *internalpb.MsgPosition `protobuf:"bytes,5,opt,name=seek_position,json=seekPosition,proto3" json:"seek_position,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *QueryChannelInfo) Reset() { *m = QueryChannelInfo{} } @@ -1749,6 +1767,20 @@ func (m *QueryChannelInfo) GetQueryResultChannelID() string { return "" } +func (m *QueryChannelInfo) GetGlobalSealedSegments() []*SegmentInfo { + if m != nil { + return m.GlobalSealedSegments + } + return nil +} + +func (m *QueryChannelInfo) GetSeekPosition() *internalpb.MsgPosition { + if m != nil { + return m.SeekPosition + } + return nil +} + type CollectionInfo struct { CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` PartitionIDs []int64 `protobuf:"varint,2,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` @@ -2152,143 +2184,145 @@ func init() { func init() { proto.RegisterFile("query_coord.proto", fileDescriptor_aab7cc9a69ed26e8) } var fileDescriptor_aab7cc9a69ed26e8 = []byte{ - // 2171 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcf, 0x6f, 0xdc, 0xc6, - 0xf5, 0x17, 0xf7, 0x97, 0x76, 0xdf, 0xfe, 0xa2, 0xc7, 0x96, 0xb2, 0xde, 0xaf, 0xed, 0xf8, 0x4b, - 0xc7, 0xb1, 0xa3, 0x34, 0x72, 0x22, 0xa7, 0x45, 0x0d, 0x34, 0x87, 0x58, 0x1b, 0x2b, 0x9b, 0xda, - 0xb2, 0x4a, 0x39, 0x29, 0x6a, 0x18, 0x60, 0xb9, 0xcb, 0xd1, 0x8a, 0x30, 0xc9, 0x59, 0x73, 0xb8, - 0x96, 0xe5, 0x43, 0x4f, 0x3d, 0xf4, 0xd2, 0x63, 0x4f, 0x2d, 0x0a, 0x14, 0xe8, 0x0f, 0xf4, 0xd0, - 0x1e, 0x7b, 0xe8, 0x29, 0x97, 0xde, 0xfb, 0x0f, 0xb4, 0x40, 0xd1, 0xde, 0xfb, 0x2f, 0x14, 0xf3, - 0x83, 0x5c, 0x92, 0x3b, 0x2b, 0xad, 0xa5, 0xb8, 0x36, 0x82, 0xde, 0x38, 0x6f, 0xde, 0xbc, 0xf7, - 0xe6, 0xfd, 0xf8, 0xcc, 0xcc, 0x23, 0x9c, 0x79, 0x32, 0xc1, 0xe1, 0xa1, 0x35, 0x24, 0x24, 0x74, - 0xd6, 0xc7, 0x21, 0x89, 0x08, 0x42, 0xbe, 0xeb, 0x3d, 0x9d, 0x50, 0x31, 0x5a, 0xe7, 0xf3, 0xdd, - 0xc6, 0x90, 0xf8, 0x3e, 0x09, 0x04, 0xad, 0xdb, 0x48, 0x73, 0x74, 0x5b, 0x6e, 0x10, 0xe1, 0x30, - 0xb0, 0xbd, 0x78, 0x96, 0x0e, 0xf7, 0xb1, 0x6f, 0xcb, 0x91, 0xee, 0xd8, 0x91, 0x9d, 0x96, 0x6f, - 0xfc, 0x58, 0x83, 0xd5, 0xdd, 0x7d, 0x72, 0xb0, 0x49, 0x3c, 0x0f, 0x0f, 0x23, 0x97, 0x04, 0xd4, - 0xc4, 0x4f, 0x26, 0x98, 0x46, 0xe8, 0x7d, 0x28, 0x0d, 0x6c, 0x8a, 0x3b, 0xda, 0x65, 0xed, 0x7a, - 0x7d, 0xe3, 0xc2, 0x7a, 0xc6, 0x12, 0x69, 0xc2, 0x3d, 0x3a, 0xba, 0x6d, 0x53, 0x6c, 0x72, 0x4e, - 0x84, 0xa0, 0xe4, 0x0c, 0xfa, 0xbd, 0x4e, 0xe1, 0xb2, 0x76, 0xbd, 0x68, 0xf2, 0x6f, 0xf4, 0x16, - 0x34, 0x87, 0x89, 0xec, 0x7e, 0x8f, 0x76, 0x8a, 0x97, 0x8b, 0xd7, 0x8b, 0x66, 0x96, 0x68, 0xfc, - 0x4e, 0x83, 0x37, 0x66, 0xcc, 0xa0, 0x63, 0x12, 0x50, 0x8c, 0x6e, 0x42, 0x85, 0x46, 0x76, 0x34, - 0xa1, 0xd2, 0x92, 0xff, 0x53, 0x5a, 0xb2, 0xcb, 0x59, 0x4c, 0xc9, 0x3a, 0xab, 0xb6, 0xa0, 0x50, - 0x8b, 0x3e, 0x80, 0x73, 0x6e, 0x70, 0x0f, 0xfb, 0x24, 0x3c, 0xb4, 0xc6, 0x38, 0x1c, 0xe2, 0x20, - 0xb2, 0x47, 0x38, 0xb6, 0xf1, 0x6c, 0x3c, 0xb7, 0x33, 0x9d, 0x32, 0x7e, 0xa3, 0xc1, 0x0a, 0xb3, - 0x74, 0xc7, 0x0e, 0x23, 0xf7, 0x25, 0xf8, 0xcb, 0x80, 0x46, 0xda, 0xc6, 0x4e, 0x91, 0xcf, 0x65, - 0x68, 0x8c, 0x67, 0x1c, 0xab, 0x67, 0x7b, 0x2b, 0x71, 0x73, 0x33, 0x34, 0xe3, 0xd7, 0x32, 0xb0, - 0x69, 0x3b, 0x4f, 0xe3, 0xd0, 0xbc, 0xce, 0xc2, 0xac, 0xce, 0x93, 0xb8, 0xf3, 0x4b, 0x0d, 0x56, - 0xee, 0x12, 0xdb, 0x99, 0x06, 0xfe, 0xbf, 0xef, 0xce, 0x8f, 0xa0, 0x22, 0xaa, 0xa4, 0x53, 0xe2, - 0xba, 0xae, 0x66, 0x75, 0xc9, 0x0a, 0x9a, 0x5a, 0xb8, 0xcb, 0x09, 0xa6, 0x5c, 0x64, 0xfc, 0x42, - 0x83, 0x8e, 0x89, 0x3d, 0x6c, 0x53, 0xfc, 0x2a, 0x77, 0xb1, 0x0a, 0x95, 0x80, 0x38, 0xb8, 0xdf, - 0xe3, 0xbb, 0x28, 0x9a, 0x72, 0x64, 0xfc, 0x4b, 0x7a, 0xf8, 0x35, 0x4f, 0xd8, 0x54, 0x14, 0xca, - 0x27, 0x89, 0xc2, 0x97, 0xd3, 0x28, 0xbc, 0xee, 0x3b, 0x9d, 0x46, 0xaa, 0x9c, 0x89, 0xd4, 0x0f, - 0xe0, 0xfc, 0x66, 0x88, 0xed, 0x08, 0x7f, 0x8f, 0xc1, 0xfc, 0xe6, 0xbe, 0x1d, 0x04, 0xd8, 0x8b, - 0xb7, 0x90, 0x57, 0xae, 0x29, 0x94, 0x77, 0x60, 0x79, 0x1c, 0x92, 0x67, 0x87, 0x89, 0xdd, 0xf1, - 0xd0, 0xf8, 0x95, 0x06, 0x5d, 0x95, 0xec, 0xd3, 0x20, 0xc2, 0x35, 0x68, 0x87, 0xc2, 0x38, 0x6b, - 0x28, 0xe4, 0x71, 0xad, 0x35, 0xb3, 0x25, 0xc9, 0x52, 0x0b, 0xba, 0x0a, 0xad, 0x10, 0xd3, 0x89, - 0x37, 0xe5, 0x2b, 0x72, 0xbe, 0xa6, 0xa0, 0x4a, 0x36, 0xe3, 0xf7, 0x1a, 0x9c, 0xdf, 0xc2, 0x51, - 0x12, 0x3d, 0xa6, 0x0e, 0xbf, 0xa6, 0xe8, 0xfa, 0x4b, 0x0d, 0xda, 0x39, 0x43, 0xd1, 0x65, 0xa8, - 0xa7, 0x78, 0x64, 0x80, 0xd2, 0x24, 0xf4, 0x6d, 0x28, 0x33, 0xdf, 0x61, 0x6e, 0x52, 0x6b, 0xc3, - 0x58, 0x9f, 0x3d, 0xdc, 0xd7, 0xb3, 0x52, 0x4d, 0xb1, 0x00, 0xdd, 0x80, 0xb3, 0x0a, 0x64, 0x95, - 0xe6, 0xa3, 0x59, 0x60, 0x35, 0xfe, 0xa0, 0x41, 0x57, 0xe5, 0xcc, 0xd3, 0x04, 0xfc, 0x21, 0xac, - 0x26, 0xbb, 0xb1, 0x1c, 0x4c, 0x87, 0xa1, 0x3b, 0xe6, 0x65, 0xc6, 0x0f, 0x83, 0xfa, 0xc6, 0x95, - 0xe3, 0xf7, 0x43, 0xcd, 0x95, 0x44, 0x44, 0x2f, 0x25, 0xc1, 0xf8, 0xa9, 0x06, 0x2b, 0x5b, 0x38, - 0xda, 0xc5, 0x23, 0x1f, 0x07, 0x51, 0x3f, 0xd8, 0x23, 0x27, 0x0f, 0xfc, 0x25, 0x00, 0x2a, 0xe5, - 0x24, 0x07, 0x55, 0x8a, 0xb2, 0x48, 0x12, 0x18, 0x7f, 0x2b, 0x40, 0x3d, 0x65, 0x0c, 0xba, 0x00, - 0xb5, 0x44, 0x82, 0x0c, 0xed, 0x94, 0x30, 0x23, 0xb1, 0xa0, 0x48, 0xab, 0x5c, 0x7a, 0x14, 0x67, - 0xd3, 0x63, 0x0e, 0x82, 0xa3, 0xf3, 0x50, 0xf5, 0xb1, 0x6f, 0x51, 0xf7, 0x39, 0x96, 0x88, 0xb1, - 0xec, 0x63, 0x7f, 0xd7, 0x7d, 0x8e, 0xd9, 0x54, 0x30, 0xf1, 0xad, 0x90, 0x1c, 0xd0, 0x4e, 0x45, - 0x4c, 0x05, 0x13, 0xdf, 0x24, 0x07, 0x14, 0x5d, 0x04, 0x70, 0x03, 0x07, 0x3f, 0xb3, 0x02, 0xdb, - 0xc7, 0x9d, 0x65, 0x5e, 0x71, 0x35, 0x4e, 0xd9, 0xb6, 0x7d, 0xcc, 0xb0, 0x82, 0x0f, 0xfa, 0xbd, - 0x4e, 0x55, 0x2c, 0x94, 0x43, 0xb6, 0x55, 0x59, 0xa7, 0xfd, 0x5e, 0xa7, 0x26, 0xd6, 0x25, 0x04, - 0xf4, 0x09, 0x34, 0xe5, 0xbe, 0x2d, 0x91, 0xcb, 0xc0, 0x73, 0xf9, 0xb2, 0x2a, 0xf6, 0xd2, 0x81, - 0x22, 0x93, 0x1b, 0x34, 0x35, 0xe2, 0xf7, 0xce, 0x7c, 0xbc, 0x4f, 0x93, 0x9b, 0xdf, 0x84, 0xb2, - 0x1b, 0xec, 0x91, 0x38, 0x15, 0xdf, 0x3c, 0xc2, 0x1c, 0xae, 0x4c, 0x70, 0x1b, 0x7f, 0x2a, 0xc2, - 0xea, 0xc7, 0x8e, 0xa3, 0x02, 0xdc, 0x17, 0xcf, 0xbb, 0x69, 0xfc, 0x0a, 0x99, 0xf8, 0x2d, 0x02, - 0x3a, 0xef, 0xc2, 0x99, 0x1c, 0x98, 0xca, 0x34, 0xa8, 0x99, 0x7a, 0x16, 0x4e, 0xfb, 0x3d, 0xf4, - 0x0e, 0xe8, 0x59, 0x40, 0x95, 0x47, 0x49, 0xcd, 0x6c, 0x67, 0x20, 0xb5, 0xdf, 0x43, 0xdf, 0x82, - 0x37, 0x46, 0x1e, 0x19, 0xd8, 0x9e, 0x45, 0xb1, 0xed, 0x61, 0xc7, 0x9a, 0x66, 0x71, 0x85, 0x17, - 0xc6, 0x8a, 0x98, 0xde, 0xe5, 0xb3, 0xbb, 0x49, 0x46, 0x6f, 0xb1, 0x30, 0xe3, 0xc7, 0xd6, 0x98, - 0x50, 0x9e, 0x9e, 0x3c, 0x81, 0xea, 0x79, 0xc8, 0x4a, 0x1e, 0x1b, 0xf7, 0xe8, 0x68, 0x47, 0x72, - 0xb2, 0x40, 0xe3, 0xc7, 0xf1, 0x08, 0x7d, 0x0e, 0xab, 0x4a, 0x03, 0x68, 0xa7, 0xba, 0x58, 0xa4, - 0xce, 0x29, 0x0c, 0xa4, 0xc6, 0x3f, 0x34, 0x38, 0x6f, 0x62, 0x9f, 0x3c, 0xc5, 0x5f, 0xdb, 0xd8, - 0x19, 0xff, 0x2c, 0xc0, 0xea, 0xf7, 0xed, 0x68, 0xb8, 0xdf, 0xf3, 0x25, 0x91, 0xbe, 0x9a, 0x0d, - 0xe6, 0xa0, 0xab, 0x34, 0x0b, 0x5d, 0x49, 0xf9, 0x95, 0x55, 0x41, 0x65, 0xaf, 0xce, 0xf5, 0x2f, - 0xe2, 0xfd, 0x4e, 0xcb, 0x2f, 0x75, 0xe7, 0xab, 0x9c, 0xe0, 0xce, 0x87, 0x36, 0xa1, 0x89, 0x9f, - 0x0d, 0xbd, 0x89, 0x83, 0x2d, 0xa1, 0x7d, 0x99, 0x6b, 0xbf, 0xa4, 0xd0, 0x9e, 0xce, 0xa8, 0x86, - 0x5c, 0xd4, 0xe7, 0x10, 0xf0, 0x93, 0x22, 0xb4, 0xe5, 0x2c, 0xbb, 0x26, 0x2f, 0x80, 0xf6, 0x39, - 0x77, 0x14, 0x66, 0xdd, 0xb1, 0x88, 0x53, 0xe3, 0xeb, 0x49, 0x29, 0x75, 0x3d, 0xb9, 0x08, 0xb0, - 0xe7, 0x4d, 0xe8, 0xbe, 0x15, 0xb9, 0x7e, 0x8c, 0xf5, 0x35, 0x4e, 0x79, 0xe0, 0xfa, 0x18, 0x7d, - 0x0c, 0x8d, 0x81, 0x1b, 0x78, 0x64, 0x64, 0x8d, 0xed, 0x68, 0x9f, 0xf2, 0x0a, 0x56, 0x6f, 0xf7, - 0x8e, 0x8b, 0x3d, 0xe7, 0x36, 0xe7, 0x35, 0xeb, 0x62, 0xcd, 0x0e, 0x5b, 0x82, 0x2e, 0x41, 0x9d, - 0x1d, 0x18, 0x64, 0x4f, 0x9c, 0x19, 0xcb, 0x42, 0x45, 0x30, 0xf1, 0xef, 0xef, 0xf1, 0x53, 0xe3, - 0x3b, 0x50, 0x63, 0x88, 0x4a, 0x3d, 0x32, 0x8a, 0x2b, 0xf4, 0x38, 0xf9, 0xd3, 0x05, 0xe8, 0x23, - 0xa8, 0x39, 0xd8, 0x8b, 0x6c, 0xbe, 0xba, 0x36, 0x37, 0x15, 0x7a, 0x8c, 0xe7, 0x2e, 0x19, 0xf1, - 0x68, 0x4c, 0x57, 0x18, 0xbf, 0x2d, 0xc0, 0x59, 0x16, 0x83, 0xb8, 0xca, 0xbf, 0xfa, 0x6c, 0xbf, - 0x15, 0xe7, 0x69, 0x71, 0xfe, 0x8d, 0x25, 0x97, 0x0c, 0xb3, 0xb9, 0x7a, 0x92, 0x57, 0x22, 0xfa, - 0x2e, 0xb4, 0x3c, 0x62, 0x3b, 0xd6, 0x90, 0x04, 0x8e, 0x40, 0xd4, 0x32, 0x3f, 0x38, 0xdf, 0x52, - 0x99, 0xf0, 0x20, 0x74, 0x47, 0x23, 0x1c, 0x6e, 0xc6, 0xbc, 0x66, 0xd3, 0xe3, 0x6f, 0x64, 0x39, - 0x34, 0xfe, 0xae, 0xc1, 0xaa, 0x7c, 0xec, 0xbc, 0x3c, 0x5f, 0xc5, 0x09, 0x5a, 0x3c, 0xe2, 0xfe, - 0x5c, 0x5a, 0xe0, 0xfe, 0x5c, 0x56, 0x3c, 0x81, 0xb2, 0x57, 0xb4, 0x4a, 0xfe, 0x8a, 0x66, 0x3c, - 0x80, 0x66, 0x02, 0x7a, 0xbc, 0x22, 0xaf, 0x40, 0x53, 0x98, 0x65, 0x31, 0x4f, 0x60, 0x27, 0x7e, - 0xff, 0x08, 0xe2, 0x5d, 0x4e, 0x63, 0x52, 0x13, 0x50, 0x15, 0x37, 0x81, 0x9a, 0x99, 0xa2, 0x18, - 0x3f, 0xd3, 0x40, 0x4f, 0x1f, 0x17, 0x5c, 0xf2, 0x22, 0x0f, 0xab, 0x6b, 0xd0, 0x96, 0xad, 0xb9, - 0x04, 0xb3, 0xe5, 0x53, 0xe7, 0x49, 0x5a, 0x5c, 0x0f, 0x7d, 0x08, 0xab, 0x82, 0x71, 0x06, 0xe3, - 0xc5, 0x93, 0xe7, 0x1c, 0x9f, 0x35, 0x73, 0x40, 0xff, 0xd7, 0x22, 0xb4, 0xa6, 0x89, 0xb3, 0xb0, - 0x55, 0x8b, 0xb4, 0x64, 0xb6, 0x41, 0x9f, 0xde, 0xd9, 0xf9, 0x85, 0xed, 0xc8, 0xdc, 0xcf, 0xdf, - 0xd6, 0xdb, 0xe3, 0xdc, 0x23, 0xe7, 0x0e, 0x34, 0xe5, 0x9e, 0x24, 0xe4, 0x96, 0xb8, 0xb0, 0xff, - 0x57, 0x09, 0xcb, 0x44, 0xd0, 0x6c, 0xa4, 0xf0, 0x9f, 0xa2, 0x5b, 0x50, 0xe3, 0xe5, 0x10, 0x1d, - 0x8e, 0xb1, 0xac, 0x84, 0x0b, 0x2a, 0x19, 0x2c, 0xb2, 0x0f, 0x0e, 0xc7, 0xd8, 0xac, 0x7a, 0xf2, - 0xeb, 0xb4, 0x87, 0xc6, 0x4d, 0x58, 0x09, 0x45, 0xe9, 0x38, 0x56, 0xc6, 0x7d, 0xcb, 0xdc, 0x7d, - 0xe7, 0xe2, 0xc9, 0x9d, 0xb4, 0x1b, 0xe7, 0xbc, 0xbf, 0xaa, 0x73, 0xdf, 0x5f, 0x3f, 0x82, 0xf6, - 0xa7, 0x76, 0xe0, 0x90, 0xbd, 0xbd, 0xb8, 0x40, 0x4f, 0x50, 0x99, 0xb7, 0xb2, 0x97, 0xda, 0x17, - 0x40, 0x2b, 0xe3, 0xe7, 0x05, 0x58, 0x65, 0xb4, 0xdb, 0xb6, 0x67, 0x07, 0x43, 0xbc, 0xf8, 0x53, - 0xe6, 0xab, 0x39, 0xdc, 0xae, 0x40, 0x93, 0x92, 0x49, 0x38, 0xc4, 0x56, 0xe6, 0x45, 0xd3, 0x10, - 0xc4, 0x6d, 0x01, 0x30, 0x17, 0x01, 0x1c, 0x1a, 0x59, 0x99, 0x5e, 0x48, 0xcd, 0xa1, 0x91, 0x9c, - 0x7e, 0x13, 0xea, 0x52, 0x86, 0x43, 0x02, 0xcc, 0x83, 0x5d, 0x35, 0x41, 0x90, 0x7a, 0x24, 0xe0, - 0x8f, 0x1f, 0xb6, 0x9e, 0xcf, 0x2e, 0xf3, 0xd9, 0x65, 0x87, 0x46, 0x7c, 0xea, 0x22, 0xc0, 0x53, - 0xdb, 0x73, 0x1d, 0x9e, 0xa4, 0x3c, 0x4c, 0x55, 0xb3, 0xc6, 0x29, 0xcc, 0x05, 0xc6, 0x9f, 0x35, - 0x40, 0x29, 0xef, 0x9c, 0x1c, 0x3b, 0xaf, 0x42, 0x2b, 0xb3, 0xcf, 0xa4, 0xcf, 0x9c, 0xde, 0x28, - 0x65, 0xe0, 0x3f, 0x10, 0xaa, 0xac, 0x10, 0xdb, 0x94, 0x04, 0xdc, 0x69, 0x0b, 0x83, 0xff, 0x20, - 0x36, 0x93, 0x2d, 0x35, 0xfe, 0x58, 0x80, 0x4e, 0xf6, 0x36, 0xcc, 0xca, 0x6c, 0xc4, 0xaf, 0x33, - 0x27, 0xd8, 0xc2, 0x15, 0x68, 0x92, 0xc0, 0x73, 0x83, 0x24, 0x54, 0xf2, 0xf1, 0x2a, 0x88, 0x32, - 0x16, 0x9f, 0x42, 0x5b, 0x32, 0x25, 0xd7, 0xf7, 0xe2, 0x62, 0xd7, 0xf7, 0x96, 0x58, 0x97, 0x54, - 0xc1, 0x55, 0x68, 0x91, 0xbd, 0xbd, 0xb4, 0x3e, 0x91, 0x1a, 0x4d, 0x49, 0x95, 0x0a, 0x3f, 0x03, - 0x3d, 0x66, 0x4b, 0x34, 0x96, 0x17, 0xd3, 0xd8, 0x96, 0x0b, 0x63, 0x95, 0x6b, 0xcf, 0xa1, 0x95, - 0xc5, 0x35, 0xd4, 0x80, 0xea, 0x36, 0x89, 0x3e, 0x79, 0xe6, 0xd2, 0x48, 0x5f, 0x42, 0x2d, 0x80, - 0x6d, 0x12, 0xed, 0x84, 0x98, 0xe2, 0x20, 0xd2, 0x35, 0x04, 0x50, 0xb9, 0x1f, 0xf4, 0x5c, 0xfa, - 0x58, 0x2f, 0xa0, 0xb3, 0xb2, 0xcf, 0x63, 0x7b, 0x7d, 0x59, 0xe4, 0x7a, 0x91, 0x2d, 0x4f, 0x46, - 0x25, 0xa4, 0x43, 0x23, 0x61, 0xd9, 0xda, 0xf9, 0x5c, 0x2f, 0xa3, 0x1a, 0x94, 0xc5, 0x67, 0x65, - 0xed, 0x3e, 0xe8, 0xf9, 0x78, 0xa2, 0x3a, 0x2c, 0xef, 0x0b, 0x6c, 0xd0, 0x97, 0x50, 0x1b, 0xea, - 0xde, 0x34, 0x13, 0x75, 0x8d, 0x11, 0x46, 0xe1, 0x78, 0x28, 0x73, 0x52, 0x2f, 0x30, 0x6d, 0xcc, - 0x53, 0x3d, 0x72, 0x10, 0xe8, 0xc5, 0xb5, 0xcf, 0xa0, 0x91, 0x7e, 0x56, 0xa3, 0x2a, 0x94, 0xb6, - 0x49, 0x80, 0xf5, 0x25, 0x26, 0x76, 0x2b, 0x24, 0x07, 0x6e, 0x30, 0x12, 0x7b, 0xb8, 0x13, 0x92, - 0xe7, 0x38, 0xd0, 0x0b, 0x6c, 0x82, 0xbd, 0xbd, 0xd8, 0x44, 0x91, 0x4d, 0x88, 0x87, 0x98, 0x5e, - 0x5a, 0xfb, 0x00, 0xaa, 0x31, 0xbe, 0xa2, 0x33, 0xd0, 0xcc, 0x74, 0x89, 0xf5, 0x25, 0x84, 0xc4, - 0x95, 0x65, 0x8a, 0xa4, 0xba, 0xb6, 0xf1, 0x6f, 0x00, 0x10, 0x47, 0x28, 0x21, 0xa1, 0x83, 0xc6, - 0x80, 0xb6, 0x70, 0xb4, 0x49, 0xfc, 0x31, 0x09, 0x62, 0x93, 0x28, 0x7a, 0x7f, 0xce, 0x2b, 0x71, - 0x96, 0x55, 0xee, 0xb2, 0xfb, 0xf6, 0x9c, 0x15, 0x39, 0x76, 0x63, 0x09, 0xf9, 0x5c, 0x23, 0xbb, - 0x0e, 0x3f, 0x70, 0x87, 0x8f, 0xe3, 0x16, 0xe3, 0x11, 0x1a, 0x73, 0xac, 0xb1, 0xc6, 0x1c, 0x98, - 0xca, 0xc1, 0x6e, 0x14, 0xba, 0xc1, 0x28, 0x6e, 0x45, 0x18, 0x4b, 0xe8, 0x09, 0x9c, 0xdb, 0xc2, - 0x5c, 0xbb, 0x4b, 0x23, 0x77, 0x48, 0x63, 0x85, 0x1b, 0xf3, 0x15, 0xce, 0x30, 0xbf, 0xa0, 0x4a, - 0x0f, 0xda, 0xb9, 0x5f, 0x61, 0x68, 0x4d, 0x99, 0xf3, 0xca, 0xdf, 0x76, 0xdd, 0x77, 0x17, 0xe2, - 0x4d, 0xb4, 0xb9, 0xd0, 0xca, 0xfe, 0x26, 0x42, 0xef, 0xcc, 0x13, 0x30, 0xd3, 0x57, 0xef, 0xae, - 0x2d, 0xc2, 0x9a, 0xa8, 0x7a, 0x08, 0xad, 0xec, 0x8f, 0x08, 0xb5, 0x2a, 0xe5, 0xcf, 0x8a, 0xee, - 0x51, 0x5d, 0x20, 0x63, 0x09, 0xfd, 0x10, 0xce, 0xcc, 0x74, 0xff, 0xd1, 0x37, 0x54, 0xe2, 0xe7, - 0xfd, 0x24, 0x38, 0x4e, 0x83, 0xb4, 0x7e, 0xea, 0xc5, 0xf9, 0xd6, 0xcf, 0xfc, 0x06, 0x5a, 0xdc, - 0xfa, 0x94, 0xf8, 0xa3, 0xac, 0x7f, 0x61, 0x0d, 0x13, 0x40, 0xb3, 0xfd, 0x7f, 0xf4, 0x9e, 0x4a, - 0xc5, 0xdc, 0x7f, 0x10, 0xdd, 0xf5, 0x45, 0xd9, 0x93, 0x90, 0x4f, 0x78, 0xb5, 0xe6, 0x3b, 0xe5, - 0x4a, 0xb5, 0x73, 0x5b, 0xff, 0x6a, 0xb5, 0xf3, 0x9b, 0xdb, 0x22, 0xa9, 0xb3, 0xcd, 0x45, 0x75, - 0xac, 0x94, 0x0d, 0x67, 0x75, 0x52, 0xab, 0x7b, 0x95, 0xc6, 0x12, 0xb2, 0x00, 0xb6, 0x70, 0x74, - 0x0f, 0x47, 0xa1, 0x3b, 0xa4, 0xe8, 0x6d, 0x65, 0x89, 0x4f, 0x19, 0x62, 0x1d, 0xd7, 0x8e, 0xe5, - 0x8b, 0x15, 0x6c, 0xfc, 0xa5, 0x06, 0x35, 0xee, 0x5d, 0x76, 0x32, 0xfe, 0x0f, 0x70, 0x5f, 0x02, - 0xe0, 0x3e, 0x82, 0x76, 0xae, 0x07, 0xac, 0x06, 0x5c, 0x75, 0xa3, 0xf8, 0xb8, 0xca, 0x1b, 0x00, - 0x9a, 0x6d, 0x54, 0xaa, 0x4b, 0x60, 0x6e, 0x43, 0xf3, 0x38, 0x1d, 0x8f, 0xa0, 0x9d, 0x6b, 0x14, - 0xaa, 0x77, 0xa0, 0xee, 0x26, 0x1e, 0x27, 0xfd, 0x0b, 0x68, 0xa4, 0xbb, 0x32, 0xe8, 0xda, 0x3c, - 0xdc, 0xcb, 0xf5, 0x22, 0x5e, 0x3d, 0xea, 0xbd, 0xfc, 0x53, 0xe1, 0x11, 0xb4, 0x73, 0x8d, 0x18, - 0xb5, 0xe7, 0xd5, 0xdd, 0x9a, 0xe3, 0xa4, 0x7f, 0x8d, 0x70, 0xec, 0xf6, 0x87, 0x0f, 0x37, 0x46, - 0x6e, 0xb4, 0x3f, 0x19, 0xb0, 0x5d, 0xde, 0x10, 0x9c, 0xef, 0xb9, 0x44, 0x7e, 0xdd, 0x88, 0x0b, - 0xfa, 0x06, 0x97, 0x74, 0x83, 0x5b, 0x3b, 0x1e, 0x0c, 0x2a, 0x7c, 0x78, 0xf3, 0x3f, 0x01, 0x00, - 0x00, 0xff, 0xff, 0xd0, 0xfe, 0x66, 0x42, 0x1b, 0x25, 0x00, 0x00, + // 2200 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x19, 0x4d, 0x6f, 0x1c, 0x49, + 0xd5, 0x3d, 0xdf, 0xf3, 0xe6, 0xab, 0x53, 0x89, 0xbd, 0x93, 0x21, 0xc9, 0x86, 0xce, 0x66, 0x93, + 0xf5, 0xb2, 0xce, 0xae, 0xb3, 0x20, 0x22, 0xb1, 0x87, 0x8d, 0x67, 0xe3, 0x9d, 0x25, 0x71, 0x4c, + 0xdb, 0xbb, 0x88, 0x28, 0x52, 0xd3, 0x33, 0x5d, 0x1e, 0xb7, 0xd2, 0xdd, 0x35, 0xe9, 0xea, 0x89, + 0xe3, 0x1c, 0x38, 0x71, 0xe0, 0xc2, 0x2f, 0x00, 0x21, 0x21, 0x81, 0x10, 0x07, 0xb8, 0xc1, 0x81, + 0xd3, 0x5e, 0xb8, 0xf3, 0x07, 0x40, 0x42, 0x70, 0xe7, 0xc6, 0x19, 0x75, 0x55, 0x75, 0x4f, 0x7f, + 0xd4, 0xd8, 0x63, 0x9b, 0x90, 0x68, 0xc5, 0xad, 0xfb, 0xd5, 0xab, 0xf7, 0x5e, 0xbd, 0xef, 0x7a, + 0x05, 0xe7, 0x9e, 0x4e, 0xb1, 0x7f, 0x68, 0x8c, 0x08, 0xf1, 0xad, 0xb5, 0x89, 0x4f, 0x02, 0x82, + 0x90, 0x6b, 0x3b, 0xcf, 0xa6, 0x94, 0xff, 0xad, 0xb1, 0xf5, 0x5e, 0x73, 0x44, 0x5c, 0x97, 0x78, + 0x1c, 0xd6, 0x6b, 0x26, 0x31, 0x7a, 0x6d, 0xdb, 0x0b, 0xb0, 0xef, 0x99, 0x4e, 0xb4, 0x4a, 0x47, + 0xfb, 0xd8, 0x35, 0xc5, 0x9f, 0x6a, 0x99, 0x81, 0x99, 0xa4, 0xaf, 0xfd, 0x58, 0x81, 0x95, 0x9d, + 0x7d, 0x72, 0xb0, 0x41, 0x1c, 0x07, 0x8f, 0x02, 0x9b, 0x78, 0x54, 0xc7, 0x4f, 0xa7, 0x98, 0x06, + 0xe8, 0x7d, 0x28, 0x0d, 0x4d, 0x8a, 0xbb, 0xca, 0x55, 0xe5, 0x66, 0x63, 0xfd, 0xd2, 0x5a, 0x4a, + 0x12, 0x21, 0xc2, 0x03, 0x3a, 0xbe, 0x6b, 0x52, 0xac, 0x33, 0x4c, 0x84, 0xa0, 0x64, 0x0d, 0x07, + 0xfd, 0x6e, 0xe1, 0xaa, 0x72, 0xb3, 0xa8, 0xb3, 0x6f, 0xf4, 0x16, 0xb4, 0x46, 0x31, 0xed, 0x41, + 0x9f, 0x76, 0x8b, 0x57, 0x8b, 0x37, 0x8b, 0x7a, 0x1a, 0xa8, 0xfd, 0x46, 0x81, 0x37, 0x72, 0x62, + 0xd0, 0x09, 0xf1, 0x28, 0x46, 0xb7, 0xa1, 0x42, 0x03, 0x33, 0x98, 0x52, 0x21, 0xc9, 0xd7, 0xa4, + 0x92, 0xec, 0x30, 0x14, 0x5d, 0xa0, 0xe6, 0xd9, 0x16, 0x24, 0x6c, 0xd1, 0x07, 0x70, 0xc1, 0xf6, + 0x1e, 0x60, 0x97, 0xf8, 0x87, 0xc6, 0x04, 0xfb, 0x23, 0xec, 0x05, 0xe6, 0x18, 0x47, 0x32, 0x9e, + 0x8f, 0xd6, 0xb6, 0x67, 0x4b, 0xda, 0xaf, 0x15, 0x58, 0x0e, 0x25, 0xdd, 0x36, 0xfd, 0xc0, 0x7e, + 0x09, 0xfa, 0xd2, 0xa0, 0x99, 0x94, 0xb1, 0x5b, 0x64, 0x6b, 0x29, 0x58, 0x88, 0x33, 0x89, 0xd8, + 0x87, 0x67, 0x2b, 0x31, 0x71, 0x53, 0x30, 0xed, 0x57, 0xc2, 0xb0, 0x49, 0x39, 0xcf, 0xa2, 0xd0, + 0x2c, 0xcf, 0x42, 0x9e, 0xe7, 0x69, 0xd4, 0xf9, 0xa5, 0x02, 0xcb, 0xf7, 0x89, 0x69, 0xcd, 0x0c, + 0xff, 0xbf, 0x57, 0xe7, 0x47, 0x50, 0xe1, 0x51, 0xd2, 0x2d, 0x31, 0x5e, 0xd7, 0xd3, 0xbc, 0x44, + 0x04, 0xcd, 0x24, 0xdc, 0x61, 0x00, 0x5d, 0x6c, 0xd2, 0x7e, 0xae, 0x40, 0x57, 0xc7, 0x0e, 0x36, + 0x29, 0x7e, 0x95, 0xa7, 0x58, 0x81, 0x8a, 0x47, 0x2c, 0x3c, 0xe8, 0xb3, 0x53, 0x14, 0x75, 0xf1, + 0xa7, 0xfd, 0x53, 0x68, 0xf8, 0x35, 0x77, 0xd8, 0x84, 0x15, 0xca, 0xa7, 0xb1, 0xc2, 0x97, 0x33, + 0x2b, 0xbc, 0xee, 0x27, 0x9d, 0x59, 0xaa, 0x9c, 0xb2, 0xd4, 0x0f, 0xe0, 0xe2, 0x86, 0x8f, 0xcd, + 0x00, 0x7f, 0x2f, 0x4c, 0xf3, 0x1b, 0xfb, 0xa6, 0xe7, 0x61, 0x27, 0x3a, 0x42, 0x96, 0xb9, 0x22, + 0x61, 0xde, 0x85, 0xea, 0xc4, 0x27, 0xcf, 0x0f, 0x63, 0xb9, 0xa3, 0x5f, 0xed, 0x97, 0x0a, 0xf4, + 0x64, 0xb4, 0xcf, 0x92, 0x11, 0x6e, 0x40, 0xc7, 0xe7, 0xc2, 0x19, 0x23, 0x4e, 0x8f, 0x71, 0xad, + 0xeb, 0x6d, 0x01, 0x16, 0x5c, 0xd0, 0x75, 0x68, 0xfb, 0x98, 0x4e, 0x9d, 0x19, 0x5e, 0x91, 0xe1, + 0xb5, 0x38, 0x54, 0xa0, 0x69, 0xbf, 0x55, 0xe0, 0xe2, 0x26, 0x0e, 0x62, 0xeb, 0x85, 0xec, 0xf0, + 0x6b, 0x9a, 0x5d, 0x7f, 0xa1, 0x40, 0x27, 0x23, 0x28, 0xba, 0x0a, 0x8d, 0x04, 0x8e, 0x30, 0x50, + 0x12, 0x84, 0xbe, 0x0d, 0xe5, 0x50, 0x77, 0x98, 0x89, 0xd4, 0x5e, 0xd7, 0xd6, 0xf2, 0xc5, 0x7d, + 0x2d, 0x4d, 0x55, 0xe7, 0x1b, 0xd0, 0x2d, 0x38, 0x2f, 0xc9, 0xac, 0x42, 0x7c, 0x94, 0x4f, 0xac, + 0xda, 0xef, 0x14, 0xe8, 0xc9, 0x94, 0x79, 0x16, 0x83, 0x3f, 0x82, 0x95, 0xf8, 0x34, 0x86, 0x85, + 0xe9, 0xc8, 0xb7, 0x27, 0x2c, 0xcc, 0x58, 0x31, 0x68, 0xac, 0x5f, 0x3b, 0xfe, 0x3c, 0x54, 0x5f, + 0x8e, 0x49, 0xf4, 0x13, 0x14, 0xb4, 0x9f, 0x2a, 0xb0, 0xbc, 0x89, 0x83, 0x1d, 0x3c, 0x76, 0xb1, + 0x17, 0x0c, 0xbc, 0x3d, 0x72, 0x7a, 0xc3, 0x5f, 0x01, 0xa0, 0x82, 0x4e, 0x5c, 0xa8, 0x12, 0x90, + 0x45, 0x9c, 0x40, 0xfb, 0x6b, 0x01, 0x1a, 0x09, 0x61, 0xd0, 0x25, 0xa8, 0xc7, 0x14, 0x84, 0x69, + 0x67, 0x80, 0x1c, 0xc5, 0x82, 0xc4, 0xad, 0x32, 0xee, 0x51, 0xcc, 0xbb, 0xc7, 0x9c, 0x0c, 0x8e, + 0x2e, 0x42, 0xcd, 0xc5, 0xae, 0x41, 0xed, 0x17, 0x58, 0x64, 0x8c, 0xaa, 0x8b, 0xdd, 0x1d, 0xfb, + 0x05, 0x0e, 0x97, 0xbc, 0xa9, 0x6b, 0xf8, 0xe4, 0x80, 0x76, 0x2b, 0x7c, 0xc9, 0x9b, 0xba, 0x3a, + 0x39, 0xa0, 0xe8, 0x32, 0x80, 0xed, 0x59, 0xf8, 0xb9, 0xe1, 0x99, 0x2e, 0xee, 0x56, 0x59, 0xc4, + 0xd5, 0x19, 0x64, 0xcb, 0x74, 0x71, 0x98, 0x2b, 0xd8, 0xcf, 0xa0, 0xdf, 0xad, 0xf1, 0x8d, 0xe2, + 0x37, 0x3c, 0xaa, 0x88, 0xd3, 0x41, 0xbf, 0x5b, 0xe7, 0xfb, 0x62, 0x00, 0xfa, 0x04, 0x5a, 0xe2, + 0xdc, 0x06, 0xf7, 0x65, 0x60, 0xbe, 0x7c, 0x55, 0x66, 0x7b, 0xa1, 0x40, 0xee, 0xc9, 0x4d, 0x9a, + 0xf8, 0x63, 0x7d, 0x67, 0xd6, 0xde, 0x67, 0xf1, 0xcd, 0x6f, 0x42, 0xd9, 0xf6, 0xf6, 0x48, 0xe4, + 0x8a, 0x6f, 0x1e, 0x21, 0x0e, 0x63, 0xc6, 0xb1, 0xb5, 0x3f, 0x16, 0x61, 0xe5, 0x63, 0xcb, 0x92, + 0x25, 0xdc, 0x93, 0xfb, 0xdd, 0xcc, 0x7e, 0x85, 0x94, 0xfd, 0x16, 0x49, 0x3a, 0xef, 0xc2, 0xb9, + 0x4c, 0x32, 0x15, 0x6e, 0x50, 0xd7, 0xd5, 0x74, 0x3a, 0x1d, 0xf4, 0xd1, 0x3b, 0xa0, 0xa6, 0x13, + 0xaa, 0x28, 0x25, 0x75, 0xbd, 0x93, 0x4a, 0xa9, 0x83, 0x3e, 0xfa, 0x16, 0xbc, 0x31, 0x76, 0xc8, + 0xd0, 0x74, 0x0c, 0x8a, 0x4d, 0x07, 0x5b, 0xc6, 0xcc, 0x8b, 0x2b, 0x2c, 0x30, 0x96, 0xf9, 0xf2, + 0x0e, 0x5b, 0xdd, 0x89, 0x3d, 0x7a, 0x33, 0x34, 0x33, 0x7e, 0x62, 0x4c, 0x08, 0x65, 0xee, 0xc9, + 0x1c, 0xa8, 0x91, 0x4d, 0x59, 0xf1, 0x65, 0xe3, 0x01, 0x1d, 0x6f, 0x0b, 0xcc, 0xd0, 0xd0, 0xf8, + 0x49, 0xf4, 0x87, 0x3e, 0x87, 0x15, 0xa9, 0x00, 0xb4, 0x5b, 0x5b, 0xcc, 0x52, 0x17, 0x24, 0x02, + 0x52, 0xed, 0xef, 0x0a, 0x5c, 0xd4, 0xb1, 0x4b, 0x9e, 0xe1, 0xaf, 0xac, 0xed, 0xb4, 0x7f, 0x14, + 0x60, 0xe5, 0xfb, 0x66, 0x30, 0xda, 0xef, 0xbb, 0x02, 0x48, 0x5f, 0xcd, 0x01, 0x33, 0xa9, 0xab, + 0x94, 0x4f, 0x5d, 0x71, 0xf8, 0x95, 0x65, 0x46, 0x0d, 0x6f, 0x9d, 0x6b, 0x5f, 0x44, 0xe7, 0x9d, + 0x85, 0x5f, 0xa2, 0xe7, 0xab, 0x9c, 0xa2, 0xe7, 0x43, 0x1b, 0xd0, 0xc2, 0xcf, 0x47, 0xce, 0xd4, + 0xc2, 0x06, 0xe7, 0x5e, 0x65, 0xdc, 0xaf, 0x48, 0xb8, 0x27, 0x3d, 0xaa, 0x29, 0x36, 0x0d, 0x58, + 0x0a, 0xf8, 0x49, 0x11, 0x3a, 0x62, 0x35, 0x6c, 0x93, 0x17, 0xc8, 0xf6, 0x19, 0x75, 0x14, 0xf2, + 0xea, 0x58, 0x44, 0xa9, 0x51, 0x7b, 0x52, 0x4a, 0xb4, 0x27, 0x97, 0x01, 0xf6, 0x9c, 0x29, 0xdd, + 0x37, 0x02, 0xdb, 0x8d, 0x72, 0x7d, 0x9d, 0x41, 0x76, 0x6d, 0x17, 0xa3, 0x8f, 0xa1, 0x39, 0xb4, + 0x3d, 0x87, 0x8c, 0x8d, 0x89, 0x19, 0xec, 0x53, 0x16, 0xc1, 0xf2, 0xe3, 0xde, 0xb3, 0xb1, 0x63, + 0xdd, 0x65, 0xb8, 0x7a, 0x83, 0xef, 0xd9, 0x0e, 0xb7, 0xa0, 0x2b, 0xd0, 0x08, 0x0b, 0x06, 0xd9, + 0xe3, 0x35, 0xa3, 0xca, 0x59, 0x78, 0x53, 0xf7, 0xe1, 0x1e, 0xab, 0x1a, 0xdf, 0x81, 0x7a, 0x98, + 0x51, 0xa9, 0x43, 0xc6, 0x51, 0x84, 0x1e, 0x47, 0x7f, 0xb6, 0x01, 0x7d, 0x04, 0x75, 0x0b, 0x3b, + 0x81, 0xc9, 0x76, 0xd7, 0xe7, 0xba, 0x42, 0x3f, 0xc4, 0xb9, 0x4f, 0xc6, 0xcc, 0x1a, 0xb3, 0x1d, + 0xda, 0xbf, 0x0b, 0x70, 0x3e, 0xb4, 0x41, 0x14, 0xe5, 0xa7, 0xf7, 0xf6, 0xcb, 0x00, 0x16, 0x0d, + 0x8c, 0x94, 0xc7, 0xd7, 0x2d, 0x1a, 0x6c, 0x71, 0xa7, 0xbf, 0x13, 0xb9, 0x6b, 0x71, 0x7e, 0xe3, + 0x92, 0xf1, 0x89, 0xbc, 0xcb, 0x9e, 0xe6, 0xb2, 0x88, 0xbe, 0x0b, 0x6d, 0x87, 0x98, 0x96, 0x31, + 0x22, 0x9e, 0xc5, 0x13, 0x6b, 0x99, 0xd5, 0xcf, 0xb7, 0x64, 0x22, 0xec, 0xfa, 0xf6, 0x78, 0x8c, + 0xfd, 0x8d, 0x08, 0x57, 0x6f, 0x39, 0xec, 0xaa, 0x2c, 0x7e, 0xd1, 0x35, 0x68, 0x51, 0x32, 0xf5, + 0x47, 0x38, 0x3a, 0x28, 0x6f, 0x01, 0x9a, 0x1c, 0xb8, 0x25, 0x0f, 0xf0, 0xaa, 0xa4, 0xdb, 0xf9, + 0x9b, 0x02, 0x2b, 0xe2, 0xf2, 0x74, 0x76, 0xdd, 0xcf, 0xcb, 0x34, 0x91, 0xc3, 0x17, 0x8f, 0xe8, + 0xc7, 0x4b, 0x0b, 0xf4, 0xe3, 0x65, 0xc9, 0x95, 0x2a, 0xdd, 0xf2, 0x55, 0xb2, 0x2d, 0x9f, 0xb6, + 0x0b, 0xad, 0x38, 0x89, 0xb2, 0x08, 0xbf, 0x06, 0x2d, 0x2e, 0x96, 0x11, 0xaa, 0x14, 0x5b, 0xd1, + 0x7d, 0x8a, 0x03, 0xef, 0x33, 0x58, 0x48, 0x35, 0x4e, 0xd2, 0xbc, 0xb3, 0xa8, 0xeb, 0x09, 0x88, + 0xf6, 0x87, 0x02, 0xa8, 0xc9, 0xf2, 0xc3, 0x28, 0x2f, 0x72, 0x51, 0xbb, 0x01, 0x1d, 0x31, 0xea, + 0x8b, 0x6b, 0x80, 0xb8, 0x3a, 0x3d, 0x4d, 0x92, 0xeb, 0xa3, 0x0f, 0x61, 0x85, 0x23, 0xe6, 0x6a, + 0x06, 0xbf, 0x42, 0x5d, 0x60, 0xab, 0x7a, 0xa6, 0xe8, 0xcf, 0xaf, 0xb9, 0xa5, 0x33, 0xd4, 0xdc, + 0x7c, 0x4f, 0x50, 0x3e, 0x5d, 0x4f, 0xa0, 0xfd, 0xa5, 0x08, 0xed, 0x59, 0x84, 0x2c, 0xac, 0xb5, + 0x45, 0x46, 0x50, 0x5b, 0xa0, 0xce, 0xee, 0x28, 0xac, 0x41, 0x3d, 0x32, 0xc8, 0xb3, 0xb7, 0x93, + 0xce, 0x24, 0x73, 0xa9, 0xbb, 0x07, 0x2d, 0xa1, 0x73, 0x51, 0x62, 0xb8, 0x06, 0xbf, 0x2e, 0x23, + 0x96, 0xf2, 0x30, 0xbd, 0x99, 0xa8, 0x77, 0x14, 0xdd, 0x81, 0x3a, 0x8b, 0xfb, 0xe0, 0x70, 0x82, + 0x45, 0xc8, 0x5f, 0x92, 0xd1, 0x08, 0x3d, 0x6f, 0xf7, 0x70, 0x82, 0xf5, 0x9a, 0x23, 0xbe, 0xce, + 0x5a, 0x24, 0x6f, 0xc3, 0xb2, 0xcf, 0x43, 0xdb, 0x32, 0x52, 0xea, 0xab, 0x32, 0xf5, 0x5d, 0x88, + 0x16, 0xb7, 0x93, 0x6a, 0x9c, 0x73, 0xdf, 0xac, 0xcd, 0xbd, 0x6f, 0xfe, 0x08, 0x3a, 0x9f, 0x9a, + 0x9e, 0x45, 0xf6, 0xf6, 0x62, 0x77, 0x39, 0x79, 0xe6, 0xb8, 0x93, 0x6e, 0xe2, 0x4f, 0x90, 0x96, + 0xb5, 0x9f, 0x15, 0x60, 0x25, 0x84, 0xdd, 0x35, 0x1d, 0xd3, 0x1b, 0xe1, 0xc5, 0xaf, 0x6e, 0xff, + 0x9d, 0x62, 0x9e, 0xcb, 0xc4, 0x25, 0x49, 0x26, 0x4e, 0x17, 0xa5, 0x72, 0xb6, 0x28, 0xbd, 0x09, + 0x0d, 0x41, 0xc3, 0x22, 0x1e, 0x66, 0xc6, 0xae, 0xe9, 0xc0, 0x41, 0x7d, 0xe2, 0xb1, 0xcb, 0x5e, + 0xb8, 0x9f, 0xad, 0x56, 0xd9, 0x6a, 0xd5, 0xa2, 0x01, 0x5b, 0xba, 0x0c, 0xf0, 0xcc, 0x74, 0x6c, + 0x8b, 0x39, 0x29, 0x33, 0x53, 0x4d, 0xaf, 0x33, 0x48, 0xa8, 0x02, 0xed, 0x4f, 0x0a, 0xa0, 0x84, + 0x76, 0x4e, 0x9f, 0xdb, 0xaf, 0x43, 0x3b, 0x75, 0xce, 0x78, 0xae, 0x9e, 0x3c, 0x28, 0x0d, 0xab, + 0xdc, 0x90, 0xb3, 0x32, 0x7c, 0x6c, 0x52, 0xe2, 0x31, 0xa5, 0x2d, 0x5c, 0xe5, 0x86, 0x91, 0x98, + 0xe1, 0x56, 0xed, 0xf7, 0x05, 0xe8, 0xa6, 0x33, 0x51, 0x18, 0x66, 0x63, 0xd6, 0xbe, 0x9d, 0xe2, + 0x08, 0xd7, 0xa0, 0x45, 0x3c, 0xc7, 0xf6, 0x70, 0xba, 0x3b, 0x68, 0x72, 0xa0, 0xb0, 0xc5, 0xa7, + 0xd0, 0x11, 0x48, 0x71, 0xea, 0x2c, 0x2e, 0x96, 0x3a, 0xdb, 0x7c, 0x5f, 0x1c, 0x05, 0xd7, 0xa1, + 0x4d, 0xf6, 0xf6, 0x92, 0xfc, 0xb8, 0x6b, 0xb4, 0x04, 0x54, 0x30, 0xfc, 0x0c, 0xd4, 0x08, 0x2d, + 0xe6, 0x58, 0x5e, 0x8c, 0x63, 0x47, 0x6c, 0x8c, 0x58, 0xae, 0xbe, 0x80, 0x76, 0x3a, 0xaf, 0xa1, + 0x26, 0xd4, 0xb6, 0x48, 0xf0, 0xc9, 0x73, 0x9b, 0x06, 0xea, 0x12, 0x6a, 0x03, 0x6c, 0x91, 0x60, + 0xdb, 0xc7, 0x14, 0x7b, 0x81, 0xaa, 0x20, 0x80, 0xca, 0x43, 0xaf, 0x6f, 0xd3, 0x27, 0x6a, 0x01, + 0x9d, 0x17, 0x73, 0x2d, 0xd3, 0x19, 0x88, 0x20, 0x57, 0x8b, 0xe1, 0xf6, 0xf8, 0xaf, 0x84, 0x54, + 0x68, 0xc6, 0x28, 0x9b, 0xdb, 0x9f, 0xab, 0x65, 0x54, 0x87, 0x32, 0xff, 0xac, 0xac, 0x3e, 0x04, + 0x35, 0x6b, 0x4f, 0xd4, 0x80, 0xea, 0x3e, 0xcf, 0x0d, 0xea, 0x12, 0xea, 0x40, 0xc3, 0x99, 0x79, + 0xa2, 0xaa, 0x84, 0x80, 0xb1, 0x3f, 0x19, 0x09, 0x9f, 0x54, 0x0b, 0x21, 0xb7, 0x50, 0x53, 0x7d, + 0x72, 0xe0, 0xa9, 0xc5, 0xd5, 0xcf, 0xa0, 0x99, 0x1c, 0x23, 0xa0, 0x1a, 0x94, 0xb6, 0x88, 0x87, + 0xd5, 0xa5, 0x90, 0xec, 0xa6, 0x4f, 0x0e, 0x6c, 0x6f, 0xcc, 0xcf, 0x70, 0xcf, 0x27, 0x2f, 0xb0, + 0xa7, 0x16, 0xc2, 0x85, 0xb0, 0xee, 0x85, 0x0b, 0xc5, 0x70, 0x81, 0x17, 0x41, 0xb5, 0xb4, 0xfa, + 0x01, 0xd4, 0xa2, 0xfc, 0x8a, 0xce, 0x41, 0x2b, 0x35, 0x15, 0x57, 0x97, 0x10, 0xe2, 0xbd, 0xd9, + 0x2c, 0x93, 0xaa, 0xca, 0xfa, 0xbf, 0x00, 0x80, 0x97, 0x78, 0x42, 0x7c, 0x0b, 0x4d, 0x00, 0x6d, + 0xe2, 0x60, 0x83, 0xb8, 0x13, 0xe2, 0x45, 0x22, 0x51, 0xf4, 0xfe, 0x9c, 0x0a, 0x98, 0x47, 0x15, + 0xa7, 0xec, 0xbd, 0x3d, 0x67, 0x47, 0x06, 0x5d, 0x5b, 0x42, 0x2e, 0xe3, 0x18, 0xb6, 0xff, 0xbb, + 0xf6, 0xe8, 0x49, 0x34, 0x52, 0x3d, 0x82, 0x63, 0x06, 0x35, 0xe2, 0x98, 0x49, 0xa6, 0xe2, 0x67, + 0x27, 0xf0, 0x6d, 0x6f, 0x1c, 0x8d, 0x5e, 0xb4, 0x25, 0xf4, 0x14, 0x2e, 0x6c, 0x62, 0xc6, 0xdd, + 0xa6, 0x81, 0x3d, 0xa2, 0x11, 0xc3, 0xf5, 0xf9, 0x0c, 0x73, 0xc8, 0x27, 0x64, 0xe9, 0x40, 0x27, + 0xf3, 0xf4, 0x87, 0x56, 0xa5, 0x3e, 0x2f, 0x7d, 0xa6, 0xec, 0xbd, 0xbb, 0x10, 0x6e, 0xcc, 0xcd, + 0x86, 0x76, 0xfa, 0x59, 0x0c, 0xbd, 0x33, 0x8f, 0x40, 0xee, 0x1d, 0xa1, 0xb7, 0xba, 0x08, 0x6a, + 0xcc, 0xea, 0x11, 0xb4, 0xd3, 0x0f, 0x2f, 0x72, 0x56, 0xd2, 0xc7, 0x99, 0xde, 0x51, 0x53, 0x2f, + 0x6d, 0x09, 0xfd, 0x10, 0xce, 0xe5, 0x5e, 0x3b, 0xd0, 0x37, 0x64, 0xe4, 0xe7, 0x3d, 0x8a, 0x1c, + 0xc7, 0x41, 0x48, 0x3f, 0xd3, 0xe2, 0x7c, 0xe9, 0x73, 0xcf, 0x5e, 0x8b, 0x4b, 0x9f, 0x20, 0x7f, + 0x94, 0xf4, 0x27, 0xe6, 0x30, 0x05, 0x94, 0x7f, 0xef, 0x40, 0xef, 0xc9, 0x58, 0xcc, 0x7d, 0x73, + 0xe9, 0xad, 0x2d, 0x8a, 0x1e, 0x9b, 0x7c, 0xca, 0xa2, 0x35, 0xfb, 0x32, 0x20, 0x65, 0x3b, 0xf7, + 0xa9, 0x43, 0xce, 0x76, 0xfe, 0x30, 0x9f, 0x3b, 0x75, 0x7a, 0x98, 0x2a, 0xb7, 0x95, 0x74, 0xc0, + 0x2e, 0x77, 0x6a, 0xf9, 0x6c, 0x56, 0x5b, 0x42, 0x06, 0xc0, 0x26, 0x0e, 0x1e, 0xe0, 0xc0, 0xb7, + 0x47, 0x14, 0xbd, 0x2d, 0x0d, 0xf1, 0x19, 0x42, 0xc4, 0xe3, 0xc6, 0xb1, 0x78, 0x11, 0x83, 0xf5, + 0x3f, 0xd7, 0xa1, 0xce, 0xb4, 0x1b, 0x56, 0xc6, 0xff, 0x27, 0xdc, 0x97, 0x90, 0x70, 0x1f, 0x43, + 0x27, 0x33, 0xf3, 0x96, 0x27, 0x5c, 0xf9, 0x60, 0xfc, 0xb8, 0xc8, 0x1b, 0x02, 0xca, 0x0f, 0x66, + 0xe5, 0x21, 0x30, 0x77, 0x80, 0x7b, 0x1c, 0x8f, 0xc7, 0xd0, 0xc9, 0x0c, 0x46, 0xe5, 0x27, 0x90, + 0x4f, 0x4f, 0x8f, 0xa3, 0xfe, 0x05, 0x34, 0x93, 0x53, 0x28, 0x74, 0x63, 0x5e, 0xde, 0xcb, 0xcc, + 0x4a, 0x5e, 0x7d, 0xd6, 0x7b, 0xf9, 0x55, 0xe1, 0x31, 0x74, 0x32, 0x83, 0x22, 0xb9, 0xe6, 0xe5, + 0xd3, 0xa4, 0xe3, 0xa8, 0x7f, 0x85, 0xf2, 0xd8, 0xdd, 0x0f, 0x1f, 0xad, 0x8f, 0xed, 0x60, 0x7f, + 0x3a, 0x0c, 0x4f, 0x79, 0x8b, 0x63, 0xbe, 0x67, 0x13, 0xf1, 0x75, 0x2b, 0x0a, 0xe8, 0x5b, 0x8c, + 0xd2, 0x2d, 0x26, 0xed, 0x64, 0x38, 0xac, 0xb0, 0xdf, 0xdb, 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, + 0xa8, 0x96, 0x54, 0x9d, 0x0b, 0x26, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/querycoord/cluster.go b/internal/querycoord/cluster.go index 0f2eda4c22..7a1ae1ada9 100644 --- a/internal/querycoord/cluster.go +++ b/internal/querycoord/cluster.go @@ -214,40 +214,12 @@ func (c *queryNodeCluster) loadSegments(ctx context.Context, nodeID int64, in *q defer c.Unlock() if node, ok := c.nodes[nodeID]; ok { - segmentInfos := make(map[UniqueID]*querypb.SegmentInfo) - for _, info := range in.Infos { - segmentID := info.SegmentID - segmentInfo, err := c.clusterMeta.getSegmentInfoByID(segmentID) - if err == nil { - segmentInfos[segmentID] = proto.Clone(segmentInfo).(*querypb.SegmentInfo) - if in.LoadCondition != querypb.TriggerCondition_loadBalance { - segmentInfo.SegmentState = querypb.SegmentState_sealing - segmentInfo.NodeID = nodeID - } - } else { - segmentInfo = &querypb.SegmentInfo{ - SegmentID: segmentID, - CollectionID: info.CollectionID, - PartitionID: info.PartitionID, - NodeID: nodeID, - SegmentState: querypb.SegmentState_sealing, - } - } - c.clusterMeta.setSegmentInfo(segmentID, segmentInfo) - } err := node.loadSegments(ctx, in) if err != nil { - for _, info := range in.Infos { - segmentID := info.SegmentID - if _, ok = segmentInfos[segmentID]; ok { - c.clusterMeta.setSegmentInfo(segmentID, segmentInfos[segmentID]) - continue - } - c.clusterMeta.deleteSegmentInfoByID(segmentID) - } log.Debug("LoadSegments: queryNode load segments error", zap.Int64("nodeID", nodeID), zap.String("error info", err.Error())) return err } + return nil } return errors.New("LoadSegments: Can't find query node by nodeID ") @@ -268,9 +240,6 @@ func (c *queryNodeCluster) releaseSegments(ctx context.Context, nodeID int64, in return err } - for _, segmentID := range in.SegmentIDs { - c.clusterMeta.deleteSegmentInfoByID(segmentID) - } return nil } diff --git a/internal/querycoord/cluster_test.go b/internal/querycoord/cluster_test.go index b6ec91573f..eb75accae1 100644 --- a/internal/querycoord/cluster_test.go +++ b/internal/querycoord/cluster_test.go @@ -116,7 +116,7 @@ func TestGrpcRequest(t *testing.T) { assert.Nil(t, err) clusterSession := sessionutil.NewSession(context.Background(), Params.MetaRootPath, Params.EtcdEndpoints) clusterSession.Init(typeutil.QueryCoordRole, Params.Address, true) - meta, err := newMeta(kv) + meta, err := newMeta(baseCtx, kv, nil, nil) assert.Nil(t, err) cluster := &queryNodeCluster{ ctx: baseCtx, @@ -147,8 +147,8 @@ func TestGrpcRequest(t *testing.T) { CollectionID: defaultCollectionID, } loadSegmentReq := &querypb.LoadSegmentsRequest{ - NodeID: nodeID, - Infos: []*querypb.SegmentLoadInfo{segmentLoadInfo}, + DstNodeID: nodeID, + Infos: []*querypb.SegmentLoadInfo{segmentLoadInfo}, } err := cluster.loadSegments(baseCtx, nodeID, loadSegmentReq) assert.Nil(t, err) @@ -166,26 +166,26 @@ func TestGrpcRequest(t *testing.T) { }) t.Run("Test AddQueryChannel", func(t *testing.T) { - reqChannel, resChannel, err := cluster.clusterMeta.getQueryChannel(defaultCollectionID) + info, err := cluster.clusterMeta.getQueryChannelInfoByID(defaultCollectionID) assert.Nil(t, err) addQueryChannelReq := &querypb.AddQueryChannelRequest{ NodeID: nodeID, CollectionID: defaultCollectionID, - RequestChannelID: reqChannel, - ResultChannelID: resChannel, + RequestChannelID: info.QueryChannelID, + ResultChannelID: info.QueryResultChannelID, } err = cluster.addQueryChannel(baseCtx, nodeID, addQueryChannelReq) assert.Nil(t, err) }) t.Run("Test RemoveQueryChannel", func(t *testing.T) { - reqChannel, resChannel, err := cluster.clusterMeta.getQueryChannel(defaultCollectionID) + info, err := cluster.clusterMeta.getQueryChannelInfoByID(defaultCollectionID) assert.Nil(t, err) removeQueryChannelReq := &querypb.RemoveQueryChannelRequest{ NodeID: nodeID, CollectionID: defaultCollectionID, - RequestChannelID: reqChannel, - ResultChannelID: resChannel, + RequestChannelID: info.QueryChannelID, + ResultChannelID: info.QueryResultChannelID, } err = cluster.removeQueryChannel(baseCtx, nodeID, removeQueryChannelReq) assert.Nil(t, err) diff --git a/internal/querycoord/impl.go b/internal/querycoord/impl.go index bd0a1e7baa..08c48bc223 100644 --- a/internal/querycoord/impl.go +++ b/internal/querycoord/impl.go @@ -449,7 +449,7 @@ func (qc *QueryCoord) CreateQueryChannel(ctx context.Context, req *querypb.Creat } collectionID := req.CollectionID - queryChannel, queryResultChannel, err := qc.meta.getQueryChannel(collectionID) + info, err := qc.meta.getQueryChannelInfoByID(collectionID) if err != nil { status.ErrorCode = commonpb.ErrorCode_UnexpectedError status.Reason = err.Error() @@ -461,8 +461,8 @@ func (qc *QueryCoord) CreateQueryChannel(ctx context.Context, req *querypb.Creat return &querypb.CreateQueryChannelResponse{ Status: status, - RequestChannel: queryChannel, - ResultChannel: queryResultChannel, + RequestChannel: info.QueryChannelID, + ResultChannel: info.QueryResultChannelID, }, nil } diff --git a/internal/querycoord/impl_test.go b/internal/querycoord/impl_test.go index 2798b52da1..8ba4e2b685 100644 --- a/internal/querycoord/impl_test.go +++ b/internal/querycoord/impl_test.go @@ -35,6 +35,8 @@ func TestGrpcTask(t *testing.T) { node, err := startQueryNodeServer(ctx) assert.Nil(t, err) + waitQueryNodeOnline(queryCoord.cluster, node.queryNodeID) + t.Run("Test ShowParsOnNotLoadedCol", func(t *testing.T) { res, err := queryCoord.ShowPartitions(ctx, &querypb.ShowPartitionsRequest{ Base: &commonpb.MsgBase{ @@ -641,7 +643,7 @@ func Test_GrpcGetQueryChannelFail(t *testing.T) { kv := &testKv{ returnFn: failedResult, } - meta, err := newMeta(kv) + meta, err := newMeta(context.Background(), kv, nil, nil) assert.Nil(t, err) queryCoord := &QueryCoord{ diff --git a/internal/querycoord/meta.go b/internal/querycoord/meta.go index 2bfac10fb1..6c02ba4772 100644 --- a/internal/querycoord/meta.go +++ b/internal/querycoord/meta.go @@ -12,6 +12,7 @@ package querycoord import ( + "context" "errors" "fmt" "path/filepath" @@ -23,19 +24,28 @@ import ( "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/msgstream" + "github.com/milvus-io/milvus/internal/proto/commonpb" + "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/schemapb" + "github.com/milvus-io/milvus/internal/util/mqclient" ) const ( - collectionMetaPrefix = "queryCoord-collectionMeta" - segmentMetaPrefix = "queryCoord-segmentMeta" - queryChannelMetaPrefix = "queryCoord-queryChannel" + collectionMetaPrefix = "queryCoord-collectionMeta" + segmentMetaPrefix = "queryCoord-segmentMeta" + queryChannelMetaPrefix = "queryCoord-queryChannel" + sealedSegmentChangeInfoPrefix = "queryCoord-sealedSegmentChangeInfo" ) +type col2SegmentInfos = map[UniqueID][]*querypb.SegmentInfo +type col2SealedSegmentChangeInfos = map[UniqueID][]*querypb.SealedSegmentsChangeInfo + // Meta contains information about all loaded collections and partitions, including segment information and vchannel information type Meta interface { reloadFromKV() error + setKvClient(kv kv.MetaKv) showCollections() []*querypb.CollectionInfo hasCollection(collectionID UniqueID) bool @@ -49,10 +59,8 @@ type Meta interface { hasReleasePartition(collectionID UniqueID, partitionID UniqueID) bool releasePartition(collectionID UniqueID, partitionID UniqueID) error - deleteSegmentInfoByID(segmentID UniqueID) error deleteSegmentInfoByNodeID(nodeID UniqueID) error - setSegmentInfo(segmentID UniqueID, info *querypb.SegmentInfo) error - hasSegmentInfo(segmentID UniqueID) bool + setSegmentInfos(segmentInfos map[UniqueID]*querypb.SegmentInfo) error showSegmentInfos(collectionID UniqueID, partitionIDs []UniqueID) []*querypb.SegmentInfo getSegmentInfoByID(segmentID UniqueID) (*querypb.SegmentInfo, error) @@ -63,36 +71,56 @@ type Meta interface { removeDmChannel(collectionID UniqueID, nodeID int64, channels []string) error getQueryChannelInfoByID(collectionID UniqueID) (*querypb.QueryChannelInfo, error) - getQueryChannel(collectionID UniqueID) (string, string, error) + getQueryStreamByID(collectionID UniqueID) (msgstream.MsgStream, error) setLoadType(collectionID UniqueID, loadType querypb.LoadType) error getLoadType(collectionID UniqueID) (querypb.LoadType, error) setLoadPercentage(collectionID UniqueID, partitionID UniqueID, percentage int64, loadType querypb.LoadType) error //printMeta() + saveGlobalSealedSegInfos(saves col2SegmentInfos) (col2SealedSegmentChangeInfos, error) + removeGlobalSealedSegInfos(collectionID UniqueID, partitionIDs []UniqueID) (col2SealedSegmentChangeInfos, error) + sendSealedSegmentChangeInfos(collectionID UniqueID, changeInfos []*querypb.SealedSegmentsChangeInfo) (*querypb.QueryChannelInfo, map[string][]mqclient.MessageID, error) } // MetaReplica records the current load information on all querynodes type MetaReplica struct { - client kv.MetaKv // client of a reliable kv service, i.e. etcd client + ctx context.Context + cancel context.CancelFunc + client kv.MetaKv // client of a reliable kv service, i.e. etcd client + msFactory msgstream.Factory + idAllocator func() (UniqueID, error) - sync.RWMutex + //sync.RWMutex collectionInfos map[UniqueID]*querypb.CollectionInfo + collectionMu sync.RWMutex segmentInfos map[UniqueID]*querypb.SegmentInfo + segmentMu sync.RWMutex queryChannelInfos map[UniqueID]*querypb.QueryChannelInfo + channelMu sync.RWMutex + queryStreams map[UniqueID]msgstream.MsgStream + streamMu sync.RWMutex //partitionStates map[UniqueID]*querypb.PartitionStates } -func newMeta(kv kv.MetaKv) (Meta, error) { +func newMeta(ctx context.Context, kv kv.MetaKv, factory msgstream.Factory, idAllocator func() (UniqueID, error)) (Meta, error) { + childCtx, cancel := context.WithCancel(ctx) collectionInfos := make(map[UniqueID]*querypb.CollectionInfo) segmentInfos := make(map[UniqueID]*querypb.SegmentInfo) queryChannelInfos := make(map[UniqueID]*querypb.QueryChannelInfo) + queryMsgStream := make(map[UniqueID]msgstream.MsgStream) m := &MetaReplica{ - client: kv, + ctx: childCtx, + cancel: cancel, + client: kv, + msFactory: factory, + idAllocator: idAllocator, + collectionInfos: collectionInfos, segmentInfos: segmentInfos, queryChannelInfos: queryChannelInfos, + queryStreams: queryMsgStream, } err := m.reloadFromKV() @@ -159,9 +187,13 @@ func (m *MetaReplica) reloadFromKV() error { return nil } +func (m *MetaReplica) setKvClient(kv kv.MetaKv) { + m.client = kv +} + func (m *MetaReplica) showCollections() []*querypb.CollectionInfo { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() collections := make([]*querypb.CollectionInfo, 0) for _, info := range m.collectionInfos { @@ -171,8 +203,8 @@ func (m *MetaReplica) showCollections() []*querypb.CollectionInfo { } func (m *MetaReplica) showPartitions(collectionID UniqueID) ([]*querypb.PartitionStates, error) { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() //TODO::should update after load collection results := make([]*querypb.PartitionStates, 0) @@ -187,8 +219,8 @@ func (m *MetaReplica) showPartitions(collectionID UniqueID) ([]*querypb.Partitio } func (m *MetaReplica) hasCollection(collectionID UniqueID) bool { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() if _, ok := m.collectionInfos[collectionID]; ok { return true @@ -198,8 +230,8 @@ func (m *MetaReplica) hasCollection(collectionID UniqueID) bool { } func (m *MetaReplica) hasPartition(collectionID UniqueID, partitionID UniqueID) bool { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() if info, ok := m.collectionInfos[collectionID]; ok { for _, id := range info.PartitionIDs { @@ -213,8 +245,8 @@ func (m *MetaReplica) hasPartition(collectionID UniqueID, partitionID UniqueID) } func (m *MetaReplica) hasReleasePartition(collectionID UniqueID, partitionID UniqueID) bool { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() if info, ok := m.collectionInfos[collectionID]; ok { for _, id := range info.ReleasedPartitionIDs { @@ -228,10 +260,8 @@ func (m *MetaReplica) hasReleasePartition(collectionID UniqueID, partitionID Uni } func (m *MetaReplica) addCollection(collectionID UniqueID, schema *schemapb.CollectionSchema) error { - m.Lock() - defer m.Unlock() - - if _, ok := m.collectionInfos[collectionID]; !ok { + hasCollection := m.hasCollection(collectionID) + if !hasCollection { partitions := make([]UniqueID, 0) partitionStates := make([]*querypb.PartitionStates, 0) channels := make([]*querypb.DmChannelInfo, 0) @@ -242,21 +272,25 @@ func (m *MetaReplica) addCollection(collectionID UniqueID, schema *schemapb.Coll ChannelInfos: channels, Schema: schema, } - m.collectionInfos[collectionID] = newCollection err := saveGlobalCollectionInfo(collectionID, newCollection, m.client) if err != nil { log.Error("save collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) return err } + m.collectionMu.Lock() + m.collectionInfos[collectionID] = newCollection + m.collectionMu.Unlock() } return nil } func (m *MetaReplica) addPartition(collectionID UniqueID, partitionID UniqueID) error { - m.Lock() - defer m.Unlock() - if col, ok := m.collectionInfos[collectionID]; ok { + m.collectionMu.Lock() + defer m.collectionMu.Unlock() + + if info, ok := m.collectionInfos[collectionID]; ok { + col := proto.Clone(info).(*querypb.CollectionInfo) log.Debug("add a partition to MetaReplica...", zap.Int64s("partitionIDs", col.PartitionIDs)) for _, id := range col.PartitionIDs { if id == partitionID { @@ -282,61 +316,324 @@ func (m *MetaReplica) addPartition(collectionID UniqueID, partitionID UniqueID) log.Error("save collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) return err } + m.collectionInfos[collectionID] = col return nil } return errors.New("addPartition: can't find collection when add partition") } -func (m *MetaReplica) deleteSegmentInfoByID(segmentID UniqueID) error { - m.Lock() - defer m.Unlock() +func (m *MetaReplica) deleteSegmentInfoByNodeID(nodeID UniqueID) error { + m.segmentMu.Lock() + defer m.segmentMu.Unlock() - if _, ok := m.segmentInfos[segmentID]; ok { - err := removeSegmentInfo(segmentID, m.client) - if err != nil { - log.Error("remove segmentInfo error", zap.Any("error", err.Error()), zap.Int64("segmentID", segmentID)) - return err + segmentIDsToRemove := make([]UniqueID, 0) + for segmentID, info := range m.segmentInfos { + if info.NodeID == nodeID { + segmentIDsToRemove = append(segmentIDsToRemove, segmentID) } + } + + err := multiRemoveSegmentInfo(segmentIDsToRemove, m.client) + if err != nil { + log.Error("remove segmentInfo from etcd error", zap.Any("error", err.Error()), zap.Int64s("segmentIDs", segmentIDsToRemove)) + return err + } + for _, segmentID := range segmentIDsToRemove { delete(m.segmentInfos, segmentID) } return nil } -func (m *MetaReplica) deleteSegmentInfoByNodeID(nodeID UniqueID) error { - m.Lock() - defer m.Unlock() +func (m *MetaReplica) setSegmentInfos(segmentInfos map[UniqueID]*querypb.SegmentInfo) error { + m.segmentMu.Lock() + defer m.segmentMu.Unlock() - for segmentID, info := range m.segmentInfos { - if info.NodeID == nodeID { - err := removeSegmentInfo(segmentID, m.client) - if err != nil { - log.Error("remove segmentInfo error", zap.Any("error", err.Error()), zap.Int64("segmentID", segmentID)) - return err + err := multiSaveSegmentInfos(segmentInfos, m.client) + if err != nil { + log.Error("save segmentInfos error", zap.Any("segmentInfos", segmentInfos), zap.Error(err)) + return err + } + + for segmentID, info := range segmentInfos { + m.segmentInfos[segmentID] = info + } + + return nil +} + +func (m *MetaReplica) saveGlobalSealedSegInfos(saves col2SegmentInfos) (col2SealedSegmentChangeInfos, error) { + if len(saves) == 0 { + return nil, nil + } + // generate segment change info according segment info to updated + col2SegmentChangeInfos := make(col2SealedSegmentChangeInfos) + + // get segmentInfos to save + for collectionID, onlineInfos := range saves { + for _, info := range onlineInfos { + segmentID := info.SegmentID + onlineNodeID := info.NodeID + segmentChangeInfo := querypb.SealedSegmentsChangeInfo{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_SealedSegmentsChangeInfo, + }, + OnlineNodeID: onlineNodeID, + OnlineSegments: []*querypb.SegmentInfo{info}, } - delete(m.segmentInfos, segmentID) + offlineInfo, err := m.getSegmentInfoByID(segmentID) + if err == nil { + offlineNodeID := offlineInfo.NodeID + // if the offline segment state is growing, it will not impact the global sealed segments + if offlineInfo.SegmentState == querypb.SegmentState_sealed { + segmentChangeInfo.OfflineNodeID = offlineNodeID + segmentChangeInfo.OfflineSegments = []*querypb.SegmentInfo{offlineInfo} + } + } + + if _, ok := col2SegmentChangeInfos[collectionID]; !ok { + col2SegmentChangeInfos[collectionID] = []*querypb.SealedSegmentsChangeInfo{} + } + col2SegmentChangeInfos[collectionID] = append(col2SegmentChangeInfos[collectionID], &segmentChangeInfo) } } - return nil + queryChannelInfosMap := make(map[UniqueID]*querypb.QueryChannelInfo) + for collectionID, segmentChangeInfos := range col2SegmentChangeInfos { + // get msgStream to produce sealedSegmentChangeInfos to query channel + queryChannelInfo, messageIDInfos, err := m.sendSealedSegmentChangeInfos(collectionID, segmentChangeInfos) + if err != nil { + return nil, err + } + messageIDs, ok := messageIDInfos[queryChannelInfo.QueryChannelID] + if !ok || len(messageIDs) == 0 { + return col2SegmentChangeInfos, errors.New("updateGlobalSealedSegmentInfos: send sealed segment change info failed") + } + seekMessageID := messageIDs[len(messageIDs)-1] + queryChannelInfo.SeekPosition.MsgID = seekMessageID.Serialize() + + // update segmentInfo, queryChannelInfo meta to cache and etcd + seg2Info := make(map[UniqueID]*querypb.SegmentInfo) + for _, segmentInfo := range queryChannelInfo.GlobalSealedSegments { + segmentID := segmentInfo.SegmentID + seg2Info[segmentID] = segmentInfo + } + if infos, ok := saves[collectionID]; ok { + for _, segmentInfo := range infos { + segmentID := segmentInfo.SegmentID + seg2Info[segmentID] = segmentInfo + } + } + + globalSealedSegmentInfos := make([]*querypb.SegmentInfo, 0) + for _, info := range seg2Info { + globalSealedSegmentInfos = append(globalSealedSegmentInfos, info) + } + queryChannelInfo.GlobalSealedSegments = globalSealedSegmentInfos + queryChannelInfosMap[collectionID] = queryChannelInfo + } + + saveKvs := make(map[string]string) + for _, infos := range saves { + for _, info := range infos { + segmentInfoBytes, err := proto.Marshal(info) + if err != nil { + return col2SegmentChangeInfos, err + } + segmentKey := fmt.Sprintf("%s/%d", segmentMetaPrefix, info.SegmentID) + saveKvs[segmentKey] = string(segmentInfoBytes) + } + } + + for collectionID, queryChannelInfo := range queryChannelInfosMap { + channelInfoBytes, err := proto.Marshal(queryChannelInfo) + if err != nil { + return col2SegmentChangeInfos, err + } + channelKey := fmt.Sprintf("%s/%d", queryChannelMetaPrefix, collectionID) + saveKvs[channelKey] = string(channelInfoBytes) + } + + // save segmentChangeInfo into etcd, query node will deal the changeInfo if the msgID key exist in etcd + // avoid the produce process success but save meta to etcd failed + // then the msgID key will not exist, and changeIndo will be ignored by query node + for _, changeInfos := range col2SegmentChangeInfos { + for _, info := range changeInfos { + changeInfoBytes, err := proto.Marshal(info) + if err != nil { + return col2SegmentChangeInfos, err + } + // TODO:: segmentChangeInfo clear in etcd with coord gc and queryNode watch the changeInfo meta to deal changeInfoMsg + changeInfoKey := fmt.Sprintf("%s/%d", sealedSegmentChangeInfoPrefix, info.Base.MsgID) + saveKvs[changeInfoKey] = string(changeInfoBytes) + } + } + + err := m.client.MultiSave(saveKvs) + if err != nil { + log.Error("updateGlobalSealedSegmentInfos: save info to etcd error", zap.Error(err)) + return col2SegmentChangeInfos, err + } + m.segmentMu.Lock() + for _, segmentInfos := range saves { + for _, info := range segmentInfos { + segmentID := info.SegmentID + m.segmentInfos[segmentID] = info + } + } + m.segmentMu.Unlock() + + m.channelMu.Lock() + for collectionID, channelInfo := range queryChannelInfosMap { + m.queryChannelInfos[collectionID] = channelInfo + } + m.channelMu.Unlock() + + return col2SegmentChangeInfos, nil } -func (m *MetaReplica) setSegmentInfo(segmentID UniqueID, info *querypb.SegmentInfo) error { - m.Lock() - defer m.Unlock() - - err := saveSegmentInfo(segmentID, info, m.client) - if err != nil { - log.Error("save segmentInfo error", zap.Any("error", err.Error()), zap.Int64("segmentID", segmentID)) - return err +func (m *MetaReplica) removeGlobalSealedSegInfos(collectionID UniqueID, partitionIDs []UniqueID) (col2SealedSegmentChangeInfos, error) { + segmentChangeInfos := make([]*querypb.SealedSegmentsChangeInfo, 0) + removes := m.showSegmentInfos(collectionID, partitionIDs) + if len(removes) == 0 { + return nil, nil } - m.segmentInfos[segmentID] = info - return nil + // get segmentInfos to remove + for _, info := range removes { + offlineNodeID := info.NodeID + changeInfo := querypb.SealedSegmentsChangeInfo{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_SealedSegmentsChangeInfo, + }, + OfflineNodeID: offlineNodeID, + OfflineSegments: []*querypb.SegmentInfo{info}, + } + + segmentChangeInfos = append(segmentChangeInfos, &changeInfo) + } + + // get msgStream to produce sealedSegmentChangeInfos to query channel + queryChannelInfo, messageIDInfos, err := m.sendSealedSegmentChangeInfos(collectionID, segmentChangeInfos) + if err != nil { + return nil, err + } + messageIDs, ok := messageIDInfos[queryChannelInfo.QueryChannelID] + if !ok || len(messageIDs) == 0 { + return col2SealedSegmentChangeInfos{collectionID: segmentChangeInfos}, errors.New("updateGlobalSealedSegmentInfos: send sealed segment change info failed") + } + seekMessageID := messageIDs[len(messageIDs)-1] + queryChannelInfo.SeekPosition.MsgID = seekMessageID.Serialize() + + // update segmentInfo, queryChannelInfo meta to cache and etcd + seg2Info := make(map[UniqueID]*querypb.SegmentInfo) + for _, segmentInfo := range queryChannelInfo.GlobalSealedSegments { + segmentID := segmentInfo.SegmentID + seg2Info[segmentID] = segmentInfo + } + + for _, segmentInfo := range removes { + segmentID := segmentInfo.SegmentID + delete(seg2Info, segmentID) + } + + globalSealedSegmentInfos := make([]*querypb.SegmentInfo, 0) + for _, info := range seg2Info { + globalSealedSegmentInfos = append(globalSealedSegmentInfos, info) + } + queryChannelInfo.GlobalSealedSegments = globalSealedSegmentInfos + + // save meta to etcd + saveKvs := make(map[string]string) + channelInfoBytes, err := proto.Marshal(queryChannelInfo) + if err != nil { + return col2SealedSegmentChangeInfos{collectionID: segmentChangeInfos}, err + } + channelKey := fmt.Sprintf("%s/%d", queryChannelMetaPrefix, collectionID) + saveKvs[channelKey] = string(channelInfoBytes) + + // save segmentChangeInfo into etcd, query node will deal the changeInfo if the msgID key exist in etcd + // avoid the produce process success but save meta to etcd failed + // then the msgID key will not exist, and changeIndo will be ignored by query node + for _, info := range segmentChangeInfos { + changeInfoBytes, err := proto.Marshal(info) + if err != nil { + return col2SealedSegmentChangeInfos{collectionID: segmentChangeInfos}, err + } + // TODO:: segmentChangeInfo clear in etcd with coord gc and queryNode watch the changeInfo meta to deal changeInfoMsg + changeInfoKey := fmt.Sprintf("%s/%d", sealedSegmentChangeInfoPrefix, info.Base.MsgID) + saveKvs[changeInfoKey] = string(changeInfoBytes) + } + + removeKeys := make([]string, 0) + for _, info := range removes { + segmentKey := fmt.Sprintf("%s/%d", segmentMetaPrefix, info.SegmentID) + removeKeys = append(removeKeys, segmentKey) + } + + err = m.client.MultiSaveAndRemove(saveKvs, removeKeys) + if err != nil { + log.Error("updateGlobalSealedSegmentInfos: save info to etcd error", zap.Error(err)) + return col2SealedSegmentChangeInfos{collectionID: segmentChangeInfos}, err + } + m.segmentMu.Lock() + for _, info := range removes { + delete(m.segmentInfos, info.SegmentID) + } + m.segmentMu.Unlock() + + m.channelMu.Lock() + m.queryChannelInfos[collectionID] = queryChannelInfo + m.channelMu.Unlock() + + return col2SealedSegmentChangeInfos{collectionID: segmentChangeInfos}, nil +} + +func (m *MetaReplica) sendSealedSegmentChangeInfos(collectionID UniqueID, changeInfos []*querypb.SealedSegmentsChangeInfo) (*querypb.QueryChannelInfo, map[string][]mqclient.MessageID, error) { + // get msgStream to produce sealedSegmentChangeInfos to query channel + queryChannelInfo, err := m.getQueryChannelInfoByID(collectionID) + if err != nil { + log.Error("updateGlobalSealedSegmentInfos: get query channel info failed", zap.Int64("collectionID", collectionID), zap.Error(err)) + return nil, nil, err + } + + queryStream, err := m.getQueryStreamByID(collectionID) + if err != nil { + log.Error("updateGlobalSealedSegmentInfos: get query stream failed", zap.Int64("collectionID", collectionID), zap.Error(err)) + return nil, nil, err + } + + var msgPack = &msgstream.MsgPack{ + Msgs: []msgstream.TsMsg{}, + } + for _, changeInfo := range changeInfos { + id, err := m.idAllocator() + if err != nil { + log.Error("allocator trigger taskID failed", zap.Error(err)) + return nil, nil, err + } + changeInfo.Base.MsgID = id + segmentChangeMsg := &msgstream.SealedSegmentsChangeInfoMsg{ + BaseMsg: msgstream.BaseMsg{ + HashValues: []uint32{0}, + }, + SealedSegmentsChangeInfo: *changeInfo, + } + msgPack.Msgs = append(msgPack.Msgs, segmentChangeMsg) + } + + messageIDInfos, err := queryStream.ProduceMark(msgPack) + if err != nil { + log.Error("updateGlobalSealedSegmentInfos: send sealed segment change info failed", zap.Int64("collectionID", collectionID), zap.Error(err)) + return nil, nil, err + } + log.Debug("updateGlobalSealedSegmentInfos: send sealed segment change info to queryChannel", zap.Any("msgPack", msgPack)) + + return queryChannelInfo, messageIDInfos, nil } func (m *MetaReplica) showSegmentInfos(collectionID UniqueID, partitionIDs []UniqueID) []*querypb.SegmentInfo { - m.RLock() - defer m.RUnlock() + m.segmentMu.RLock() + defer m.segmentMu.RUnlock() results := make([]*querypb.SegmentInfo, 0) segmentInfos := make([]*querypb.SegmentInfo, 0) @@ -358,20 +655,9 @@ func (m *MetaReplica) showSegmentInfos(collectionID UniqueID, partitionIDs []Uni return results } -func (m *MetaReplica) hasSegmentInfo(segmentID UniqueID) bool { - m.RLock() - defer m.RUnlock() - - if _, ok := m.segmentInfos[segmentID]; ok { - return true - } - - return false -} - func (m *MetaReplica) getSegmentInfoByID(segmentID UniqueID) (*querypb.SegmentInfo, error) { - m.Lock() - defer m.Unlock() + m.segmentMu.RLock() + defer m.segmentMu.RUnlock() if info, ok := m.segmentInfos[segmentID]; ok { return proto.Clone(info).(*querypb.SegmentInfo), nil @@ -381,8 +667,8 @@ func (m *MetaReplica) getSegmentInfoByID(segmentID UniqueID) (*querypb.SegmentIn } func (m *MetaReplica) getCollectionInfoByID(collectionID UniqueID) (*querypb.CollectionInfo, error) { - m.Lock() - defer m.Unlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() if info, ok := m.collectionInfos[collectionID]; ok { return proto.Clone(info).(*querypb.CollectionInfo), nil @@ -391,20 +677,9 @@ func (m *MetaReplica) getCollectionInfoByID(collectionID UniqueID) (*querypb.Col return nil, errors.New("getCollectionInfoByID: can't find collectionID in collectionInfo") } -func (m *MetaReplica) getQueryChannelInfoByID(collectionID UniqueID) (*querypb.QueryChannelInfo, error) { - m.Lock() - defer m.Unlock() - - if info, ok := m.queryChannelInfos[collectionID]; ok { - return proto.Clone(info).(*querypb.QueryChannelInfo), nil - } - - return nil, errors.New("getQueryChannelInfoByID: can't find collectionID in queryChannelInfo") -} - func (m *MetaReplica) getPartitionStatesByID(collectionID UniqueID, partitionID UniqueID) (*querypb.PartitionStates, error) { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() if info, ok := m.collectionInfos[collectionID]; ok { for offset, id := range info.PartitionIDs { @@ -419,34 +694,22 @@ func (m *MetaReplica) getPartitionStatesByID(collectionID UniqueID, partitionID } func (m *MetaReplica) releaseCollection(collectionID UniqueID) error { - m.Lock() - defer m.Unlock() - - delete(m.collectionInfos, collectionID) - var err error - for id, info := range m.segmentInfos { - if info.CollectionID == collectionID { - err = removeSegmentInfo(id, m.client) - if err != nil { - log.Warn("remove segmentInfo error", zap.Any("error", err.Error()), zap.Int64("segmentID", id)) - } - delete(m.segmentInfos, id) - } - } - - err = removeGlobalCollectionInfo(collectionID, m.client) + err := removeGlobalCollectionInfo(collectionID, m.client) if err != nil { - log.Warn("remove collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) + log.Warn("remove collectionInfo from etcd failed", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) + return err } - return err + m.collectionMu.Lock() + delete(m.collectionInfos, collectionID) + m.collectionMu.Unlock() + + return nil } func (m *MetaReplica) releasePartition(collectionID UniqueID, partitionID UniqueID) error { - m.Lock() - defer m.Unlock() - - if info, ok := m.collectionInfos[collectionID]; ok { + info, err := m.getCollectionInfoByID(collectionID) + if err == nil { newPartitionIDs := make([]UniqueID, 0) newPartitionStates := make([]*querypb.PartitionStates, 0) for offset, id := range info.PartitionIDs { @@ -474,29 +737,25 @@ func (m *MetaReplica) releasePartition(collectionID UniqueID, partitionID Unique // So if releasing partition, inMemoryPercentage should be set to 0. info.InMemoryPercentage = 0 - err := saveGlobalCollectionInfo(collectionID, info, m.client) + err = saveGlobalCollectionInfo(collectionID, info, m.client) if err != nil { - log.Error("save collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) + log.Error("releasePartition: save collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID), zap.Int64("partitionID", partitionID)) return err } - } - for id, info := range m.segmentInfos { - if info.PartitionID == partitionID { - err := removeSegmentInfo(id, m.client) - if err != nil { - log.Error("delete segmentInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID), zap.Int64("segmentID", id)) - return err - } - delete(m.segmentInfos, id) - } + + m.collectionMu.Lock() + m.collectionInfos[collectionID] = info + m.collectionMu.Unlock() + + return nil } - return nil + return err } func (m *MetaReplica) getDmChannelsByNodeID(collectionID UniqueID, nodeID int64) ([]string, error) { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() if info, ok := m.collectionInfos[collectionID]; ok { channels := make([]string, 0) @@ -512,11 +771,9 @@ func (m *MetaReplica) getDmChannelsByNodeID(collectionID UniqueID, nodeID int64) } func (m *MetaReplica) addDmChannel(collectionID UniqueID, nodeID int64, channels []string) error { - m.Lock() - defer m.Unlock() - //before add channel, should ensure toAddedChannels not in MetaReplica - if info, ok := m.collectionInfos[collectionID]; ok { + info, err := m.getCollectionInfoByID(collectionID) + if err == nil { findNodeID := false for _, channelInfo := range info.ChannelInfos { if channelInfo.NodeIDLoaded == nodeID { @@ -532,11 +789,14 @@ func (m *MetaReplica) addDmChannel(collectionID UniqueID, nodeID int64, channels info.ChannelInfos = append(info.ChannelInfos, newChannelInfo) } - err := saveGlobalCollectionInfo(collectionID, info, m.client) + err = saveGlobalCollectionInfo(collectionID, info, m.client) if err != nil { log.Error("save collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) return err } + m.collectionMu.Lock() + m.collectionInfos[collectionID] = info + m.collectionMu.Unlock() return nil } @@ -544,10 +804,8 @@ func (m *MetaReplica) addDmChannel(collectionID UniqueID, nodeID int64, channels } func (m *MetaReplica) removeDmChannel(collectionID UniqueID, nodeID int64, channels []string) error { - m.Lock() - defer m.Unlock() - - if info, ok := m.collectionInfos[collectionID]; ok { + info, err := m.getCollectionInfoByID(collectionID) + if err == nil { for _, channelInfo := range info.ChannelInfos { if channelInfo.NodeIDLoaded == nodeID { newChannelIDs := make([]string, 0) @@ -572,54 +830,96 @@ func (m *MetaReplica) removeDmChannel(collectionID UniqueID, nodeID int64, chann return err } + m.collectionMu.Lock() + m.collectionInfos[collectionID] = info + m.collectionMu.Unlock() + return nil } return errors.New("addDmChannels: can't find collection in collectionInfos") } -func (m *MetaReplica) getQueryChannel(collectionID UniqueID) (string, string, error) { - m.Lock() - defer m.Unlock() - - //TODO::to remove - collectionID = 0 - if info, ok := m.queryChannelInfos[collectionID]; ok { - return info.QueryChannelID, info.QueryResultChannelID, nil - } - +func createQueryChannel(collectionID UniqueID) *querypb.QueryChannelInfo { searchPrefix := Params.SearchChannelPrefix searchResultPrefix := Params.SearchResultChannelPrefix allocatedQueryChannel := searchPrefix + "-" + strconv.FormatInt(collectionID, 10) allocatedQueryResultChannel := searchResultPrefix + "-" + strconv.FormatInt(collectionID, 10) log.Debug("query coordinator create query channel", zap.String("queryChannelName", allocatedQueryChannel), zap.String("queryResultChannelName", allocatedQueryResultChannel)) - queryChannelInfo := &querypb.QueryChannelInfo{ + seekPosition := &internalpb.MsgPosition{ + ChannelName: allocatedQueryChannel, + } + info := &querypb.QueryChannelInfo{ CollectionID: collectionID, QueryChannelID: allocatedQueryChannel, QueryResultChannelID: allocatedQueryResultChannel, + GlobalSealedSegments: []*querypb.SegmentInfo{}, + SeekPosition: seekPosition, } - err := saveQueryChannelInfo(collectionID, queryChannelInfo, m.client) + + return info +} + +func (m *MetaReplica) getQueryChannelInfoByID(collectionID UniqueID) (*querypb.QueryChannelInfo, error) { + m.channelMu.Lock() + defer m.channelMu.Unlock() + + //TODO::to remove + collectionID = 0 + if info, ok := m.queryChannelInfos[collectionID]; ok { + return proto.Clone(info).(*querypb.QueryChannelInfo), nil + } + + info := createQueryChannel(collectionID) + err := saveQueryChannelInfo(collectionID, info, m.client) if err != nil { log.Error("getQueryChannel: save channel to etcd error", zap.Error(err)) - return "", "", err + return nil, err } - m.queryChannelInfos[collectionID] = queryChannelInfo - //TODO::return channel according collectionID - return allocatedQueryChannel, allocatedQueryResultChannel, nil + m.queryChannelInfos[collectionID] = info + return proto.Clone(info).(*querypb.QueryChannelInfo), nil +} + +func (m *MetaReplica) getQueryStreamByID(collectionID UniqueID) (msgstream.MsgStream, error) { + m.streamMu.Lock() + defer m.streamMu.Unlock() + + info, err := m.getQueryChannelInfoByID(collectionID) + if err != nil { + return nil, err + } + + stream, ok := m.queryStreams[collectionID] + if !ok { + stream, err = m.msFactory.NewMsgStream(m.ctx) + if err != nil { + log.Error("updateGlobalSealedSegmentInfos: create msgStream failed", zap.Error(err)) + return nil, err + } + + queryChannel := info.QueryChannelID + stream.AsProducer([]string{queryChannel}) + m.queryStreams[collectionID] = stream + log.Debug("getQueryStreamByID: create query msgStream for collection", zap.Int64("collectionID", collectionID)) + } + + return stream, nil } func (m *MetaReplica) setLoadType(collectionID UniqueID, loadType querypb.LoadType) error { - m.Lock() - defer m.Unlock() - - if info, ok := m.collectionInfos[collectionID]; ok { + info, err := m.getCollectionInfoByID(collectionID) + if err == nil { info.LoadType = loadType err := saveGlobalCollectionInfo(collectionID, info, m.client) if err != nil { log.Error("save collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) return err } + m.collectionMu.Lock() + m.collectionInfos[collectionID] = info + m.collectionMu.Unlock() + return nil } @@ -627,8 +927,8 @@ func (m *MetaReplica) setLoadType(collectionID UniqueID, loadType querypb.LoadTy } func (m *MetaReplica) getLoadType(collectionID UniqueID) (querypb.LoadType, error) { - m.RLock() - defer m.RUnlock() + m.collectionMu.RLock() + defer m.collectionMu.RUnlock() if info, ok := m.collectionInfos[collectionID]; ok { return info.LoadType, nil @@ -638,11 +938,8 @@ func (m *MetaReplica) getLoadType(collectionID UniqueID) (querypb.LoadType, erro } func (m *MetaReplica) setLoadPercentage(collectionID UniqueID, partitionID UniqueID, percentage int64, loadType querypb.LoadType) error { - m.Lock() - defer m.Unlock() - - info, ok := m.collectionInfos[collectionID] - if !ok { + info, err := m.getCollectionInfoByID(collectionID) + if err != nil { return errors.New("setLoadPercentage: can't find collection in collectionInfos") } @@ -662,8 +959,10 @@ func (m *MetaReplica) setLoadPercentage(collectionID UniqueID, partitionID Uniqu return err } } else { + findPartition := false for _, partitionState := range info.PartitionStates { if partitionState.PartitionID == partitionID { + findPartition = true if percentage >= 100 { partitionState.State = querypb.PartitionState_InMemory } else { @@ -675,12 +974,17 @@ func (m *MetaReplica) setLoadPercentage(collectionID UniqueID, partitionID Uniqu log.Error("save collectionInfo error", zap.Any("error", err.Error()), zap.Int64("collectionID", collectionID)) return err } - return nil } } - return errors.New("setLoadPercentage: can't find partitionID in collectionInfos") + if !findPartition { + return errors.New("setLoadPercentage: can't find partitionID in collectionInfos") + } } + m.collectionMu.Lock() + m.collectionInfos[collectionID] = info + m.collectionMu.Unlock() + return nil } @@ -715,19 +1019,28 @@ func removeGlobalCollectionInfo(collectionID UniqueID, kv kv.MetaKv) error { return kv.Remove(key) } -func saveSegmentInfo(segmentID UniqueID, info *querypb.SegmentInfo, kv kv.MetaKv) error { - infoBytes, err := proto.Marshal(info) - if err != nil { - return err +func multiSaveSegmentInfos(segmentInfos map[UniqueID]*querypb.SegmentInfo, kv kv.MetaKv) error { + kvs := make(map[string]string) + for segmentID, info := range segmentInfos { + infoBytes, err := proto.Marshal(info) + if err != nil { + return err + } + key := fmt.Sprintf("%s/%d", segmentMetaPrefix, segmentID) + kvs[key] = string(infoBytes) } - key := fmt.Sprintf("%s/%d", segmentMetaPrefix, segmentID) - return kv.Save(key, string(infoBytes)) + return kv.MultiSave(kvs) } -func removeSegmentInfo(segmentID UniqueID, kv kv.MetaKv) error { - key := fmt.Sprintf("%s/%d", segmentMetaPrefix, segmentID) - return kv.Remove(key) +func multiRemoveSegmentInfo(segmentIDs []UniqueID, kv kv.MetaKv) error { + keys := make([]string, 0) + for _, segmentID := range segmentIDs { + key := fmt.Sprintf("%s/%d", segmentMetaPrefix, segmentID) + keys = append(keys, key) + } + + return kv.MultiRemove(keys) } func saveQueryChannelInfo(collectionID UniqueID, info *querypb.QueryChannelInfo, kv kv.MetaKv) error { diff --git a/internal/querycoord/meta_test.go b/internal/querycoord/meta_test.go index 7bde224ad7..271454bbe4 100644 --- a/internal/querycoord/meta_test.go +++ b/internal/querycoord/meta_test.go @@ -12,6 +12,7 @@ package querycoord import ( + "context" "errors" "fmt" "testing" @@ -37,6 +38,10 @@ func (tk *testKv) Save(key, value string) error { return tk.returnFn() } +func (tk *testKv) MultiSave(saves map[string]string) error { + return tk.returnFn() +} + func (tk *testKv) Remove(key string) error { return tk.returnFn() } @@ -49,7 +54,7 @@ func TestReplica_Release(t *testing.T) { refreshParams() etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath) assert.Nil(t, err) - meta, err := newMeta(etcdKV) + meta, err := newMeta(context.Background(), etcdKV, nil, nil) assert.Nil(t, err) err = meta.addCollection(1, nil) require.NoError(t, err) @@ -110,11 +115,6 @@ func TestMetaFunc(t *testing.T) { assert.Equal(t, false, hasReleasePartition) }) - t.Run("Test HasSegmentInfoFalse", func(t *testing.T) { - hasSegmentInfo := meta.hasSegmentInfo(defaultSegmentID) - assert.Equal(t, false, hasSegmentInfo) - }) - t.Run("Test GetSegmentInfoByIDFail", func(t *testing.T) { res, err := meta.getSegmentInfoByID(defaultSegmentID) assert.NotNil(t, err) @@ -129,8 +129,8 @@ func TestMetaFunc(t *testing.T) { t.Run("Test GetQueryChannelInfoByIDFail", func(t *testing.T) { res, err := meta.getQueryChannelInfoByID(defaultCollectionID) - assert.NotNil(t, err) - assert.Nil(t, res) + assert.Nil(t, err) + assert.NotNil(t, res) }) t.Run("Test GetPartitionStatesByIDFail", func(t *testing.T) { @@ -222,7 +222,9 @@ func TestMetaFunc(t *testing.T) { CollectionID: defaultCollectionID, NodeID: nodeID, } - err := meta.setSegmentInfo(defaultSegmentID, info) + segmentInfos := make(map[UniqueID]*querypb.SegmentInfo) + segmentInfos[defaultSegmentID] = info + err := meta.setSegmentInfos(segmentInfos) assert.Nil(t, err) }) @@ -233,9 +235,9 @@ func TestMetaFunc(t *testing.T) { }) t.Run("Test getQueryChannel", func(t *testing.T) { - reqChannel, resChannel, err := meta.getQueryChannel(defaultCollectionID) - assert.NotNil(t, reqChannel) - assert.NotNil(t, resChannel) + info, err := meta.getQueryChannelInfoByID(defaultCollectionID) + assert.NotNil(t, info.QueryChannelID) + assert.NotNil(t, info.QueryResultChannelID) assert.Nil(t, err) }) diff --git a/internal/querycoord/param_table.go b/internal/querycoord/param_table.go index a92703480b..a96975bf4c 100644 --- a/internal/querycoord/param_table.go +++ b/internal/querycoord/param_table.go @@ -61,6 +61,9 @@ type ParamTable struct { CreatedTime time.Time UpdatedTime time.Time + + // --- Pulsar --- + PulsarAddress string } // Params are variables of the ParamTable type @@ -108,6 +111,9 @@ func (p *ParamTable) Init() { p.initMinioSecretAccessKey() p.initMinioUseSSLStr() p.initMinioBucketName() + + //--- Pulsar ---- + p.initPulsarAddress() } func (p *ParamTable) initQueryCoordAddress() { @@ -242,3 +248,11 @@ func (p *ParamTable) initMinioBucketName() { func (p *ParamTable) initRoleName() { p.RoleName = "querycoord" } + +func (p *ParamTable) initPulsarAddress() { + addr, err := p.Load("_PulsarAddress") + if err != nil { + panic(err) + } + p.PulsarAddress = addr +} diff --git a/internal/querycoord/query_coord.go b/internal/querycoord/query_coord.go index 7abe5d6c46..f638773caf 100644 --- a/internal/querycoord/query_coord.go +++ b/internal/querycoord/query_coord.go @@ -14,17 +14,16 @@ package querycoord import ( "context" "errors" + "math/rand" - "path/filepath" "strconv" "sync" "sync/atomic" "time" - "github.com/golang/protobuf/proto" - "go.etcd.io/etcd/api/v3/mvccpb" "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/allocator" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/msgstream" @@ -35,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/sessionutil" + "github.com/milvus-io/milvus/internal/util/tsoutil" "github.com/milvus-io/milvus/internal/util/typeutil" ) @@ -60,6 +60,7 @@ type QueryCoord struct { cluster Cluster newNodeFn newQueryNodeFn scheduler *TaskScheduler + idAllocator func() (UniqueID, error) metricsCacheManager *metricsinfo.MetricsCacheManager @@ -106,19 +107,39 @@ func (qc *QueryCoord) Init() error { return } log.Debug("query coordinator try to connect etcd success") - qc.meta, initError = newMeta(qc.kvClient) + + // init id allocator + var idAllocatorKV *etcdkv.EtcdKV + idAllocatorKV, initError = tsoutil.NewTSOKVBase(Params.EtcdEndpoints, Params.KvRootPath, "queryCoordTaskID") + if initError != nil { + return + } + idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV) + initError = idAllocator.Initialize() + if initError != nil { + log.Debug("query coordinator idAllocator initialize failed", zap.Error(initError)) + return + } + qc.idAllocator = func() (UniqueID, error) { + return idAllocator.AllocOne() + } + + // init meta + qc.meta, initError = newMeta(qc.loopCtx, qc.kvClient, qc.msFactory, qc.idAllocator) if initError != nil { log.Error("query coordinator init meta failed", zap.Error(initError)) return } + // init cluster qc.cluster, initError = newQueryNodeCluster(qc.loopCtx, qc.meta, qc.kvClient, qc.newNodeFn, qc.session) if initError != nil { log.Error("query coordinator init cluster failed", zap.Error(initError)) return } - qc.scheduler, initError = NewTaskScheduler(qc.loopCtx, qc.meta, qc.cluster, qc.kvClient, qc.rootCoordClient, qc.dataCoordClient) + // init task scheduler + qc.scheduler, initError = NewTaskScheduler(qc.loopCtx, qc.meta, qc.cluster, qc.kvClient, qc.rootCoordClient, qc.dataCoordClient, qc.idAllocator) if initError != nil { log.Error("query coordinator init task scheduler failed", zap.Error(initError)) return @@ -132,6 +153,14 @@ func (qc *QueryCoord) Init() error { // Start function starts the goroutines to watch the meta and node updates func (qc *QueryCoord) Start() error { + m := map[string]interface{}{ + "PulsarAddress": Params.PulsarAddress, + "ReceiveBufSize": 1024, + "PulsarBufSize": 1024} + err := qc.msFactory.SetParams(m) + if err != nil { + return err + } qc.scheduler.Start() log.Debug("start scheduler ...") @@ -143,9 +172,6 @@ func (qc *QueryCoord) Start() error { qc.loopWg.Add(1) go qc.watchNodeLoop() - qc.loopWg.Add(1) - go qc.watchMetaLoop() - go qc.session.LivenessCheck(qc.loopCtx, func() { qc.Stop() }) @@ -305,41 +331,3 @@ func (qc *QueryCoord) watchNodeLoop() { } } } - -func (qc *QueryCoord) watchMetaLoop() { - ctx, cancel := context.WithCancel(qc.loopCtx) - - defer cancel() - defer qc.loopWg.Done() - log.Debug("query coordinator start watch MetaReplica loop") - - watchChan := qc.kvClient.WatchWithPrefix("queryNode-segmentMeta") - - for { - select { - case <-ctx.Done(): - return - case resp := <-watchChan: - log.Debug("segment MetaReplica updated.") - for _, event := range resp.Events { - segmentID, err := strconv.ParseInt(filepath.Base(string(event.Kv.Key)), 10, 64) - if err != nil { - log.Error("watch MetaReplica loop error when get segmentID", zap.Any("error", err.Error())) - } - segmentInfo := &querypb.SegmentInfo{} - err = proto.Unmarshal(event.Kv.Value, segmentInfo) - if err != nil { - log.Error("watch MetaReplica loop error when unmarshal", zap.Any("error", err.Error())) - } - switch event.Type { - case mvccpb.PUT: - //TODO:: - qc.meta.setSegmentInfo(segmentID, segmentInfo) - case mvccpb.DELETE: - //TODO:: - } - } - } - } - -} diff --git a/internal/querycoord/querynode_test.go b/internal/querycoord/querynode_test.go index 40dab8584a..f26bb37338 100644 --- a/internal/querycoord/querynode_test.go +++ b/internal/querycoord/querynode_test.go @@ -230,3 +230,49 @@ func TestReleaseCollectionOnOfflineNode(t *testing.T) { cancel() } + +func TestSealedSegmentChangeAfterQueryNodeStop(t *testing.T) { + refreshParams() + baseCtx := context.Background() + + queryCoord, err := startQueryCoord(baseCtx) + assert.Nil(t, err) + + queryNode1, err := startQueryNodeServer(baseCtx) + assert.Nil(t, err) + waitQueryNodeOnline(queryCoord.cluster, queryNode1.queryNodeID) + + queryCoord.LoadCollection(baseCtx, &querypb.LoadCollectionRequest{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_LoadCollection, + }, + CollectionID: defaultCollectionID, + Schema: genCollectionSchema(defaultCollectionID, false), + }) + + queryNode2, err := startQueryNodeServer(baseCtx) + assert.Nil(t, err) + waitQueryNodeOnline(queryCoord.cluster, queryNode2.queryNodeID) + + queryNode1.stop() + err = removeNodeSession(queryNode1.queryNodeID) + assert.Nil(t, err) + + for { + segmentInfos := queryCoord.meta.showSegmentInfos(defaultCollectionID, nil) + recoverDone := true + for _, info := range segmentInfos { + if info.NodeID != queryNode2.queryNodeID { + recoverDone = false + break + } + } + if recoverDone { + break + } + } + + queryCoord.Stop() + err = removeAllSession() + assert.Nil(t, err) +} diff --git a/internal/querycoord/task.go b/internal/querycoord/task.go index 199466ac2a..ff552abe41 100644 --- a/internal/querycoord/task.go +++ b/internal/querycoord/task.go @@ -942,7 +942,7 @@ func (lst *loadSegmentTask) marshal() ([]byte, error) { } func (lst *loadSegmentTask) isValid() bool { - online, err := lst.cluster.isOnline(lst.NodeID) + online, err := lst.cluster.isOnline(lst.DstNodeID) if err != nil { return false } @@ -975,7 +975,7 @@ func (lst *loadSegmentTask) preExecute(context.Context) error { lst.setResultInfo(nil) log.Debug("start do loadSegmentTask", zap.Int64s("segmentIDs", segmentIDs), - zap.Int64("loaded nodeID", lst.NodeID), + zap.Int64("loaded nodeID", lst.DstNodeID), zap.Int64("taskID", lst.getTaskID())) return nil } @@ -985,7 +985,7 @@ func (lst *loadSegmentTask) execute(ctx context.Context) error { lst.retryCount-- }() - err := lst.cluster.loadSegments(ctx, lst.NodeID, lst.LoadSegmentsRequest) + err := lst.cluster.loadSegments(ctx, lst.DstNodeID, lst.LoadSegmentsRequest) if err != nil { log.Warn("loadSegmentTask: loadSegment occur error", zap.Int64("taskID", lst.getTaskID())) lst.setResultInfo(err) @@ -1010,7 +1010,7 @@ func (lst *loadSegmentTask) reschedule(ctx context.Context) ([]task, error) { for _, info := range lst.Infos { segmentIDs = append(segmentIDs, info.SegmentID) } - lst.excludeNodeIDs = append(lst.excludeNodeIDs, lst.NodeID) + lst.excludeNodeIDs = append(lst.excludeNodeIDs, lst.DstNodeID) segment2Nodes, err := shuffleSegmentsToQueryNode(segmentIDs, lst.cluster, false, lst.excludeNodeIDs) if err != nil { log.Error("loadSegment reschedule failed", zap.Int64s("excludeNodes", lst.excludeNodeIDs), zap.Error(err)) @@ -1032,7 +1032,7 @@ func (lst *loadSegmentTask) reschedule(ctx context.Context) ([]task, error) { baseTask: loadSegmentBaseTask, LoadSegmentsRequest: &querypb.LoadSegmentsRequest{ Base: lst.Base, - NodeID: nodeID, + DstNodeID: nodeID, Infos: infos, Schema: lst.Schema, LoadCondition: lst.LoadCondition, @@ -1046,7 +1046,7 @@ func (lst *loadSegmentTask) reschedule(ctx context.Context) ([]task, error) { hasWatchQueryChannel := lst.cluster.hasWatchedQueryChannel(lst.ctx, nodeID, collectionID) if !hasWatchQueryChannel { - queryChannel, queryResultChannel, err := lst.meta.getQueryChannel(collectionID) + queryChannelInfo, err := lst.meta.getQueryChannelInfoByID(collectionID) if err != nil { return nil, err } @@ -1054,11 +1054,13 @@ func (lst *loadSegmentTask) reschedule(ctx context.Context) ([]task, error) { msgBase := proto.Clone(lst.Base).(*commonpb.MsgBase) msgBase.MsgType = commonpb.MsgType_WatchQueryChannels addQueryChannelRequest := &querypb.AddQueryChannelRequest{ - Base: msgBase, - NodeID: nodeID, - CollectionID: collectionID, - RequestChannelID: queryChannel, - ResultChannelID: queryResultChannel, + Base: msgBase, + NodeID: nodeID, + CollectionID: collectionID, + RequestChannelID: queryChannelInfo.QueryChannelID, + ResultChannelID: queryChannelInfo.QueryResultChannelID, + GlobalSealedSegments: queryChannelInfo.GlobalSealedSegments, + SeekPosition: queryChannelInfo.SeekPosition, } watchQueryChannelBaseTask := newBaseTask(ctx, lst.getTriggerCondition()) watchQueryChannelBaseTask.setParentTask(lst.getParentTask()) @@ -1265,7 +1267,7 @@ func (wdt *watchDmChannelTask) reschedule(ctx context.Context) ([]task, error) { hasWatchQueryChannel := wdt.cluster.hasWatchedQueryChannel(wdt.ctx, nodeID, collectionID) if !hasWatchQueryChannel { - queryChannel, queryResultChannel, err := wdt.meta.getQueryChannel(collectionID) + queryChannelInfo, err := wdt.meta.getQueryChannelInfoByID(collectionID) if err != nil { return nil, err } @@ -1273,11 +1275,13 @@ func (wdt *watchDmChannelTask) reschedule(ctx context.Context) ([]task, error) { msgBase := proto.Clone(wdt.Base).(*commonpb.MsgBase) msgBase.MsgType = commonpb.MsgType_WatchQueryChannels addQueryChannelRequest := &querypb.AddQueryChannelRequest{ - Base: msgBase, - NodeID: nodeID, - CollectionID: collectionID, - RequestChannelID: queryChannel, - ResultChannelID: queryResultChannel, + Base: msgBase, + NodeID: nodeID, + CollectionID: collectionID, + RequestChannelID: queryChannelInfo.QueryChannelID, + ResultChannelID: queryChannelInfo.QueryResultChannelID, + GlobalSealedSegments: queryChannelInfo.GlobalSealedSegments, + SeekPosition: queryChannelInfo.SeekPosition, } watchQueryChannelBaseTask := newBaseTask(ctx, wdt.getTriggerCondition()) watchQueryChannelBaseTask.setParentTask(wdt.getParentTask()) @@ -1475,6 +1479,7 @@ func (lbt *loadBalanceTask) execute(ctx context.Context) error { Infos: []*querypb.SegmentLoadInfo{segmentLoadInfo}, Schema: schema, LoadCondition: querypb.TriggerCondition_nodeDown, + SourceNodeID: nodeID, } segmentsToLoad = append(segmentsToLoad, segmentID) @@ -1555,10 +1560,6 @@ func (lbt *loadBalanceTask) postExecute(context.Context) error { if err != nil { log.Error("loadBalanceTask: occur error when removing node info from cluster", zap.Int64("nodeID", id)) } - err = lbt.meta.deleteSegmentInfoByNodeID(id) - if err != nil { - log.Error("loadBalanceTask: occur error when removing node info from meta", zap.Int64("nodeID", id)) - } } } else { lbt.childTasks = []task{} @@ -1808,7 +1809,7 @@ func assignInternalTask(ctx context.Context, for nodeID, loadSegmentsReqs := range node2Segments { for _, req := range loadSegmentsReqs { ctx = opentracing.ContextWithSpan(context.Background(), sp) - req.NodeID = nodeID + req.DstNodeID = nodeID baseTask := newBaseTask(ctx, parentTask.getTriggerCondition()) baseTask.setParentTask(parentTask) loadSegmentTask := &loadSegmentTask{ @@ -1843,7 +1844,7 @@ func assignInternalTask(ctx context.Context, for nodeID, watched := range watchQueryChannelInfo { if !watched { ctx = opentracing.ContextWithSpan(context.Background(), sp) - queryChannel, queryResultChannel, err := meta.getQueryChannel(collectionID) + queryChannelInfo, err := meta.getQueryChannelInfoByID(collectionID) if err != nil { return err } @@ -1851,11 +1852,13 @@ func assignInternalTask(ctx context.Context, msgBase := proto.Clone(parentTask.msgBase()).(*commonpb.MsgBase) msgBase.MsgType = commonpb.MsgType_WatchQueryChannels addQueryChannelRequest := &querypb.AddQueryChannelRequest{ - Base: msgBase, - NodeID: nodeID, - CollectionID: collectionID, - RequestChannelID: queryChannel, - ResultChannelID: queryResultChannel, + Base: msgBase, + NodeID: nodeID, + CollectionID: collectionID, + RequestChannelID: queryChannelInfo.QueryChannelID, + ResultChannelID: queryChannelInfo.QueryResultChannelID, + GlobalSealedSegments: queryChannelInfo.GlobalSealedSegments, + SeekPosition: queryChannelInfo.SeekPosition, } baseTask := newBaseTask(ctx, parentTask.getTriggerCondition()) baseTask.setParentTask(parentTask) diff --git a/internal/querycoord/task_scheduler.go b/internal/querycoord/task_scheduler.go index 425add3044..1136174015 100644 --- a/internal/querycoord/task_scheduler.go +++ b/internal/querycoord/task_scheduler.go @@ -24,14 +24,13 @@ import ( "github.com/opentracing/opentracing-go" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/allocator" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/trace" - "github.com/milvus-io/milvus/internal/util/tsoutil" oplog "github.com/opentracing/opentracing-go/log" ) @@ -139,7 +138,13 @@ type TaskScheduler struct { } // NewTaskScheduler reloads tasks from kv and returns a new taskScheduler -func NewTaskScheduler(ctx context.Context, meta Meta, cluster Cluster, kv *etcdkv.EtcdKV, rootCoord types.RootCoord, dataCoord types.DataCoord) (*TaskScheduler, error) { +func NewTaskScheduler(ctx context.Context, + meta Meta, + cluster Cluster, + kv *etcdkv.EtcdKV, + rootCoord types.RootCoord, + dataCoord types.DataCoord, + idAllocator func() (UniqueID, error)) (*TaskScheduler, error) { ctx1, cancel := context.WithCancel(ctx) taskChan := make(chan task, 1024) stopTaskLoopChan := make(chan int, 1) @@ -148,6 +153,7 @@ func NewTaskScheduler(ctx context.Context, meta Meta, cluster Cluster, kv *etcdk cancel: cancel, meta: meta, cluster: cluster, + taskIDAllocator: idAllocator, activateTaskChan: taskChan, client: kv, stopActivateTaskLoopChan: stopTaskLoopChan, @@ -155,20 +161,8 @@ func NewTaskScheduler(ctx context.Context, meta Meta, cluster Cluster, kv *etcdk dataCoord: dataCoord, } s.triggerTaskQueue = NewTaskQueue() - //init id allocator - etcdKV, err := tsoutil.NewTSOKVBase(Params.EtcdEndpoints, Params.KvRootPath, "queryCoordTaskID") - if err != nil { - return nil, err - } - idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", etcdKV) - if err := idAllocator.Initialize(); err != nil { - log.Debug("query coordinator idAllocator initialize failed", zap.Error(err)) - return nil, err - } - s.taskIDAllocator = func() (UniqueID, error) { - return idAllocator.AllocOne() - } - err = s.reloadFromKV() + + err := s.reloadFromKV() if err != nil { log.Error("reload task from kv failed", zap.Error(err)) return nil, err @@ -627,8 +621,18 @@ func (scheduler *TaskScheduler) scheduleLoop() { activateTasks := make([]task, len(childTasks)) copy(activateTasks, childTasks) processInternalTaskFn(activateTasks, triggerTask) - resultStatus := triggerTask.getResultInfo() - if resultStatus.ErrorCode != commonpb.ErrorCode_Success { + if triggerTask.getResultInfo().ErrorCode == commonpb.ErrorCode_Success { + err = updateSegmentInfoFromTask(scheduler.ctx, triggerTask, scheduler.meta) + if err != nil { + triggerTask.setResultInfo(err) + } + } + resultInfo := triggerTask.getResultInfo() + if resultInfo.ErrorCode != commonpb.ErrorCode_Success { + if !alreadyNotify { + triggerTask.notify(errors.New(resultInfo.Reason)) + alreadyNotify = true + } rollBackTasks := triggerTask.rollBack(scheduler.ctx) log.Debug("scheduleLoop: start rollBack after triggerTask failed", zap.Int64("triggerTaskID", triggerTask.getTaskID()), @@ -638,7 +642,7 @@ func (scheduler *TaskScheduler) scheduleLoop() { log.Error("scheduleLoop: rollBackInternalTask error", zap.Int64("triggerTaskID", triggerTask.getTaskID()), zap.Error(err)) - triggerTask.setResultInfo(err) + } else { processInternalTaskFn(rollBackTasks, triggerTask) } @@ -816,11 +820,13 @@ func (scheduler *TaskScheduler) processActivateTaskLoop() { continue } - log.Debug("processActivateTaskLoop: pop a active task from activateChan", zap.Int64("taskID", t.getTaskID())) - go func() { - err := scheduler.processTask(t) - t.notify(err) - }() + if t.getState() != taskDone { + log.Debug("processActivateTaskLoop: pop a active task from activateChan", zap.Int64("taskID", t.getTaskID())) + go func() { + err := scheduler.processTask(t) + t.notify(err) + }() + } } } } @@ -838,3 +844,96 @@ func (scheduler *TaskScheduler) Close() { scheduler.cancel() scheduler.wg.Wait() } + +func updateSegmentInfoFromTask(ctx context.Context, triggerTask task, meta Meta) error { + segmentInfosToSave := make(map[UniqueID][]*querypb.SegmentInfo) + segmentInfosToRemove := make(map[UniqueID][]*querypb.SegmentInfo) + + var sealedSegmentChangeInfos col2SealedSegmentChangeInfos + var err error + switch triggerTask.msgType() { + case commonpb.MsgType_ReleaseCollection: + // release all segmentInfo of the collection when release collection + req := triggerTask.(*releaseCollectionTask).ReleaseCollectionRequest + collectionID := req.CollectionID + sealedSegmentChangeInfos, err = meta.removeGlobalSealedSegInfos(collectionID, nil) + case commonpb.MsgType_ReleasePartitions: + // release all segmentInfo of the partitions when release partitions + req := triggerTask.(*releasePartitionTask).ReleasePartitionsRequest + collectionID := req.CollectionID + segmentInfos := meta.showSegmentInfos(collectionID, req.PartitionIDs) + for _, info := range segmentInfos { + if info.CollectionID == collectionID { + if _, ok := segmentInfosToRemove[collectionID]; !ok { + segmentInfosToRemove[collectionID] = make([]*querypb.SegmentInfo, 0) + } + segmentInfosToRemove[collectionID] = append(segmentInfosToRemove[collectionID], info) + } + } + sealedSegmentChangeInfos, err = meta.removeGlobalSealedSegInfos(collectionID, req.PartitionIDs) + default: + // save new segmentInfo when load segment + for _, childTask := range triggerTask.getChildTask() { + if childTask.msgType() == commonpb.MsgType_LoadSegments { + req := childTask.(*loadSegmentTask).LoadSegmentsRequest + dstNodeID := req.DstNodeID + for _, loadInfo := range req.Infos { + collectionID := loadInfo.CollectionID + segmentID := loadInfo.SegmentID + segmentInfo := &querypb.SegmentInfo{ + SegmentID: segmentID, + CollectionID: loadInfo.CollectionID, + PartitionID: loadInfo.PartitionID, + NodeID: dstNodeID, + SegmentState: querypb.SegmentState_sealed, + } + if _, ok := segmentInfosToSave[collectionID]; !ok { + segmentInfosToSave[collectionID] = make([]*querypb.SegmentInfo, 0) + } + segmentInfosToSave[collectionID] = append(segmentInfosToSave[collectionID], segmentInfo) + } + } + } + sealedSegmentChangeInfos, err = meta.saveGlobalSealedSegInfos(segmentInfosToSave) + } + + if err != nil { + rollBackSegmentChangeInfoErr := retry.Do(ctx, func() error { + rollBackChangeInfos := reverseSealedSegmentChangeInfo(sealedSegmentChangeInfos) + for collectionID, infos := range rollBackChangeInfos { + _, _, sendErr := meta.sendSealedSegmentChangeInfos(collectionID, infos) + if sendErr != nil { + return sendErr + } + } + return nil + }, retry.Attempts(20)) + if rollBackSegmentChangeInfoErr != nil { + log.Error("scheduleLoop: Restore the information of global sealed segments in query node failed", zap.Error(rollBackSegmentChangeInfoErr)) + } + return err + } + + return nil +} + +func reverseSealedSegmentChangeInfo(changeInfosMap map[UniqueID][]*querypb.SealedSegmentsChangeInfo) map[UniqueID][]*querypb.SealedSegmentsChangeInfo { + result := make(map[UniqueID][]*querypb.SealedSegmentsChangeInfo) + for collectionID, changeInfos := range changeInfosMap { + result[collectionID] = []*querypb.SealedSegmentsChangeInfo{} + for _, info := range changeInfos { + segmentChangeInfo := &querypb.SealedSegmentsChangeInfo{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_SealedSegmentsChangeInfo, + }, + OnlineNodeID: info.OfflineNodeID, + OnlineSegments: info.OfflineSegments, + OfflineNodeID: info.OnlineNodeID, + OfflineSegments: info.OnlineSegments, + } + result[collectionID] = append(result[collectionID], segmentChangeInfo) + } + } + + return result +} diff --git a/internal/querycoord/task_scheduler_test.go b/internal/querycoord/task_scheduler_test.go index 75153009ff..3a00de5b37 100644 --- a/internal/querycoord/task_scheduler_test.go +++ b/internal/querycoord/task_scheduler_test.go @@ -104,7 +104,7 @@ func (tt *testTask) execute(ctx context.Context) error { Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_LoadSegments, }, - NodeID: tt.nodeID, + DstNodeID: tt.nodeID, }, meta: tt.meta, cluster: tt.cluster, diff --git a/internal/querycoord/task_test.go b/internal/querycoord/task_test.go index 7f155e11f8..695048d77b 100644 --- a/internal/querycoord/task_test.go +++ b/internal/querycoord/task_test.go @@ -153,6 +153,7 @@ func genWatchDmChannelTask(ctx context.Context, queryCoord *QueryCoord, nodeID i } baseParentTask := newBaseTask(ctx, querypb.TriggerCondition_grpcRequest) baseParentTask.taskID = 10 + baseParentTask.setState(taskDone) parentTask := &loadCollectionTask{ baseTask: baseParentTask, LoadCollectionRequest: parentReq, @@ -180,9 +181,9 @@ func genLoadSegmentTask(ctx context.Context, queryCoord *QueryCoord, nodeID int6 Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_LoadSegments, }, - NodeID: nodeID, - Schema: schema, - Infos: []*querypb.SegmentLoadInfo{segmentInfo}, + DstNodeID: nodeID, + Schema: schema, + Infos: []*querypb.SegmentLoadInfo{segmentInfo}, } baseTask := newBaseTask(ctx, querypb.TriggerCondition_grpcRequest) baseTask.taskID = 100 @@ -203,6 +204,7 @@ func genLoadSegmentTask(ctx context.Context, queryCoord *QueryCoord, nodeID int6 } baseParentTask := newBaseTask(ctx, querypb.TriggerCondition_grpcRequest) baseParentTask.taskID = 10 + baseParentTask.setState(taskDone) parentTask := &loadCollectionTask{ baseTask: baseParentTask, LoadCollectionRequest: parentReq, @@ -656,7 +658,7 @@ func Test_RescheduleDmChannelsEndWithFail(t *testing.T) { assert.Nil(t, err) } -func Test_assignInternalTask(t *testing.T) { +func Test_AssignInternalTask(t *testing.T) { refreshParams() ctx := context.Background() queryCoord, err := startQueryCoord(ctx) @@ -674,7 +676,7 @@ func Test_assignInternalTask(t *testing.T) { FieldID: 0, Binlogs: []string{funcutil.RandomString(1000)}, }) - for id := 0; id < 10000; id++ { + for id := 0; id < 3000; id++ { segmentInfo := &querypb.SegmentLoadInfo{ SegmentID: UniqueID(id), PartitionID: defaultPartitionID, @@ -685,9 +687,9 @@ func Test_assignInternalTask(t *testing.T) { Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_LoadSegments, }, - NodeID: node1.queryNodeID, - Schema: schema, - Infos: []*querypb.SegmentLoadInfo{segmentInfo}, + DstNodeID: node1.queryNodeID, + Schema: schema, + Infos: []*querypb.SegmentLoadInfo{segmentInfo}, } loadSegmentRequests = append(loadSegmentRequests, req) } @@ -701,3 +703,37 @@ func Test_assignInternalTask(t *testing.T) { err = removeAllSession() assert.Nil(t, err) } + +func Test_reverseSealedSegmentChangeInfo(t *testing.T) { + refreshParams() + ctx := context.Background() + queryCoord, err := startQueryCoord(ctx) + assert.Nil(t, err) + + node1, err := startQueryNodeServer(ctx) + assert.Nil(t, err) + waitQueryNodeOnline(queryCoord.cluster, node1.queryNodeID) + + loadCollectionTask := genLoadCollectionTask(ctx, queryCoord) + queryCoord.scheduler.Enqueue(loadCollectionTask) + waitTaskFinalState(loadCollectionTask, taskExpired) + + node2, err := startQueryNodeServer(ctx) + assert.Nil(t, err) + waitQueryNodeOnline(queryCoord.cluster, node2.queryNodeID) + + loadSegmentTask := genLoadSegmentTask(ctx, queryCoord, node2.queryNodeID) + parentTask := loadSegmentTask.parentTask + + kv := &testKv{ + returnFn: failedResult, + } + queryCoord.meta.setKvClient(kv) + + err = updateSegmentInfoFromTask(ctx, parentTask, queryCoord.meta) + assert.NotNil(t, err) + + queryCoord.Stop() + err = removeAllSession() + assert.Nil(t, err) +} diff --git a/internal/querynode/impl_test.go b/internal/querynode/impl_test.go index 1d0cc1d39a..15ae2af516 100644 --- a/internal/querynode/impl_test.go +++ b/internal/querynode/impl_test.go @@ -273,7 +273,7 @@ func TestImpl_LoadSegments(t *testing.T) { MsgType: commonpb.MsgType_WatchQueryChannels, MsgID: rand.Int63(), }, - NodeID: 0, + DstNodeID: 0, Schema: schema, LoadCondition: queryPb.TriggerCondition_grpcRequest, } diff --git a/internal/querynode/query_collection.go b/internal/querynode/query_collection.go index 9d517dc4f5..b47468d346 100644 --- a/internal/querynode/query_collection.go +++ b/internal/querynode/query_collection.go @@ -285,8 +285,6 @@ func (q *queryCollection) consumeQuery() { if err != nil { log.Warn(err.Error()) } - case *msgstream.LoadBalanceSegmentsMsg: - q.loadBalance(sm) case *msgstream.RetrieveMsg: err := q.receiveQueryMsg(sm) if err != nil { @@ -306,10 +304,6 @@ func (q *queryCollection) consumeQuery() { } } -func (q *queryCollection) loadBalance(msg *msgstream.LoadBalanceSegmentsMsg) { - //TODO:: get loadBalance info from etcd -} - func (q *queryCollection) adjustByChangeInfo(msg *msgstream.SealedSegmentsChangeInfoMsg) error { // for OnlineSegments: for _, segment := range msg.OnlineSegments { diff --git a/internal/querynode/query_node.go b/internal/querynode/query_node.go index 0002a3e5cf..c7b54b06ca 100644 --- a/internal/querynode/query_node.go +++ b/internal/querynode/query_node.go @@ -52,7 +52,7 @@ import ( "github.com/milvus-io/milvus/internal/util/typeutil" ) -const changeInfoMetaPrefix = "query-changeInfo" +const changeInfoMetaPrefix = "queryCoord-sealedSegmentChangeInfo" // make sure QueryNode implements types.QueryNode var _ types.QueryNode = (*QueryNode)(nil) diff --git a/internal/querynode/segment_loader.go b/internal/querynode/segment_loader.go index f551613948..d568949c28 100644 --- a/internal/querynode/segment_loader.go +++ b/internal/querynode/segment_loader.go @@ -18,7 +18,6 @@ import ( "path" "strconv" - "github.com/golang/protobuf/proto" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/common" @@ -34,8 +33,7 @@ import ( ) const ( - queryCoordSegmentMetaPrefix = "queryCoord-segmentMeta" - queryNodeSegmentMetaPrefix = "queryNode-segmentMeta" + queryNodeSegmentMetaPrefix = "queryNode-segmentMeta" ) // segmentLoader is only responsible for loading the field data from binlog @@ -50,23 +48,7 @@ type segmentLoader struct { indexLoader *indexLoader } -func (loader *segmentLoader) loadSegmentOfConditionHandOff(req *querypb.LoadSegmentsRequest) error { - return errors.New("TODO: implement hand off") -} - -func (loader *segmentLoader) loadSegmentOfConditionLoadBalance(req *querypb.LoadSegmentsRequest) error { - return loader.loadSegment(req, false) -} - -func (loader *segmentLoader) loadSegmentOfConditionGRPC(req *querypb.LoadSegmentsRequest) error { - return loader.loadSegment(req, true) -} - -func (loader *segmentLoader) loadSegmentOfConditionNodeDown(req *querypb.LoadSegmentsRequest) error { - return loader.loadSegment(req, true) -} - -func (loader *segmentLoader) loadSegment(req *querypb.LoadSegmentsRequest, onService bool) error { +func (loader *segmentLoader) loadSegment(req *querypb.LoadSegmentsRequest) error { // no segment needs to load, return if len(req.Infos) == 0 { return nil @@ -101,7 +83,7 @@ func (loader *segmentLoader) loadSegment(req *querypb.LoadSegmentsRequest, onSer segmentGC() return err } - segment := newSegment(collection, segmentID, partitionID, collectionID, "", segmentTypeSealed, onService) + segment := newSegment(collection, segmentID, partitionID, collectionID, "", segmentTypeSealed, true) err = loader.loadSegmentInternal(collectionID, segment, info) if err != nil { deleteSegment(segment) @@ -109,40 +91,7 @@ func (loader *segmentLoader) loadSegment(req *querypb.LoadSegmentsRequest, onSer segmentGC() return err } - if onService { - key := fmt.Sprintf("%s/%d", queryCoordSegmentMetaPrefix, segmentID) - value, err := loader.etcdKV.Load(key) - if err != nil { - deleteSegment(segment) - log.Warn("error when load segment info from etcd", zap.Any("error", err.Error())) - segmentGC() - return err - } - segmentInfo := &querypb.SegmentInfo{} - err = proto.Unmarshal([]byte(value), segmentInfo) - if err != nil { - deleteSegment(segment) - log.Warn("error when unmarshal segment info from etcd", zap.Any("error", err.Error())) - segmentGC() - return err - } - segmentInfo.SegmentState = querypb.SegmentState_sealed - newKey := fmt.Sprintf("%s/%d", queryNodeSegmentMetaPrefix, segmentID) - newValue, err := proto.Marshal(segmentInfo) - if err != nil { - deleteSegment(segment) - log.Warn("error when marshal segment info", zap.Error(err)) - segmentGC() - return err - } - err = loader.etcdKV.Save(newKey, string(newValue)) - if err != nil { - deleteSegment(segment) - log.Warn("error when update segment info to etcd", zap.Any("error", err.Error())) - segmentGC() - return err - } - } + newSegments = append(newSegments, segment) } diff --git a/internal/querynode/segment_loader_test.go b/internal/querynode/segment_loader_test.go index 749ad406a7..9f103b8e32 100644 --- a/internal/querynode/segment_loader_test.go +++ b/internal/querynode/segment_loader_test.go @@ -13,11 +13,9 @@ package querynode import ( "context" - "fmt" "math/rand" "testing" - "github.com/golang/protobuf/proto" "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus/internal/proto/commonpb" @@ -37,41 +35,6 @@ func TestSegmentLoader_loadSegment(t *testing.T) { fieldBinlog, err := saveSimpleBinLog(ctx) assert.NoError(t, err) - t.Run("test no segment meta", func(t *testing.T) { - historical, err := genSimpleHistorical(ctx) - assert.NoError(t, err) - - err = historical.replica.removeSegment(defaultSegmentID) - assert.NoError(t, err) - loader := newSegmentLoader(ctx, nil, nil, historical.replica, kv) - assert.NotNil(t, loader) - - req := &querypb.LoadSegmentsRequest{ - Base: &commonpb.MsgBase{ - MsgType: commonpb.MsgType_WatchQueryChannels, - MsgID: rand.Int63(), - }, - NodeID: 0, - Schema: schema, - LoadCondition: querypb.TriggerCondition_grpcRequest, - Infos: []*querypb.SegmentLoadInfo{ - { - SegmentID: defaultSegmentID, - PartitionID: defaultPartitionID, - CollectionID: defaultCollectionID, - BinlogPaths: fieldBinlog, - }, - }, - } - - key := fmt.Sprintf("%s/%d", queryCoordSegmentMetaPrefix, defaultSegmentID) - err = kv.Remove(key) - assert.NoError(t, err) - - err = loader.loadSegment(req, true) - assert.Error(t, err) - }) - t.Run("test load segment", func(t *testing.T) { historical, err := genSimpleHistorical(ctx) assert.NoError(t, err) @@ -86,7 +49,7 @@ func TestSegmentLoader_loadSegment(t *testing.T) { MsgType: commonpb.MsgType_WatchQueryChannels, MsgID: rand.Int63(), }, - NodeID: 0, + DstNodeID: 0, Schema: schema, LoadCondition: querypb.TriggerCondition_grpcRequest, Infos: []*querypb.SegmentLoadInfo{ @@ -99,14 +62,7 @@ func TestSegmentLoader_loadSegment(t *testing.T) { }, } - key := fmt.Sprintf("%s/%d", queryCoordSegmentMetaPrefix, defaultSegmentID) - segmentInfo := &querypb.SegmentInfo{} - value, err := proto.Marshal(segmentInfo) - assert.Nil(t, err) - err = kv.Save(key, string(value)) - assert.NoError(t, err) - - err = loader.loadSegment(req, true) + err = loader.loadSegment(req) assert.NoError(t, err) }) @@ -124,7 +80,7 @@ func TestSegmentLoader_loadSegment(t *testing.T) { MsgType: commonpb.MsgType_WatchQueryChannels, MsgID: rand.Int63(), }, - NodeID: 0, + DstNodeID: 0, Schema: schema, LoadCondition: querypb.TriggerCondition_grpcRequest, Infos: []*querypb.SegmentLoadInfo{ @@ -137,60 +93,11 @@ func TestSegmentLoader_loadSegment(t *testing.T) { }, } - key := fmt.Sprintf("%s/%d", queryCoordSegmentMetaPrefix, defaultSegmentID) - segmentInfo := &querypb.SegmentInfo{} - value, err := proto.Marshal(segmentInfo) - assert.Nil(t, err) - err = kv.Save(key, string(value)) - assert.NoError(t, err) - - err = loader.loadSegment(req, true) + err = loader.loadSegment(req) assert.Error(t, err) }) } -func TestSegmentLoader_notOnService(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - historical, err := genSimpleHistorical(ctx) - assert.NoError(t, err) - - err = historical.replica.removeSegment(defaultSegmentID) - assert.NoError(t, err) - - kv, err := genEtcdKV() - assert.NoError(t, err) - - loader := newSegmentLoader(ctx, nil, nil, historical.replica, kv) - assert.NotNil(t, loader) - - schema := genSimpleInsertDataSchema() - - fieldBinlog, err := saveSimpleBinLog(ctx) - assert.NoError(t, err) - - req := &querypb.LoadSegmentsRequest{ - Base: &commonpb.MsgBase{ - MsgType: commonpb.MsgType_WatchQueryChannels, - MsgID: rand.Int63(), - }, - NodeID: 0, - Schema: schema, - LoadCondition: querypb.TriggerCondition_grpcRequest, - Infos: []*querypb.SegmentLoadInfo{ - { - SegmentID: defaultSegmentID, - PartitionID: defaultPartitionID, - CollectionID: defaultCollectionID, - BinlogPaths: fieldBinlog, - }, - }, - } - err = loader.loadSegment(req, false) - assert.NoError(t, err) -} - func TestSegmentLoader_loadSegmentFieldsData(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -276,14 +183,6 @@ func TestSegmentLoader_invalid(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - t.Run("test loadSegmentOfConditionHandOff", func(t *testing.T) { - historical, err := genSimpleHistorical(ctx) - assert.NoError(t, err) - - err = historical.loader.loadSegmentOfConditionHandOff(nil) - assert.Error(t, err) - }) - t.Run("test no collection", func(t *testing.T) { historical, err := genSimpleHistorical(ctx) assert.NoError(t, err) @@ -296,7 +195,7 @@ func TestSegmentLoader_invalid(t *testing.T) { MsgType: commonpb.MsgType_WatchQueryChannels, MsgID: rand.Int63(), }, - NodeID: 0, + DstNodeID: 0, LoadCondition: querypb.TriggerCondition_grpcRequest, Infos: []*querypb.SegmentLoadInfo{ { @@ -307,7 +206,7 @@ func TestSegmentLoader_invalid(t *testing.T) { }, } - err = historical.loader.loadSegment(req, true) + err = historical.loader.loadSegment(req) assert.Error(t, err) }) @@ -371,7 +270,7 @@ func TestSegmentLoader_invalid(t *testing.T) { MsgType: commonpb.MsgType_WatchQueryChannels, MsgID: rand.Int63(), }, - NodeID: 0, + DstNodeID: 0, Schema: schema, LoadCondition: querypb.TriggerCondition_grpcRequest, Infos: []*querypb.SegmentLoadInfo{ @@ -382,7 +281,7 @@ func TestSegmentLoader_invalid(t *testing.T) { }, }, } - err = historical.loader.loadSegment(req, false) + err = historical.loader.loadSegment(req) assert.Error(t, err) }) } diff --git a/internal/querynode/task.go b/internal/querynode/task.go index 4bef0071b3..c3f4134346 100644 --- a/internal/querynode/task.go +++ b/internal/querynode/task.go @@ -382,17 +382,7 @@ func (l *loadSegmentsTask) Execute(ctx context.Context) error { return err } - switch l.req.LoadCondition { - case queryPb.TriggerCondition_handoff: - err = l.node.historical.loader.loadSegmentOfConditionHandOff(l.req) - case queryPb.TriggerCondition_loadBalance: - err = l.node.historical.loader.loadSegmentOfConditionLoadBalance(l.req) - case queryPb.TriggerCondition_grpcRequest: - err = l.node.historical.loader.loadSegmentOfConditionGRPC(l.req) - case queryPb.TriggerCondition_nodeDown: - err = l.node.historical.loader.loadSegmentOfConditionNodeDown(l.req) - } - + err = l.node.historical.loader.loadSegment(l.req) if err != nil { log.Warn(err.Error()) return err diff --git a/internal/querynode/task_test.go b/internal/querynode/task_test.go index b556665938..9662aecce6 100644 --- a/internal/querynode/task_test.go +++ b/internal/querynode/task_test.go @@ -253,39 +253,6 @@ func TestTask_loadSegmentsTask(t *testing.T) { assert.Error(t, err) }) - t.Run("test execute load balance", func(t *testing.T) { - node, err := genSimpleQueryNode(ctx) - assert.NoError(t, err) - - task := loadSegmentsTask{ - req: genLoadEmptySegmentsRequest(), - node: node, - } - task.req.Infos = []*querypb.SegmentLoadInfo{ - { - SegmentID: defaultSegmentID + 1, - PartitionID: defaultPartitionID + 1, - CollectionID: defaultCollectionID + 1, - }, - } - task.req.LoadCondition = querypb.TriggerCondition_loadBalance - err = task.Execute(ctx) - assert.Error(t, err) - }) - - t.Run("test execute load hand-off", func(t *testing.T) { - node, err := genSimpleQueryNode(ctx) - assert.NoError(t, err) - - task := loadSegmentsTask{ - req: genLoadEmptySegmentsRequest(), - node: node, - } - task.req.LoadCondition = querypb.TriggerCondition_handoff - err = task.Execute(ctx) - assert.Error(t, err) - }) - t.Run("test OOM", func(t *testing.T) { node, err := genSimpleQueryNode(ctx) assert.NoError(t, err)