2021-11-17 19:43:50 +08:00
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
2021-06-15 12:41:40 +08:00
// with the License. You may obtain a copy of the License at
//
2021-11-17 19:43:50 +08:00
// http://www.apache.org/licenses/LICENSE-2.0
2021-06-15 12:41:40 +08:00
//
2021-11-17 19:43:50 +08:00
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
2021-06-15 12:41:40 +08:00
2021-06-22 16:44:09 +08:00
package querycoord
2021-06-15 12:41:40 +08:00
import (
2021-10-22 19:07:15 +08:00
"context"
2021-06-15 12:41:40 +08:00
"errors"
2021-06-19 11:45:09 +08:00
"fmt"
"path/filepath"
2021-06-15 12:41:40 +08:00
"strconv"
2021-11-15 15:17:09 +08:00
"strings"
2021-06-15 12:41:40 +08:00
"sync"
2021-06-19 11:45:09 +08:00
"github.com/golang/protobuf/proto"
2021-06-15 12:41:40 +08:00
"go.uber.org/zap"
2021-09-29 09:56:04 +08:00
"github.com/milvus-io/milvus/internal/kv"
2021-06-15 12:41:40 +08:00
"github.com/milvus-io/milvus/internal/log"
2021-10-22 19:07:15 +08:00
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb"
2021-11-13 08:49:08 +08:00
"github.com/milvus-io/milvus/internal/proto/datapb"
2021-10-22 19:07:15 +08:00
"github.com/milvus-io/milvus/internal/proto/internalpb"
2021-06-15 12:41:40 +08:00
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/schemapb"
2021-11-17 12:13:10 +08:00
"github.com/milvus-io/milvus/internal/util"
2021-10-22 19:07:15 +08:00
"github.com/milvus-io/milvus/internal/util/mqclient"
2021-06-15 12:41:40 +08:00
)
2021-06-19 11:45:09 +08:00
const (
2021-10-22 19:07:15 +08:00
collectionMetaPrefix = "queryCoord-collectionMeta"
queryChannelMetaPrefix = "queryCoord-queryChannel"
2021-11-13 08:49:08 +08:00
deltaChannelMetaPrefix = "queryCoord-deltaChannel"
2021-10-27 23:00:24 +08:00
globalQuerySeekPositionPrefix = "queryCoord-globalQuerySeekPosition"
2021-06-19 11:45:09 +08:00
)
2021-10-22 19:07:15 +08:00
type col2SegmentInfos = map [ UniqueID ] [ ] * querypb . SegmentInfo
2021-10-26 15:18:22 +08:00
type col2SealedSegmentChangeInfos = map [ UniqueID ] * querypb . SealedSegmentsChangeInfo
2021-10-22 19:07:15 +08:00
2021-09-26 21:57:57 +08:00
// Meta contains information about all loaded collections and partitions, including segment information and vchannel information
2021-08-02 22:39:25 +08:00
type Meta interface {
reloadFromKV ( ) error
2021-10-22 19:07:15 +08:00
setKvClient ( kv kv . MetaKv )
2021-08-02 22:39:25 +08:00
showCollections ( ) [ ] * querypb . CollectionInfo
hasCollection ( collectionID UniqueID ) bool
getCollectionInfoByID ( collectionID UniqueID ) ( * querypb . CollectionInfo , error )
addCollection ( collectionID UniqueID , schema * schemapb . CollectionSchema ) error
releaseCollection ( collectionID UniqueID ) error
addPartition ( collectionID UniqueID , partitionID UniqueID ) error
showPartitions ( collectionID UniqueID ) ( [ ] * querypb . PartitionStates , error )
hasPartition ( collectionID UniqueID , partitionID UniqueID ) bool
hasReleasePartition ( collectionID UniqueID , partitionID UniqueID ) bool
releasePartition ( collectionID UniqueID , partitionID UniqueID ) error
deleteSegmentInfoByNodeID ( nodeID UniqueID ) error
2021-10-22 19:07:15 +08:00
setSegmentInfos ( segmentInfos map [ UniqueID ] * querypb . SegmentInfo ) error
2021-08-02 22:39:25 +08:00
showSegmentInfos ( collectionID UniqueID , partitionIDs [ ] UniqueID ) [ ] * querypb . SegmentInfo
getSegmentInfoByID ( segmentID UniqueID ) ( * querypb . SegmentInfo , error )
2021-11-06 15:22:56 +08:00
getSegmentInfosByNode ( nodeID int64 ) [ ] * querypb . SegmentInfo
2021-08-02 22:39:25 +08:00
getPartitionStatesByID ( collectionID UniqueID , partitionID UniqueID ) ( * querypb . PartitionStates , error )
getDmChannelsByNodeID ( collectionID UniqueID , nodeID int64 ) ( [ ] string , error )
addDmChannel ( collectionID UniqueID , nodeID int64 , channels [ ] string ) error
removeDmChannel ( collectionID UniqueID , nodeID int64 , channels [ ] string ) error
2021-11-13 08:49:08 +08:00
getDeltaChannelsByCollectionID ( collectionID UniqueID ) ( [ ] * datapb . VchannelInfo , error )
setDeltaChannel ( collectionID UniqueID , info [ ] * datapb . VchannelInfo ) error
2021-08-02 22:39:25 +08:00
getQueryChannelInfoByID ( collectionID UniqueID ) ( * querypb . QueryChannelInfo , error )
2021-10-22 19:07:15 +08:00
getQueryStreamByID ( collectionID UniqueID ) ( msgstream . MsgStream , error )
2021-08-02 22:39:25 +08:00
setLoadType ( collectionID UniqueID , loadType querypb . LoadType ) error
getLoadType ( collectionID UniqueID ) ( querypb . LoadType , error )
setLoadPercentage ( collectionID UniqueID , partitionID UniqueID , percentage int64 , loadType querypb . LoadType ) error
2021-09-29 09:56:04 +08:00
//printMeta()
2021-10-22 19:07:15 +08:00
saveGlobalSealedSegInfos ( saves col2SegmentInfos ) ( col2SealedSegmentChangeInfos , error )
removeGlobalSealedSegInfos ( collectionID UniqueID , partitionIDs [ ] UniqueID ) ( col2SealedSegmentChangeInfos , error )
2021-10-26 15:18:22 +08:00
sendSealedSegmentChangeInfos ( collectionID UniqueID , changeInfos * querypb . SealedSegmentsChangeInfo ) ( * querypb . QueryChannelInfo , map [ string ] [ ] mqclient . MessageID , error )
2021-08-02 22:39:25 +08:00
}
2021-10-02 09:49:40 +08:00
// MetaReplica records the current load information on all querynodes
2021-08-02 22:39:25 +08:00
type MetaReplica struct {
2021-10-22 19:07:15 +08:00
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 )
2021-06-19 11:45:09 +08:00
2021-10-22 19:07:15 +08:00
//sync.RWMutex
2021-06-15 12:41:40 +08:00
collectionInfos map [ UniqueID ] * querypb . CollectionInfo
2021-10-22 19:07:15 +08:00
collectionMu sync . RWMutex
2021-06-15 12:41:40 +08:00
segmentInfos map [ UniqueID ] * querypb . SegmentInfo
2021-10-22 19:07:15 +08:00
segmentMu sync . RWMutex
2021-06-15 12:41:40 +08:00
queryChannelInfos map [ UniqueID ] * querypb . QueryChannelInfo
2021-10-22 19:07:15 +08:00
channelMu sync . RWMutex
2021-11-13 08:49:08 +08:00
deltaChannelInfos map [ UniqueID ] [ ] * datapb . VchannelInfo
deltaChannelMu sync . RWMutex
2021-10-22 19:07:15 +08:00
queryStreams map [ UniqueID ] msgstream . MsgStream
streamMu sync . RWMutex
2021-06-15 12:41:40 +08:00
2021-10-27 23:00:24 +08:00
globalSeekPosition * internalpb . MsgPosition
2021-08-02 22:39:25 +08:00
//partitionStates map[UniqueID]*querypb.PartitionStates
2021-06-15 12:41:40 +08:00
}
2021-10-22 19:07:15 +08:00
func newMeta ( ctx context . Context , kv kv . MetaKv , factory msgstream . Factory , idAllocator func ( ) ( UniqueID , error ) ) ( Meta , error ) {
childCtx , cancel := context . WithCancel ( ctx )
2021-06-15 12:41:40 +08:00
collectionInfos := make ( map [ UniqueID ] * querypb . CollectionInfo )
segmentInfos := make ( map [ UniqueID ] * querypb . SegmentInfo )
queryChannelInfos := make ( map [ UniqueID ] * querypb . QueryChannelInfo )
2021-11-13 08:49:08 +08:00
deltaChannelInfos := make ( map [ UniqueID ] [ ] * datapb . VchannelInfo )
2021-10-22 19:07:15 +08:00
queryMsgStream := make ( map [ UniqueID ] msgstream . MsgStream )
2021-10-27 23:00:24 +08:00
position := & internalpb . MsgPosition { }
2021-06-19 11:45:09 +08:00
2021-08-02 22:39:25 +08:00
m := & MetaReplica {
2021-10-22 19:07:15 +08:00
ctx : childCtx ,
cancel : cancel ,
client : kv ,
msFactory : factory ,
idAllocator : idAllocator ,
2021-10-27 23:00:24 +08:00
collectionInfos : collectionInfos ,
segmentInfos : segmentInfos ,
queryChannelInfos : queryChannelInfos ,
2021-11-13 08:49:08 +08:00
deltaChannelInfos : deltaChannelInfos ,
2021-10-27 23:00:24 +08:00
queryStreams : queryMsgStream ,
globalSeekPosition : position ,
2021-06-15 12:41:40 +08:00
}
2021-06-19 11:45:09 +08:00
err := m . reloadFromKV ( )
if err != nil {
return nil , err
}
return m , nil
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) reloadFromKV ( ) error {
2021-12-01 16:39:31 +08:00
log . Debug ( "start reload from kv" )
2021-06-19 11:45:09 +08:00
collectionKeys , collectionValues , err := m . client . LoadWithPrefix ( collectionMetaPrefix )
if err != nil {
return err
}
for index := range collectionKeys {
collectionID , err := strconv . ParseInt ( filepath . Base ( collectionKeys [ index ] ) , 10 , 64 )
if err != nil {
return err
}
collectionInfo := & querypb . CollectionInfo { }
2021-09-29 20:26:00 +08:00
err = proto . Unmarshal ( [ ] byte ( collectionValues [ index ] ) , collectionInfo )
2021-06-19 11:45:09 +08:00
if err != nil {
return err
}
m . collectionInfos [ collectionID ] = collectionInfo
}
2021-11-17 12:13:10 +08:00
segmentKeys , segmentValues , err := m . client . LoadWithPrefix ( util . SegmentMetaPrefix )
2021-06-19 11:45:09 +08:00
if err != nil {
return err
}
for index := range segmentKeys {
segmentID , err := strconv . ParseInt ( filepath . Base ( segmentKeys [ index ] ) , 10 , 64 )
if err != nil {
return err
}
segmentInfo := & querypb . SegmentInfo { }
2021-09-29 20:26:00 +08:00
err = proto . Unmarshal ( [ ] byte ( segmentValues [ index ] ) , segmentInfo )
2021-06-19 11:45:09 +08:00
if err != nil {
return err
}
m . segmentInfos [ segmentID ] = segmentInfo
}
queryChannelKeys , queryChannelValues , err := m . client . LoadWithPrefix ( queryChannelMetaPrefix )
if err != nil {
return nil
}
for index := range queryChannelKeys {
collectionID , err := strconv . ParseInt ( filepath . Base ( queryChannelKeys [ index ] ) , 10 , 64 )
if err != nil {
return err
}
queryChannelInfo := & querypb . QueryChannelInfo { }
2021-09-29 20:26:00 +08:00
err = proto . Unmarshal ( [ ] byte ( queryChannelValues [ index ] ) , queryChannelInfo )
2021-06-19 11:45:09 +08:00
if err != nil {
return err
}
m . queryChannelInfos [ collectionID ] = queryChannelInfo
}
2021-11-13 08:49:08 +08:00
deltaChannelKeys , deltaChannelValues , err := m . client . LoadWithPrefix ( deltaChannelMetaPrefix )
if err != nil {
return nil
}
for index , value := range deltaChannelValues {
2021-11-15 15:17:09 +08:00
pathStrings := strings . Split ( deltaChannelKeys [ index ] , "/" )
collectionID , err := strconv . ParseInt ( pathStrings [ len ( pathStrings ) - 2 ] , 10 , 64 )
2021-11-13 08:49:08 +08:00
if err != nil {
return err
}
deltaChannelInfo := & datapb . VchannelInfo { }
err = proto . Unmarshal ( [ ] byte ( value ) , deltaChannelInfo )
if err != nil {
return err
}
m . deltaChannelInfos [ collectionID ] = append ( m . deltaChannelInfos [ collectionID ] , deltaChannelInfo )
}
2021-10-27 23:00:24 +08:00
globalSeekPosValue , err := m . client . Load ( globalQuerySeekPositionPrefix )
if err == nil {
position := & internalpb . MsgPosition { }
err = proto . Unmarshal ( [ ] byte ( globalSeekPosValue ) , position )
if err != nil {
return err
}
m . globalSeekPosition = position
}
2021-06-19 11:45:09 +08:00
//TODO::update partition states
2021-12-01 16:39:31 +08:00
log . Debug ( "reload from kv finished" )
2021-06-19 11:45:09 +08:00
return nil
2021-06-15 12:41:40 +08:00
}
2021-10-22 19:07:15 +08:00
func ( m * MetaReplica ) setKvClient ( kv kv . MetaKv ) {
m . client = kv
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) showCollections ( ) [ ] * querypb . CollectionInfo {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-15 12:41:40 +08:00
2021-08-02 22:39:25 +08:00
collections := make ( [ ] * querypb . CollectionInfo , 0 )
for _ , info := range m . collectionInfos {
collections = append ( collections , proto . Clone ( info ) . ( * querypb . CollectionInfo ) )
2021-06-15 12:41:40 +08:00
}
return collections
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) showPartitions ( collectionID UniqueID ) ( [ ] * querypb . PartitionStates , error ) {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-15 12:41:40 +08:00
2021-06-24 21:10:13 +08:00
//TODO::should update after load collection
2021-08-02 22:39:25 +08:00
results := make ( [ ] * querypb . PartitionStates , 0 )
2021-06-15 12:41:40 +08:00
if info , ok := m . collectionInfos [ collectionID ] ; ok {
2021-08-02 22:39:25 +08:00
for _ , state := range info . PartitionStates {
results = append ( results , proto . Clone ( state ) . ( * querypb . PartitionStates ) )
}
return results , nil
2021-06-15 12:41:40 +08:00
}
return nil , errors . New ( "showPartitions: can't find collection in collectionInfos" )
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) hasCollection ( collectionID UniqueID ) bool {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-15 12:41:40 +08:00
if _ , ok := m . collectionInfos [ collectionID ] ; ok {
return true
}
return false
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) hasPartition ( collectionID UniqueID , partitionID UniqueID ) bool {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-15 12:41:40 +08:00
if info , ok := m . collectionInfos [ collectionID ] ; ok {
for _ , id := range info . PartitionIDs {
if partitionID == id {
return true
}
}
}
return false
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) hasReleasePartition ( collectionID UniqueID , partitionID UniqueID ) bool {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-23 17:44:12 +08:00
if info , ok := m . collectionInfos [ collectionID ] ; ok {
for _ , id := range info . ReleasedPartitionIDs {
if partitionID == id {
return true
}
}
}
return false
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) addCollection ( collectionID UniqueID , schema * schemapb . CollectionSchema ) error {
2021-10-22 19:07:15 +08:00
hasCollection := m . hasCollection ( collectionID )
if ! hasCollection {
2021-06-15 12:41:40 +08:00
partitions := make ( [ ] UniqueID , 0 )
2021-08-02 22:39:25 +08:00
partitionStates := make ( [ ] * querypb . PartitionStates , 0 )
2021-06-15 12:41:40 +08:00
channels := make ( [ ] * querypb . DmChannelInfo , 0 )
newCollection := & querypb . CollectionInfo {
2021-08-02 22:39:25 +08:00
CollectionID : collectionID ,
PartitionIDs : partitions ,
PartitionStates : partitionStates ,
ChannelInfos : channels ,
Schema : schema ,
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
err := saveGlobalCollectionInfo ( collectionID , newCollection , m . client )
2021-06-19 11:45:09 +08:00
if err != nil {
log . Error ( "save collectionInfo error" , zap . Any ( "error" , err . Error ( ) ) , zap . Int64 ( "collectionID" , collectionID ) )
2021-08-02 22:39:25 +08:00
return err
2021-06-19 11:45:09 +08:00
}
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
m . collectionInfos [ collectionID ] = newCollection
m . collectionMu . Unlock ( )
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
return nil
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) addPartition ( collectionID UniqueID , partitionID UniqueID ) error {
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
defer m . collectionMu . Unlock ( )
if info , ok := m . collectionInfos [ collectionID ] ; ok {
col := proto . Clone ( info ) . ( * querypb . CollectionInfo )
2021-08-02 22:39:25 +08:00
log . Debug ( "add a partition to MetaReplica..." , zap . Int64s ( "partitionIDs" , col . PartitionIDs ) )
2021-06-15 12:41:40 +08:00
for _ , id := range col . PartitionIDs {
if id == partitionID {
2021-08-02 22:39:25 +08:00
return nil
2021-06-15 12:41:40 +08:00
}
}
col . PartitionIDs = append ( col . PartitionIDs , partitionID )
2021-06-23 17:44:12 +08:00
releasedPartitionIDs := make ( [ ] UniqueID , 0 )
for _ , id := range col . ReleasedPartitionIDs {
if id != partitionID {
releasedPartitionIDs = append ( releasedPartitionIDs , id )
}
}
col . ReleasedPartitionIDs = releasedPartitionIDs
2021-08-02 22:39:25 +08:00
col . PartitionStates = append ( col . PartitionStates , & querypb . PartitionStates {
PartitionID : partitionID ,
State : querypb . PartitionState_NotPresent ,
} )
log . Debug ( "add a partition to MetaReplica" , zap . Int64s ( "partitionIDs" , col . PartitionIDs ) )
err := saveGlobalCollectionInfo ( collectionID , col , m . client )
2021-06-19 11:45:09 +08:00
if err != nil {
log . Error ( "save collectionInfo error" , zap . Any ( "error" , err . Error ( ) ) , zap . Int64 ( "collectionID" , collectionID ) )
2021-08-02 22:39:25 +08:00
return err
2021-06-19 11:45:09 +08:00
}
2021-10-22 19:07:15 +08:00
m . collectionInfos [ collectionID ] = col
2021-06-15 12:41:40 +08:00
return nil
}
return errors . New ( "addPartition: can't find collection when add partition" )
}
2021-10-22 19:07:15 +08:00
func ( m * MetaReplica ) deleteSegmentInfoByNodeID ( nodeID UniqueID ) error {
m . segmentMu . Lock ( )
defer m . segmentMu . Unlock ( )
2021-06-19 11:45:09 +08:00
2021-10-22 19:07:15 +08:00
segmentIDsToRemove := make ( [ ] UniqueID , 0 )
for segmentID , info := range m . segmentInfos {
if info . NodeID == nodeID {
segmentIDsToRemove = append ( segmentIDsToRemove , segmentID )
2021-06-19 11:45:09 +08:00
}
2021-10-22 19:07:15 +08:00
}
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 {
2021-06-19 11:45:09 +08:00
delete ( m . segmentInfos , segmentID )
}
2021-08-02 22:39:25 +08:00
return nil
2021-06-19 11:45:09 +08:00
}
2021-10-22 19:07:15 +08:00
func ( m * MetaReplica ) setSegmentInfos ( segmentInfos map [ UniqueID ] * querypb . SegmentInfo ) error {
m . segmentMu . Lock ( )
defer m . segmentMu . Unlock ( )
2021-06-19 11:45:09 +08:00
2021-10-22 19:07:15 +08:00
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 )
2021-11-08 21:00:02 +08:00
segmentsCompactionFrom := make ( [ ] UniqueID , 0 )
2021-11-17 12:13:10 +08:00
// get segmentInfos to colSegmentInfos
2021-10-22 19:07:15 +08:00
for collectionID , onlineInfos := range saves {
2021-10-26 15:18:22 +08:00
segmentsChangeInfo := & querypb . SealedSegmentsChangeInfo {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_SealedSegmentsChangeInfo ,
} ,
Infos : [ ] * querypb . SegmentChangeInfo { } ,
}
2021-10-22 19:07:15 +08:00
for _ , info := range onlineInfos {
segmentID := info . SegmentID
onlineNodeID := info . NodeID
2021-10-26 15:18:22 +08:00
changeInfo := & querypb . SegmentChangeInfo {
2021-10-22 19:07:15 +08:00
OnlineNodeID : onlineNodeID ,
OnlineSegments : [ ] * querypb . SegmentInfo { info } ,
}
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 {
2021-10-26 15:18:22 +08:00
changeInfo . OfflineNodeID = offlineNodeID
changeInfo . OfflineSegments = [ ] * querypb . SegmentInfo { offlineInfo }
2021-10-22 19:07:15 +08:00
}
}
2021-10-26 15:18:22 +08:00
segmentsChangeInfo . Infos = append ( segmentsChangeInfo . Infos , changeInfo )
2021-11-08 21:00:02 +08:00
// generate offline segment change info if the loaded segment is compacted from other sealed segments
for _ , compactionSegmentID := range info . CompactionFrom {
compactionSegmentInfo , err := m . getSegmentInfoByID ( compactionSegmentID )
if err == nil && compactionSegmentInfo . SegmentState == querypb . SegmentState_sealed {
segmentsChangeInfo . Infos = append ( segmentsChangeInfo . Infos , & querypb . SegmentChangeInfo {
OfflineNodeID : compactionSegmentInfo . NodeID ,
OfflineSegments : [ ] * querypb . SegmentInfo { compactionSegmentInfo } ,
} )
segmentsCompactionFrom = append ( segmentsCompactionFrom , compactionSegmentID )
} else {
return nil , fmt . Errorf ( "saveGlobalSealedSegInfos: the compacted segment %d has not been loaded into memory" , compactionSegmentID )
}
}
2021-10-22 19:07:15 +08:00
}
2021-10-26 15:18:22 +08:00
col2SegmentChangeInfos [ collectionID ] = segmentsChangeInfo
2021-10-22 19:07:15 +08:00
}
queryChannelInfosMap := make ( map [ UniqueID ] * querypb . QueryChannelInfo )
2021-10-27 23:00:24 +08:00
var globalSeekPositionTmp * internalpb . MsgPosition
2021-10-22 19:07:15 +08:00
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
}
2021-10-26 15:18:22 +08:00
// len(messageIDs) == 1
2021-10-22 19:07:15 +08:00
messageIDs , ok := messageIDInfos [ queryChannelInfo . QueryChannelID ]
if ! ok || len ( messageIDs ) == 0 {
return col2SegmentChangeInfos , errors . New ( "updateGlobalSealedSegmentInfos: send sealed segment change info failed" )
}
2021-10-26 20:25:05 +08:00
if queryChannelInfo . SeekPosition == nil {
queryChannelInfo . SeekPosition = & internalpb . MsgPosition {
ChannelName : queryChannelInfo . QueryChannelID ,
}
}
2021-10-26 15:18:22 +08:00
queryChannelInfo . SeekPosition . MsgID = messageIDs [ 0 ] . Serialize ( )
2021-10-22 19:07:15 +08:00
// 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
2021-10-27 23:00:24 +08:00
globalSeekPositionTmp = queryChannelInfo . SeekPosition
2021-10-22 19:07:15 +08:00
}
2021-10-29 18:04:49 +08:00
// save segmentInfo to etcd
segmentInfoKvs := make ( map [ string ] string )
2021-10-22 19:07:15 +08:00
for _ , infos := range saves {
for _ , info := range infos {
segmentInfoBytes , err := proto . Marshal ( info )
2021-06-19 11:45:09 +08:00
if err != nil {
2021-10-22 19:07:15 +08:00
return col2SegmentChangeInfos , err
2021-06-19 11:45:09 +08:00
}
2021-11-17 12:13:10 +08:00
segmentKey := fmt . Sprintf ( "%s/%d" , util . SegmentMetaPrefix , info . SegmentID )
2021-10-29 18:04:49 +08:00
segmentInfoKvs [ segmentKey ] = string ( segmentInfoBytes )
}
}
for key , value := range segmentInfoKvs {
err := m . client . Save ( key , value )
if err != nil {
panic ( err )
2021-06-19 11:45:09 +08:00
}
}
2021-08-02 22:39:25 +08:00
2021-11-08 21:00:02 +08:00
// remove compacted segment info from etcd
for _ , segmentID := range segmentsCompactionFrom {
2021-11-17 12:13:10 +08:00
segmentKey := fmt . Sprintf ( "%s/%d" , util . SegmentMetaPrefix , segmentID )
2021-11-08 21:00:02 +08:00
err := m . client . Remove ( segmentKey )
if err != nil {
panic ( err )
}
}
2021-10-29 18:04:49 +08:00
// save queryChannelInfo and sealedSegmentsChangeInfo to etcd
saveKvs := make ( map [ string ] string )
2021-10-22 19:07:15 +08:00
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 )
}
2021-10-27 23:00:24 +08:00
seekPos , err := proto . Marshal ( globalSeekPositionTmp )
if err != nil {
return col2SegmentChangeInfos , err
}
saveKvs [ globalQuerySeekPositionPrefix ] = string ( seekPos )
2021-10-22 19:07:15 +08:00
// 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 {
2021-10-26 15:18:22 +08:00
changeInfoBytes , err := proto . Marshal ( changeInfos )
if err != nil {
return col2SegmentChangeInfos , err
2021-10-22 19:07:15 +08:00
}
2021-10-26 15:18:22 +08:00
// TODO:: segmentChangeInfo clear in etcd with coord gc and queryNode watch the changeInfo meta to deal changeInfoMsg
2021-11-17 12:13:10 +08:00
changeInfoKey := fmt . Sprintf ( "%s/%d" , util . ChangeInfoMetaPrefix , changeInfos . Base . MsgID )
2021-10-26 15:18:22 +08:00
saveKvs [ changeInfoKey ] = string ( changeInfoBytes )
2021-10-22 19:07:15 +08:00
}
2021-10-27 23:00:24 +08:00
err = m . client . MultiSave ( saveKvs )
2021-10-22 19:07:15 +08:00
if err != nil {
2021-10-29 18:04:49 +08:00
panic ( err )
2021-10-22 19:07:15 +08:00
}
2021-10-29 18:04:49 +08:00
2021-10-22 19:07:15 +08:00
m . segmentMu . Lock ( )
for _ , segmentInfos := range saves {
for _ , info := range segmentInfos {
segmentID := info . SegmentID
m . segmentInfos [ segmentID ] = info
}
}
2021-11-08 21:00:02 +08:00
for _ , segmentID := range segmentsCompactionFrom {
delete ( m . segmentInfos , segmentID )
}
2021-10-22 19:07:15 +08:00
m . segmentMu . Unlock ( )
m . channelMu . Lock ( )
for collectionID , channelInfo := range queryChannelInfosMap {
m . queryChannelInfos [ collectionID ] = channelInfo
}
2021-10-27 23:00:24 +08:00
m . globalSeekPosition = globalSeekPositionTmp
2021-10-22 19:07:15 +08:00
m . channelMu . Unlock ( )
return col2SegmentChangeInfos , nil
2021-06-15 12:41:40 +08:00
}
2021-10-22 19:07:15 +08:00
func ( m * MetaReplica ) removeGlobalSealedSegInfos ( collectionID UniqueID , partitionIDs [ ] UniqueID ) ( col2SealedSegmentChangeInfos , error ) {
removes := m . showSegmentInfos ( collectionID , partitionIDs )
if len ( removes ) == 0 {
return nil , nil
}
// get segmentInfos to remove
2021-10-26 15:18:22 +08:00
segmentChangeInfos := & querypb . SealedSegmentsChangeInfo {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_SealedSegmentsChangeInfo ,
} ,
Infos : [ ] * querypb . SegmentChangeInfo { } ,
}
2021-10-22 19:07:15 +08:00
for _ , info := range removes {
offlineNodeID := info . NodeID
2021-10-26 15:18:22 +08:00
changeInfo := & querypb . SegmentChangeInfo {
2021-10-22 19:07:15 +08:00
OfflineNodeID : offlineNodeID ,
OfflineSegments : [ ] * querypb . SegmentInfo { info } ,
}
2021-10-26 15:18:22 +08:00
segmentChangeInfos . Infos = append ( segmentChangeInfos . Infos , changeInfo )
2021-10-22 19:07:15 +08:00
}
2021-06-15 12:41:40 +08:00
2021-10-22 19:07:15 +08:00
// get msgStream to produce sealedSegmentChangeInfos to query channel
queryChannelInfo , messageIDInfos , err := m . sendSealedSegmentChangeInfos ( collectionID , segmentChangeInfos )
2021-06-19 11:45:09 +08:00
if err != nil {
2021-10-22 19:07:15 +08:00
return nil , err
2021-06-19 11:45:09 +08:00
}
2021-10-26 15:18:22 +08:00
// len(messageIDs) = 1
2021-10-22 19:07:15 +08:00
messageIDs , ok := messageIDInfos [ queryChannelInfo . QueryChannelID ]
if ! ok || len ( messageIDs ) == 0 {
return col2SealedSegmentChangeInfos { collectionID : segmentChangeInfos } , errors . New ( "updateGlobalSealedSegmentInfos: send sealed segment change info failed" )
}
2021-10-26 20:25:05 +08:00
if queryChannelInfo . SeekPosition == nil {
queryChannelInfo . SeekPosition = & internalpb . MsgPosition {
ChannelName : queryChannelInfo . QueryChannelID ,
}
}
2021-10-26 15:18:22 +08:00
queryChannelInfo . SeekPosition . MsgID = messageIDs [ 0 ] . Serialize ( )
2021-10-22 19:07:15 +08:00
// 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
2021-10-29 18:04:49 +08:00
// remove meta from etcd
for _ , info := range removes {
2021-11-17 12:13:10 +08:00
segmentKey := fmt . Sprintf ( "%s/%d" , util . SegmentMetaPrefix , info . SegmentID )
2021-10-29 18:04:49 +08:00
err = m . client . Remove ( segmentKey )
if err != nil {
panic ( err )
}
}
2021-10-22 19:07:15 +08:00
// 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 )
2021-10-27 23:00:24 +08:00
seekPos , err := proto . Marshal ( queryChannelInfo . SeekPosition )
if err != nil {
return col2SealedSegmentChangeInfos { collectionID : segmentChangeInfos } , err
}
saveKvs [ globalQuerySeekPositionPrefix ] = string ( seekPos )
2021-10-22 19:07:15 +08:00
// 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
2021-10-26 15:18:22 +08:00
changeInfoBytes , err := proto . Marshal ( segmentChangeInfos )
if err != nil {
return col2SealedSegmentChangeInfos { collectionID : segmentChangeInfos } , err
2021-10-22 19:07:15 +08:00
}
2021-10-26 15:18:22 +08:00
// TODO:: segmentChangeInfo clear in etcd with coord gc and queryNode watch the changeInfo meta to deal changeInfoMsg
2021-11-17 12:13:10 +08:00
changeInfoKey := fmt . Sprintf ( "%s/%d" , util . ChangeInfoMetaPrefix , segmentChangeInfos . Base . MsgID )
2021-10-26 15:18:22 +08:00
saveKvs [ changeInfoKey ] = string ( changeInfoBytes )
2021-10-22 19:07:15 +08:00
2021-10-29 18:04:49 +08:00
err = m . client . MultiSave ( saveKvs )
2021-10-22 19:07:15 +08:00
if err != nil {
2021-10-29 18:04:49 +08:00
panic ( err )
2021-10-22 19:07:15 +08:00
}
2021-10-29 18:04:49 +08:00
2021-10-22 19:07:15 +08:00
m . segmentMu . Lock ( )
for _ , info := range removes {
delete ( m . segmentInfos , info . SegmentID )
}
m . segmentMu . Unlock ( )
m . channelMu . Lock ( )
m . queryChannelInfos [ collectionID ] = queryChannelInfo
2021-10-27 23:00:24 +08:00
m . globalSeekPosition = queryChannelInfo . SeekPosition
2021-10-22 19:07:15 +08:00
m . channelMu . Unlock ( )
return col2SealedSegmentChangeInfos { collectionID : segmentChangeInfos } , nil
}
2021-11-17 12:13:10 +08:00
// send sealed segment change infos into query channels
2021-10-26 15:18:22 +08:00
func ( m * MetaReplica ) sendSealedSegmentChangeInfos ( collectionID UniqueID , changeInfos * querypb . SealedSegmentsChangeInfo ) ( * querypb . QueryChannelInfo , map [ string ] [ ] mqclient . MessageID , error ) {
2021-10-22 19:07:15 +08:00
// 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 { } ,
}
2021-10-26 15:18:22 +08:00
id , err := m . idAllocator ( )
if err != nil {
log . Error ( "allocator trigger taskID failed" , zap . Error ( err ) )
return nil , nil , err
}
changeInfos . Base . MsgID = id
segmentChangeMsg := & msgstream . SealedSegmentsChangeInfoMsg {
BaseMsg : msgstream . BaseMsg {
HashValues : [ ] uint32 { 0 } ,
} ,
SealedSegmentsChangeInfo : * changeInfos ,
2021-10-22 19:07:15 +08:00
}
2021-10-26 15:18:22 +08:00
msgPack . Msgs = append ( msgPack . Msgs , segmentChangeMsg )
2021-10-22 19:07:15 +08:00
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
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) showSegmentInfos ( collectionID UniqueID , partitionIDs [ ] UniqueID ) [ ] * querypb . SegmentInfo {
2021-10-22 19:07:15 +08:00
m . segmentMu . RLock ( )
defer m . segmentMu . RUnlock ( )
2021-06-19 11:45:09 +08:00
2021-08-02 22:39:25 +08:00
results := make ( [ ] * querypb . SegmentInfo , 0 )
2021-06-15 12:41:40 +08:00
segmentInfos := make ( [ ] * querypb . SegmentInfo , 0 )
2021-08-02 22:39:25 +08:00
for _ , info := range m . segmentInfos {
if info . CollectionID == collectionID {
2021-06-19 11:45:09 +08:00
segmentInfos = append ( segmentInfos , proto . Clone ( info ) . ( * querypb . SegmentInfo ) )
2021-06-15 12:41:40 +08:00
}
}
2021-08-02 22:39:25 +08:00
if len ( partitionIDs ) == 0 {
return segmentInfos
}
for _ , info := range segmentInfos {
for _ , partitionID := range partitionIDs {
if info . PartitionID == partitionID {
results = append ( results , info )
}
}
}
return results
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) getSegmentInfoByID ( segmentID UniqueID ) ( * querypb . SegmentInfo , error ) {
2021-10-22 19:07:15 +08:00
m . segmentMu . RLock ( )
defer m . segmentMu . RUnlock ( )
2021-06-15 12:41:40 +08:00
if info , ok := m . segmentInfos [ segmentID ] ; ok {
2021-06-19 11:45:09 +08:00
return proto . Clone ( info ) . ( * querypb . SegmentInfo ) , nil
2021-06-15 12:41:40 +08:00
}
return nil , errors . New ( "getSegmentInfoByID: can't find segmentID in segmentInfos" )
}
2021-11-06 15:22:56 +08:00
func ( m * MetaReplica ) getSegmentInfosByNode ( nodeID int64 ) [ ] * querypb . SegmentInfo {
m . segmentMu . RLock ( )
defer m . segmentMu . RUnlock ( )
segmentInfos := make ( [ ] * querypb . SegmentInfo , 0 )
for _ , info := range m . segmentInfos {
if info . NodeID == nodeID {
segmentInfos = append ( segmentInfos , proto . Clone ( info ) . ( * querypb . SegmentInfo ) )
}
}
return segmentInfos
}
2021-06-15 12:41:40 +08:00
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) getCollectionInfoByID ( collectionID UniqueID ) ( * querypb . CollectionInfo , error ) {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-19 11:45:09 +08:00
if info , ok := m . collectionInfos [ collectionID ] ; ok {
return proto . Clone ( info ) . ( * querypb . CollectionInfo ) , nil
}
return nil , errors . New ( "getCollectionInfoByID: can't find collectionID in collectionInfo" )
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) getPartitionStatesByID ( collectionID UniqueID , partitionID UniqueID ) ( * querypb . PartitionStates , error ) {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-15 12:41:40 +08:00
2021-08-02 22:39:25 +08:00
if info , ok := m . collectionInfos [ collectionID ] ; ok {
for offset , id := range info . PartitionIDs {
if id == partitionID {
return proto . Clone ( info . PartitionStates [ offset ] ) . ( * querypb . PartitionStates ) , nil
}
}
return nil , errors . New ( "getPartitionStateByID: can't find partitionID in partitionStates" )
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
return nil , errors . New ( "getPartitionStateByID: can't find collectionID in collectionInfo" )
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) releaseCollection ( collectionID UniqueID ) error {
2021-10-22 19:07:15 +08:00
err := removeGlobalCollectionInfo ( collectionID , m . client )
2021-06-19 11:45:09 +08:00
if err != nil {
2021-10-22 19:07:15 +08:00
log . Warn ( "remove collectionInfo from etcd failed" , zap . Any ( "error" , err . Error ( ) ) , zap . Int64 ( "collectionID" , collectionID ) )
return err
2021-06-19 11:45:09 +08:00
}
2021-08-02 22:39:25 +08:00
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
delete ( m . collectionInfos , collectionID )
m . collectionMu . Unlock ( )
return nil
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) releasePartition ( collectionID UniqueID , partitionID UniqueID ) error {
2021-10-22 19:07:15 +08:00
info , err := m . getCollectionInfoByID ( collectionID )
if err == nil {
2021-06-15 12:41:40 +08:00
newPartitionIDs := make ( [ ] UniqueID , 0 )
2021-08-02 22:39:25 +08:00
newPartitionStates := make ( [ ] * querypb . PartitionStates , 0 )
for offset , id := range info . PartitionIDs {
if id != partitionID {
2021-06-15 12:41:40 +08:00
newPartitionIDs = append ( newPartitionIDs , id )
2021-08-02 22:39:25 +08:00
newPartitionStates = append ( newPartitionStates , info . PartitionStates [ offset ] )
2021-06-15 12:41:40 +08:00
}
}
info . PartitionIDs = newPartitionIDs
2021-08-02 22:39:25 +08:00
info . PartitionStates = newPartitionStates
2021-06-23 17:44:12 +08:00
releasedPartitionIDs := make ( [ ] UniqueID , 0 )
for _ , id := range info . ReleasedPartitionIDs {
if id != partitionID {
releasedPartitionIDs = append ( releasedPartitionIDs , id )
}
}
releasedPartitionIDs = append ( releasedPartitionIDs , partitionID )
info . ReleasedPartitionIDs = releasedPartitionIDs
2021-08-27 16:51:11 +08:00
// If user loaded a collectionA, and release a partitionB which belongs to collectionA,
// and then load collectionA again, if we don't set the inMemoryPercentage to 0 when releasing
// partitionB, the second loading of collectionA would directly return because
// the inMemoryPercentage in ShowCollection response is still the old value -- 100.
// So if releasing partition, inMemoryPercentage should be set to 0.
info . InMemoryPercentage = 0
2021-10-22 19:07:15 +08:00
err = saveGlobalCollectionInfo ( collectionID , info , m . client )
2021-06-23 17:44:12 +08:00
if err != nil {
2021-10-22 19:07:15 +08:00
log . Error ( "releasePartition: save collectionInfo error" , zap . Any ( "error" , err . Error ( ) ) , zap . Int64 ( "collectionID" , collectionID ) , zap . Int64 ( "partitionID" , partitionID ) )
2021-08-02 22:39:25 +08:00
return err
2021-06-23 17:44:12 +08:00
}
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
m . collectionInfos [ collectionID ] = info
m . collectionMu . Unlock ( )
return nil
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
2021-10-22 19:07:15 +08:00
return err
2021-06-15 12:41:40 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) getDmChannelsByNodeID ( collectionID UniqueID , nodeID int64 ) ( [ ] string , error ) {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-15 12:41:40 +08:00
if info , ok := m . collectionInfos [ collectionID ] ; ok {
channels := make ( [ ] string , 0 )
for _ , channelInfo := range info . ChannelInfos {
if channelInfo . NodeIDLoaded == nodeID {
channels = append ( channels , channelInfo . ChannelIDs ... )
}
}
return channels , nil
}
return nil , errors . New ( "getDmChannelsByNodeID: can't find collection in collectionInfos" )
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) addDmChannel ( collectionID UniqueID , nodeID int64 , channels [ ] string ) error {
//before add channel, should ensure toAddedChannels not in MetaReplica
2021-10-22 19:07:15 +08:00
info , err := m . getCollectionInfoByID ( collectionID )
if err == nil {
2021-06-15 12:41:40 +08:00
findNodeID := false
for _ , channelInfo := range info . ChannelInfos {
if channelInfo . NodeIDLoaded == nodeID {
findNodeID = true
channelInfo . ChannelIDs = append ( channelInfo . ChannelIDs , channels ... )
}
}
if ! findNodeID {
newChannelInfo := & querypb . DmChannelInfo {
NodeIDLoaded : nodeID ,
ChannelIDs : channels ,
}
info . ChannelInfos = append ( info . ChannelInfos , newChannelInfo )
}
2021-06-19 11:45:09 +08:00
2021-10-22 19:07:15 +08:00
err = saveGlobalCollectionInfo ( collectionID , info , m . client )
2021-06-19 11:45:09 +08:00
if err != nil {
log . Error ( "save collectionInfo error" , zap . Any ( "error" , err . Error ( ) ) , zap . Int64 ( "collectionID" , collectionID ) )
2021-08-02 22:39:25 +08:00
return err
2021-06-19 11:45:09 +08:00
}
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
m . collectionInfos [ collectionID ] = info
m . collectionMu . Unlock ( )
2021-08-02 22:39:25 +08:00
return nil
2021-06-19 11:45:09 +08:00
}
return errors . New ( "addDmChannels: can't find collection in collectionInfos" )
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) removeDmChannel ( collectionID UniqueID , nodeID int64 , channels [ ] string ) error {
2021-10-22 19:07:15 +08:00
info , err := m . getCollectionInfoByID ( collectionID )
if err == nil {
2021-06-19 11:45:09 +08:00
for _ , channelInfo := range info . ChannelInfos {
if channelInfo . NodeIDLoaded == nodeID {
newChannelIDs := make ( [ ] string , 0 )
for _ , channelID := range channelInfo . ChannelIDs {
findChannel := false
for _ , channel := range channels {
if channelID == channel {
findChannel = true
}
}
if ! findChannel {
newChannelIDs = append ( newChannelIDs , channelID )
}
}
channelInfo . ChannelIDs = newChannelIDs
}
}
2021-08-02 22:39:25 +08:00
err := saveGlobalCollectionInfo ( collectionID , info , m . client )
2021-06-19 11:45:09 +08:00
if err != nil {
log . Error ( "save collectionInfo error" , zap . Any ( "error" , err . Error ( ) ) , zap . Int64 ( "collectionID" , collectionID ) )
2021-08-02 22:39:25 +08:00
return err
2021-06-19 11:45:09 +08:00
}
2021-08-02 22:39:25 +08:00
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
m . collectionInfos [ collectionID ] = info
m . collectionMu . Unlock ( )
2021-08-02 22:39:25 +08:00
return nil
2021-06-15 12:41:40 +08:00
}
return errors . New ( "addDmChannels: can't find collection in collectionInfos" )
}
2021-10-22 19:07:15 +08:00
func createQueryChannel ( collectionID UniqueID ) * querypb . QueryChannelInfo {
2021-06-15 12:41:40 +08:00
searchPrefix := Params . SearchChannelPrefix
searchResultPrefix := Params . SearchResultChannelPrefix
allocatedQueryChannel := searchPrefix + "-" + strconv . FormatInt ( collectionID , 10 )
allocatedQueryResultChannel := searchResultPrefix + "-" + strconv . FormatInt ( collectionID , 10 )
2021-06-22 16:44:09 +08:00
log . Debug ( "query coordinator create query channel" , zap . String ( "queryChannelName" , allocatedQueryChannel ) , zap . String ( "queryResultChannelName" , allocatedQueryResultChannel ) )
2021-06-15 12:41:40 +08:00
2021-10-22 19:07:15 +08:00
seekPosition := & internalpb . MsgPosition {
ChannelName : allocatedQueryChannel ,
}
info := & querypb . QueryChannelInfo {
2021-06-15 12:41:40 +08:00
CollectionID : collectionID ,
QueryChannelID : allocatedQueryChannel ,
QueryResultChannelID : allocatedQueryResultChannel ,
2021-10-22 19:07:15 +08:00
GlobalSealedSegments : [ ] * querypb . SegmentInfo { } ,
SeekPosition : seekPosition ,
}
return info
}
2021-11-13 08:49:08 +08:00
// Get delta channel info for collection, so far all the collection share the same query channel 0
func ( m * MetaReplica ) getDeltaChannelsByCollectionID ( collectionID UniqueID ) ( [ ] * datapb . VchannelInfo , error ) {
m . deltaChannelMu . RLock ( )
defer m . deltaChannelMu . RUnlock ( )
if infos , ok := m . deltaChannelInfos [ collectionID ] ; ok {
return infos , nil
}
return nil , fmt . Errorf ( "delta channel not exist in meta" )
}
func ( m * MetaReplica ) setDeltaChannel ( collectionID UniqueID , infos [ ] * datapb . VchannelInfo ) error {
m . deltaChannelMu . Lock ( )
defer m . deltaChannelMu . Unlock ( )
_ , ok := m . deltaChannelInfos [ collectionID ]
if ok {
2021-12-01 16:39:31 +08:00
log . Debug ( "delta channel already exist" , zap . Any ( "collectionID" , collectionID ) )
2021-11-13 08:49:08 +08:00
return nil
}
err := saveDeltaChannelInfo ( collectionID , infos , m . client )
if err != nil {
2021-12-01 16:39:31 +08:00
log . Error ( "save delta channel info error" , zap . Error ( err ) )
2021-11-13 08:49:08 +08:00
return err
}
2021-12-01 16:39:31 +08:00
log . Debug ( "save delta channel infos to meta" , zap . Any ( "collectionID" , collectionID ) , zap . Any ( "infos" , infos ) )
2021-11-13 08:49:08 +08:00
m . deltaChannelInfos [ collectionID ] = infos
return nil
}
2021-11-05 14:57:44 +08:00
// Get Query channel info for collection, so far all the collection share the same query channel 0
2021-10-22 19:07:15 +08:00
func ( m * MetaReplica ) getQueryChannelInfoByID ( collectionID UniqueID ) ( * querypb . QueryChannelInfo , error ) {
m . channelMu . Lock ( )
defer m . channelMu . Unlock ( )
if info , ok := m . queryChannelInfos [ collectionID ] ; ok {
return proto . Clone ( info ) . ( * querypb . QueryChannelInfo ) , nil
2021-06-15 12:41:40 +08:00
}
2021-10-22 19:07:15 +08:00
2021-10-26 19:24:23 +08:00
// TODO::to remove
// all collection use the same query channel
colIDForAssignChannel := UniqueID ( 0 )
info := createQueryChannel ( colIDForAssignChannel )
2021-10-22 19:07:15 +08:00
err := saveQueryChannelInfo ( collectionID , info , m . client )
2021-09-29 09:56:04 +08:00
if err != nil {
2021-10-13 21:46:32 +08:00
log . Error ( "getQueryChannel: save channel to etcd error" , zap . Error ( err ) )
2021-10-22 19:07:15 +08:00
return nil , err
2021-09-29 09:56:04 +08:00
}
2021-10-26 19:24:23 +08:00
// set info.collectionID from 0 to realID
info . CollectionID = collectionID
2021-10-22 19:07:15 +08:00
m . queryChannelInfos [ collectionID ] = info
2021-10-27 23:00:24 +08:00
info . SeekPosition = m . globalSeekPosition
if info . SeekPosition != nil {
info . SeekPosition . ChannelName = info . QueryChannelID
}
2021-10-22 19:07:15 +08:00
return proto . Clone ( info ) . ( * querypb . QueryChannelInfo ) , nil
2021-06-15 12:41:40 +08:00
}
2021-06-19 11:45:09 +08:00
2021-10-22 19:07:15 +08:00
func ( m * MetaReplica ) getQueryStreamByID ( collectionID UniqueID ) ( msgstream . MsgStream , error ) {
m . streamMu . Lock ( )
defer m . streamMu . Unlock ( )
2021-06-19 11:45:09 +08:00
2021-10-22 19:07:15 +08:00
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 {
info , err := m . getCollectionInfoByID ( collectionID )
if err == nil {
2021-08-02 22:39:25 +08:00
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
}
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
m . collectionInfos [ collectionID ] = info
m . collectionMu . Unlock ( )
2021-08-02 22:39:25 +08:00
return nil
}
2021-06-19 11:45:09 +08:00
2021-08-02 22:39:25 +08:00
return errors . New ( "setLoadType: can't find collection in collectionInfos" )
2021-06-19 11:45:09 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) getLoadType ( collectionID UniqueID ) ( querypb . LoadType , error ) {
2021-10-22 19:07:15 +08:00
m . collectionMu . RLock ( )
defer m . collectionMu . RUnlock ( )
2021-06-19 11:45:09 +08:00
2021-08-02 22:39:25 +08:00
if info , ok := m . collectionInfos [ collectionID ] ; ok {
return info . LoadType , nil
}
2021-06-19 11:45:09 +08:00
2021-08-02 22:39:25 +08:00
return 0 , errors . New ( "getLoadType: can't find collection in collectionInfos" )
2021-06-19 11:45:09 +08:00
}
2021-08-02 22:39:25 +08:00
func ( m * MetaReplica ) setLoadPercentage ( collectionID UniqueID , partitionID UniqueID , percentage int64 , loadType querypb . LoadType ) error {
2021-10-22 19:07:15 +08:00
info , err := m . getCollectionInfoByID ( collectionID )
if err != nil {
2021-08-02 22:39:25 +08:00
return errors . New ( "setLoadPercentage: can't find collection in collectionInfos" )
}
if loadType == querypb . LoadType_loadCollection {
info . InMemoryPercentage = percentage
for _ , partitionState := range info . PartitionStates {
if percentage >= 100 {
partitionState . State = querypb . PartitionState_InMemory
} else {
partitionState . State = querypb . PartitionState_PartialInMemory
}
partitionState . InMemoryPercentage = percentage
}
err := saveGlobalCollectionInfo ( collectionID , info , m . client )
2021-06-19 11:45:09 +08:00
if err != nil {
log . Error ( "save collectionInfo error" , zap . Any ( "error" , err . Error ( ) ) , zap . Int64 ( "collectionID" , collectionID ) )
2021-08-02 22:39:25 +08:00
return err
2021-06-19 11:45:09 +08:00
}
2021-08-02 22:39:25 +08:00
} else {
2021-10-22 19:07:15 +08:00
findPartition := false
2021-08-02 22:39:25 +08:00
for _ , partitionState := range info . PartitionStates {
if partitionState . PartitionID == partitionID {
2021-10-22 19:07:15 +08:00
findPartition = true
2021-08-02 22:39:25 +08:00
if percentage >= 100 {
partitionState . State = querypb . PartitionState_InMemory
} else {
partitionState . State = querypb . PartitionState_PartialInMemory
}
partitionState . InMemoryPercentage = percentage
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
}
}
}
2021-10-22 19:07:15 +08:00
if ! findPartition {
return errors . New ( "setLoadPercentage: can't find partitionID in collectionInfos" )
}
2021-06-19 11:45:09 +08:00
}
2021-10-22 19:07:15 +08:00
m . collectionMu . Lock ( )
m . collectionInfos [ collectionID ] = info
m . collectionMu . Unlock ( )
2021-08-02 22:39:25 +08:00
return nil
2021-06-19 11:45:09 +08:00
}
2021-09-29 09:56:04 +08:00
//func (m *MetaReplica) printMeta() {
// m.RLock()
// defer m.RUnlock()
// for id, info := range m.collectionInfos {
// log.Debug("query coordinator MetaReplica: collectionInfo", zap.Int64("collectionID", id), zap.Any("info", info))
// }
//
// for id, info := range m.segmentInfos {
// log.Debug("query coordinator MetaReplica: segmentInfo", zap.Int64("segmentID", id), zap.Any("info", info))
// }
//
// for id, info := range m.queryChannelInfos {
// log.Debug("query coordinator MetaReplica: queryChannelInfo", zap.Int64("collectionID", id), zap.Any("info", info))
// }
//}
2021-08-02 22:39:25 +08:00
2021-09-29 09:56:04 +08:00
func saveGlobalCollectionInfo ( collectionID UniqueID , info * querypb . CollectionInfo , kv kv . MetaKv ) error {
2021-09-29 20:26:00 +08:00
infoBytes , err := proto . Marshal ( info )
if err != nil {
return err
}
2021-08-02 22:39:25 +08:00
key := fmt . Sprintf ( "%s/%d" , collectionMetaPrefix , collectionID )
2021-09-29 20:26:00 +08:00
return kv . Save ( key , string ( infoBytes ) )
2021-08-02 22:39:25 +08:00
}
2021-09-29 09:56:04 +08:00
func removeGlobalCollectionInfo ( collectionID UniqueID , kv kv . MetaKv ) error {
2021-08-02 22:39:25 +08:00
key := fmt . Sprintf ( "%s/%d" , collectionMetaPrefix , collectionID )
return kv . Remove ( key )
}
2021-10-22 19:07:15 +08:00
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
}
2021-11-17 12:13:10 +08:00
key := fmt . Sprintf ( "%s/%d" , util . SegmentMetaPrefix , segmentID )
2021-10-22 19:07:15 +08:00
kvs [ key ] = string ( infoBytes )
2021-09-29 20:26:00 +08:00
}
2021-08-02 22:39:25 +08:00
2021-10-22 19:07:15 +08:00
return kv . MultiSave ( kvs )
2021-08-02 22:39:25 +08:00
}
2021-10-22 19:07:15 +08:00
func multiRemoveSegmentInfo ( segmentIDs [ ] UniqueID , kv kv . MetaKv ) error {
keys := make ( [ ] string , 0 )
for _ , segmentID := range segmentIDs {
2021-11-17 12:13:10 +08:00
key := fmt . Sprintf ( "%s/%d" , util . SegmentMetaPrefix , segmentID )
2021-10-22 19:07:15 +08:00
keys = append ( keys , key )
}
return kv . MultiRemove ( keys )
2021-08-02 22:39:25 +08:00
}
2021-09-29 09:56:04 +08:00
func saveQueryChannelInfo ( collectionID UniqueID , info * querypb . QueryChannelInfo , kv kv . MetaKv ) error {
2021-09-29 20:26:00 +08:00
infoBytes , err := proto . Marshal ( info )
if err != nil {
return err
}
2021-08-02 22:39:25 +08:00
key := fmt . Sprintf ( "%s/%d" , queryChannelMetaPrefix , collectionID )
2021-09-29 20:26:00 +08:00
return kv . Save ( key , string ( infoBytes ) )
2021-08-02 22:39:25 +08:00
}
2021-11-13 08:49:08 +08:00
func saveDeltaChannelInfo ( collectionID UniqueID , infos [ ] * datapb . VchannelInfo , kv kv . MetaKv ) error {
kvs := make ( map [ string ] string )
for _ , info := range infos {
infoBytes , err := proto . Marshal ( info )
if err != nil {
return err
}
key := fmt . Sprintf ( "%s/%d/%s" , deltaChannelMetaPrefix , collectionID , info . ChannelName )
kvs [ key ] = string ( infoBytes )
}
return kv . MultiSave ( kvs )
}