2021-11-29 20:13: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-04-19 13:47:10 +08:00
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
2021-11-29 20:13:50 +08:00
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
2021-04-19 13:47:10 +08:00
|
|
|
//
|
2021-11-29 20:13: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-04-19 13:47:10 +08:00
|
|
|
|
2021-01-16 10:12:14 +08:00
|
|
|
package querynode
|
2020-11-09 16:27:11 +08:00
|
|
|
|
|
|
|
/*
|
2022-06-24 21:12:15 +08:00
|
|
|
#cgo pkg-config: milvus_segcore
|
2020-11-09 16:27:11 +08:00
|
|
|
|
2020-11-25 10:31:51 +08:00
|
|
|
#include "segcore/collection_c.h"
|
|
|
|
#include "segcore/segment_c.h"
|
2020-11-09 16:27:11 +08:00
|
|
|
*/
|
|
|
|
import "C"
|
|
|
|
import (
|
2021-06-09 11:37:55 +08:00
|
|
|
"errors"
|
2021-03-05 10:15:27 +08:00
|
|
|
"fmt"
|
2020-11-12 12:04:12 +08:00
|
|
|
"strconv"
|
2020-11-13 17:20:13 +08:00
|
|
|
"sync"
|
2020-11-12 12:04:12 +08:00
|
|
|
|
2021-03-05 09:21:35 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2021-10-29 17:12:43 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/common"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
2022-03-02 14:49:55 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/metrics"
|
2021-10-29 17:12:43 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2021-11-05 16:00:55 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
2020-11-09 16:27:11 +08:00
|
|
|
)
|
|
|
|
|
2021-10-22 23:57:11 +08:00
|
|
|
// ReplicaInterface specifies all the methods that the Collection object needs to implement in QueryNode.
|
|
|
|
// In common cases, the system has multiple query nodes. The full data of a collection will be distributed
|
|
|
|
// across multiple query nodes, and each query node's collectionReplica will maintain its own part.
|
2021-03-05 16:52:45 +08:00
|
|
|
type ReplicaInterface interface {
|
2020-11-13 17:20:13 +08:00
|
|
|
// collection
|
2021-10-25 19:37:55 +08:00
|
|
|
// getCollectionIDs returns all collection ids in the collectionReplica
|
2021-02-06 11:35:35 +08:00
|
|
|
getCollectionIDs() []UniqueID
|
2021-10-28 09:44:20 +08:00
|
|
|
// addCollection creates a new collection and add it to collectionReplica
|
2022-01-20 10:01:38 +08:00
|
|
|
addCollection(collectionID UniqueID, schema *schemapb.CollectionSchema) *Collection
|
2021-10-27 22:22:21 +08:00
|
|
|
// removeCollection removes the collection from collectionReplica
|
2020-11-13 17:20:13 +08:00
|
|
|
removeCollection(collectionID UniqueID) error
|
2021-10-27 22:16:24 +08:00
|
|
|
// getCollectionByID gets the collection which id is collectionID
|
2020-11-13 17:20:13 +08:00
|
|
|
getCollectionByID(collectionID UniqueID) (*Collection, error)
|
2021-10-29 17:53:17 +08:00
|
|
|
// hasCollection checks if collectionReplica has the collection which id is collectionID
|
2020-11-25 10:31:51 +08:00
|
|
|
hasCollection(collectionID UniqueID) bool
|
2021-10-29 17:55:04 +08:00
|
|
|
// getCollectionNum returns num of collections in collectionReplica
|
2021-02-05 10:53:11 +08:00
|
|
|
getCollectionNum() int
|
2021-10-29 17:56:55 +08:00
|
|
|
// getPartitionIDs returns partition ids of collection
|
2021-02-05 10:53:11 +08:00
|
|
|
getPartitionIDs(collectionID UniqueID) ([]UniqueID, error)
|
2021-11-01 09:02:04 +08:00
|
|
|
// getVecFieldIDsByCollectionID returns vector field ids of collection
|
2021-09-14 10:25:26 +08:00
|
|
|
getVecFieldIDsByCollectionID(collectionID UniqueID) ([]FieldID, error)
|
2021-11-01 09:04:06 +08:00
|
|
|
// getPKFieldIDsByCollectionID returns vector field ids of collection
|
2021-10-22 14:31:13 +08:00
|
|
|
getPKFieldIDByCollectionID(collectionID UniqueID) (FieldID, error)
|
2021-11-05 16:00:55 +08:00
|
|
|
// getSegmentInfosByColID return segments info by collectionID
|
2022-05-31 13:42:03 +08:00
|
|
|
getSegmentInfosByColID(collectionID UniqueID) []*querypb.SegmentInfo
|
2020-11-13 17:20:13 +08:00
|
|
|
|
|
|
|
// partition
|
2021-11-01 09:05:55 +08:00
|
|
|
// addPartition adds a new partition to collection
|
2021-01-21 15:20:23 +08:00
|
|
|
addPartition(collectionID UniqueID, partitionID UniqueID) error
|
2021-11-01 22:37:50 +08:00
|
|
|
// removePartition removes the partition from collectionReplica
|
2021-02-05 10:53:11 +08:00
|
|
|
removePartition(partitionID UniqueID) error
|
2021-11-01 22:40:01 +08:00
|
|
|
// getPartitionByID returns the partition which id is partitionID
|
2021-02-05 10:53:11 +08:00
|
|
|
getPartitionByID(partitionID UniqueID) (*Partition, error)
|
2021-11-01 22:41:59 +08:00
|
|
|
// hasPartition returns true if collectionReplica has the partition, false otherwise
|
2021-02-05 10:53:11 +08:00
|
|
|
hasPartition(partitionID UniqueID) bool
|
2021-11-02 21:50:42 +08:00
|
|
|
// getPartitionNum returns num of partitions
|
2021-02-05 10:53:11 +08:00
|
|
|
getPartitionNum() int
|
2021-11-02 21:52:51 +08:00
|
|
|
// getSegmentIDs returns segment ids
|
2022-05-31 13:42:03 +08:00
|
|
|
getSegmentIDs(partitionID UniqueID, segType segmentType) ([]UniqueID, error)
|
2021-11-02 21:54:48 +08:00
|
|
|
// getSegmentIDsByVChannel returns segment ids which virtual channel is vChannel
|
2021-06-15 20:06:10 +08:00
|
|
|
getSegmentIDsByVChannel(partitionID UniqueID, vChannel Channel) ([]UniqueID, error)
|
2020-11-13 17:20:13 +08:00
|
|
|
|
|
|
|
// segment
|
2021-11-03 23:17:02 +08:00
|
|
|
// addSegment add a new segment to collectionReplica
|
2022-05-23 16:41:58 +08:00
|
|
|
addSegment(segmentID UniqueID, partitionID UniqueID, collectionID UniqueID, vChannelID Channel, segType segmentType) error
|
2021-11-03 23:18:57 +08:00
|
|
|
// setSegment adds a segment to collectionReplica
|
2021-04-07 18:29:19 +08:00
|
|
|
setSegment(segment *Segment) error
|
2021-11-03 23:09:06 +08:00
|
|
|
// removeSegment removes a segment from collectionReplica
|
2022-06-01 13:18:02 +08:00
|
|
|
removeSegment(segmentID UniqueID, segType segmentType)
|
2021-11-04 18:56:42 +08:00
|
|
|
// getSegmentByID returns the segment which id is segmentID
|
2022-05-31 13:42:03 +08:00
|
|
|
getSegmentByID(segmentID UniqueID, segType segmentType) (*Segment, error)
|
2021-11-04 18:59:29 +08:00
|
|
|
// hasSegment returns true if collectionReplica has the segment, false otherwise
|
2022-05-31 13:42:03 +08:00
|
|
|
hasSegment(segmentID UniqueID, segType segmentType) (bool, error)
|
2021-11-04 19:02:04 +08:00
|
|
|
// getSegmentNum returns num of segments in collectionReplica
|
2022-05-31 13:42:03 +08:00
|
|
|
getSegmentNum(segType segmentType) int
|
2021-11-05 19:53:54 +08:00
|
|
|
// getSegmentStatistics returns the statistics of segments in collectionReplica
|
2021-03-12 14:22:09 +08:00
|
|
|
getSegmentStatistics() []*internalpb.SegmentStats
|
2021-03-12 19:23:06 +08:00
|
|
|
|
2021-03-22 16:36:10 +08:00
|
|
|
// excluded segments
|
2021-11-05 19:55:45 +08:00
|
|
|
// removeExcludedSegments will remove excludedSegments from collectionReplica
|
2021-03-22 16:36:10 +08:00
|
|
|
removeExcludedSegments(collectionID UniqueID)
|
2021-11-05 19:57:35 +08:00
|
|
|
// addExcludedSegments will add excludedSegments to collectionReplica
|
2021-10-16 17:00:33 +08:00
|
|
|
addExcludedSegments(collectionID UniqueID, segmentInfos []*datapb.SegmentInfo)
|
2021-11-06 19:55:18 +08:00
|
|
|
// getExcludedSegments returns excludedSegments of collectionReplica
|
2021-06-16 11:09:56 +08:00
|
|
|
getExcludedSegments(collectionID UniqueID) ([]*datapb.SegmentInfo, error)
|
2021-03-22 16:36:10 +08:00
|
|
|
|
2021-11-08 21:07:17 +08:00
|
|
|
// getSegmentsMemSize get the memory size in bytes of all the Segments
|
2021-09-28 22:24:03 +08:00
|
|
|
getSegmentsMemSize() int64
|
2021-11-10 23:52:06 +08:00
|
|
|
// freeAll will free all meta info from collectionReplica
|
2020-11-24 16:12:39 +08:00
|
|
|
freeAll()
|
2021-11-10 23:54:05 +08:00
|
|
|
// printReplica prints the collections, partitions and segments in the collectionReplica
|
2021-06-22 14:10:09 +08:00
|
|
|
printReplica()
|
2020-11-13 17:20:13 +08:00
|
|
|
}
|
|
|
|
|
2021-09-18 18:15:51 +08:00
|
|
|
// collectionReplica is the data replication of memory data in query node.
|
|
|
|
// It implements `ReplicaInterface` interface.
|
2022-05-16 18:23:55 +08:00
|
|
|
type metaReplica struct {
|
2022-05-31 13:42:03 +08:00
|
|
|
mu sync.RWMutex // guards all
|
|
|
|
collections map[UniqueID]*Collection
|
|
|
|
partitions map[UniqueID]*Partition
|
|
|
|
growingSegments map[UniqueID]*Segment
|
|
|
|
sealedSegments map[UniqueID]*Segment
|
2021-03-22 16:36:10 +08:00
|
|
|
|
2021-06-16 11:09:56 +08:00
|
|
|
excludedSegments map[UniqueID][]*datapb.SegmentInfo // map[collectionID]segmentIDs
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-08 06:43:26 +08:00
|
|
|
// getSegmentsMemSize get the memory size in bytes of all the Segments
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getSegmentsMemSize() int64 {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2021-09-28 22:24:03 +08:00
|
|
|
|
|
|
|
memSize := int64(0)
|
2022-05-31 13:42:03 +08:00
|
|
|
for _, segment := range replica.growingSegments {
|
|
|
|
memSize += segment.getMemSize()
|
|
|
|
}
|
|
|
|
for _, segment := range replica.sealedSegments {
|
2021-09-28 22:24:03 +08:00
|
|
|
memSize += segment.getMemSize()
|
|
|
|
}
|
|
|
|
return memSize
|
|
|
|
}
|
|
|
|
|
2021-10-08 06:45:25 +08:00
|
|
|
// printReplica prints the collections, partitions and segments in the collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) printReplica() {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2021-06-22 14:10:09 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
log.Info("collections in collectionReplica", zap.Any("info", replica.collections))
|
|
|
|
log.Info("partitions in collectionReplica", zap.Any("info", replica.partitions))
|
2022-05-31 13:42:03 +08:00
|
|
|
log.Info("growingSegments in collectionReplica", zap.Any("info", replica.growingSegments))
|
|
|
|
log.Info("sealedSegments in collectionReplica", zap.Any("info", replica.sealedSegments))
|
2022-05-16 18:23:55 +08:00
|
|
|
log.Info("excludedSegments in collectionReplica", zap.Any("info", replica.excludedSegments))
|
2021-06-22 14:10:09 +08:00
|
|
|
}
|
|
|
|
|
2020-11-09 16:27:11 +08:00
|
|
|
//----------------------------------------------------------------------------------------------------- collection
|
2021-10-08 06:47:15 +08:00
|
|
|
// getCollectionIDs gets all the collection ids in the collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getCollectionIDs() []UniqueID {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2021-02-06 11:35:35 +08:00
|
|
|
collectionIDs := make([]UniqueID, 0)
|
2022-05-16 18:23:55 +08:00
|
|
|
for id := range replica.collections {
|
2021-02-06 11:35:35 +08:00
|
|
|
collectionIDs = append(collectionIDs, id)
|
|
|
|
}
|
|
|
|
return collectionIDs
|
|
|
|
}
|
|
|
|
|
2021-10-08 23:53:25 +08:00
|
|
|
// addCollection creates a new collection and add it to collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) addCollection(collectionID UniqueID, schema *schemapb.CollectionSchema) *Collection {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2020-11-13 17:20:13 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
if col, ok := replica.collections[collectionID]; ok {
|
2022-01-20 10:01:38 +08:00
|
|
|
return col
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
|
|
|
|
2022-05-23 16:41:58 +08:00
|
|
|
var newC = newCollection(collectionID, schema)
|
|
|
|
replica.collections[collectionID] = newC
|
2022-05-16 18:23:55 +08:00
|
|
|
metrics.QueryNodeNumCollections.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Set(float64(len(replica.collections)))
|
2022-05-23 16:41:58 +08:00
|
|
|
return newC
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-09 18:39:48 +08:00
|
|
|
// removeCollection removes the collection from collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) removeCollection(collectionID UniqueID) error {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
|
|
|
return replica.removeCollectionPrivate(collectionID)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
2020-11-13 17:20:13 +08:00
|
|
|
|
2021-10-09 18:41:32 +08:00
|
|
|
// removeCollectionPrivate is the private function in collectionReplica, to remove collection from collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) removeCollectionPrivate(collectionID UniqueID) error {
|
|
|
|
collection, err := replica.getCollectionByIDPrivate(collectionID)
|
2020-11-13 17:20:13 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
// block incoming search&query
|
|
|
|
collection.Lock()
|
|
|
|
defer collection.Unlock()
|
|
|
|
|
2021-02-05 10:53:11 +08:00
|
|
|
// delete partitions
|
|
|
|
for _, partitionID := range collection.partitionIDs {
|
|
|
|
// ignore error, try to delete
|
2022-05-16 18:23:55 +08:00
|
|
|
_ = replica.removePartitionPrivate(partitionID, true)
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-02-24 17:24:51 +08:00
|
|
|
deleteCollection(collection)
|
2022-05-16 18:23:55 +08:00
|
|
|
delete(replica.collections, collectionID)
|
2021-02-24 17:24:51 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
metrics.QueryNodeNumCollections.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Set(float64(len(replica.collections)))
|
2022-04-24 22:03:44 +08:00
|
|
|
metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Sub(float64(len(collection.partitionIDs)))
|
2020-11-09 16:27:11 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-09 18:33:32 +08:00
|
|
|
// getCollectionByID gets the collection which id is collectionID
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getCollectionByID(collectionID UniqueID) (*Collection, error) {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
|
|
|
return replica.getCollectionByIDPrivate(collectionID)
|
2021-01-13 10:40:46 +08:00
|
|
|
}
|
|
|
|
|
2021-10-11 19:02:45 +08:00
|
|
|
// getCollectionByIDPrivate is the private function in collectionReplica, to get collection from collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getCollectionByIDPrivate(collectionID UniqueID) (*Collection, error) {
|
|
|
|
collection, ok := replica.collections[collectionID]
|
2021-02-05 10:53:11 +08:00
|
|
|
if !ok {
|
2022-01-06 23:43:22 +08:00
|
|
|
return nil, fmt.Errorf("collection hasn't been loaded or has been released, collection id = %d", collectionID)
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-02-05 10:53:11 +08:00
|
|
|
return collection, nil
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-11 19:04:37 +08:00
|
|
|
// hasCollection checks if collectionReplica has the collection which id is collectionID
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) hasCollection(collectionID UniqueID) bool {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
|
|
|
return replica.hasCollectionPrivate(collectionID)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
2020-11-25 10:31:51 +08:00
|
|
|
|
2021-10-11 18:56:37 +08:00
|
|
|
// hasCollectionPrivate is the private function in collectionReplica, to check collection in collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) hasCollectionPrivate(collectionID UniqueID) bool {
|
|
|
|
_, ok := replica.collections[collectionID]
|
2021-02-05 10:53:11 +08:00
|
|
|
return ok
|
|
|
|
}
|
|
|
|
|
2021-10-12 19:05:17 +08:00
|
|
|
// getCollectionNum returns num of collections in collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getCollectionNum() int {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
|
|
|
return len(replica.collections)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-12 19:07:12 +08:00
|
|
|
// getPartitionIDs returns partition ids of collection
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getPartitionIDs(collectionID UniqueID) ([]UniqueID, error) {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2021-02-05 10:53:11 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
collection, err := replica.getCollectionByIDPrivate(collectionID)
|
2021-02-05 10:53:11 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
2020-11-25 10:31:51 +08:00
|
|
|
}
|
2021-02-05 10:53:11 +08:00
|
|
|
|
2022-05-31 13:42:03 +08:00
|
|
|
return collection.getPartitionIDs(), nil
|
2020-11-25 10:31:51 +08:00
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getIndexedFieldIDByCollectionIDPrivate(collectionID UniqueID, segment *Segment) ([]FieldID, error) {
|
|
|
|
fields, err := replica.getFieldsByCollectionIDPrivate(collectionID)
|
2022-03-30 21:11:28 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
fieldIDS := make([]FieldID, 0)
|
|
|
|
for _, field := range fields {
|
|
|
|
if segment.hasLoadIndexForIndexedField(field.FieldID) {
|
|
|
|
fieldIDS = append(fieldIDS, field.FieldID)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return fieldIDS, nil
|
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getVecFieldIDsByCollectionIDPrivate(collectionID UniqueID) ([]FieldID, error) {
|
|
|
|
fields, err := replica.getFieldsByCollectionIDPrivate(collectionID)
|
2021-02-02 19:54:31 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2021-09-14 10:25:26 +08:00
|
|
|
vecFields := make([]FieldID, 0)
|
2021-02-06 11:35:35 +08:00
|
|
|
for _, field := range fields {
|
2021-03-12 14:22:09 +08:00
|
|
|
if field.DataType == schemapb.DataType_BinaryVector || field.DataType == schemapb.DataType_FloatVector {
|
2021-02-03 11:52:19 +08:00
|
|
|
vecFields = append(vecFields, field.FieldID)
|
2021-02-02 19:54:31 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return vecFields, nil
|
|
|
|
}
|
|
|
|
|
2022-02-08 21:57:46 +08:00
|
|
|
// getVecFieldIDsByCollectionID returns vector field ids of collection
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getVecFieldIDsByCollectionID(collectionID UniqueID) ([]FieldID, error) {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2022-02-08 21:57:46 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
return replica.getVecFieldIDsByCollectionIDPrivate(collectionID)
|
2022-02-08 21:57:46 +08:00
|
|
|
}
|
|
|
|
|
2021-10-22 14:31:13 +08:00
|
|
|
// getPKFieldIDsByCollectionID returns vector field ids of collection
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getPKFieldIDByCollectionID(collectionID UniqueID) (FieldID, error) {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2021-10-22 14:31:13 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
fields, err := replica.getFieldsByCollectionIDPrivate(collectionID)
|
2021-10-22 14:31:13 +08:00
|
|
|
if err != nil {
|
|
|
|
return common.InvalidFieldID, err
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, field := range fields {
|
|
|
|
if field.IsPrimaryKey {
|
|
|
|
return field.FieldID, nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return common.InvalidFieldID, nil
|
|
|
|
}
|
|
|
|
|
2021-10-13 19:10:46 +08:00
|
|
|
// getFieldsByCollectionIDPrivate is the private function in collectionReplica, to return vector field ids of collection
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getFieldsByCollectionIDPrivate(collectionID UniqueID) ([]*schemapb.FieldSchema, error) {
|
|
|
|
collection, err := replica.getCollectionByIDPrivate(collectionID)
|
2021-02-06 11:35:35 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(collection.Schema().Fields) <= 0 {
|
2021-03-22 16:36:10 +08:00
|
|
|
return nil, errors.New("no field in collection %d" + strconv.FormatInt(collectionID, 10))
|
2021-02-06 11:35:35 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
return collection.Schema().Fields, nil
|
|
|
|
}
|
|
|
|
|
2021-11-29 19:59:54 +08:00
|
|
|
// getSegmentInfosByColID return segments info by collectionID
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) getSegmentInfosByColID(collectionID UniqueID) []*querypb.SegmentInfo {
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2021-11-05 16:00:55 +08:00
|
|
|
|
|
|
|
segmentInfos := make([]*querypb.SegmentInfo, 0)
|
2022-05-31 13:42:03 +08:00
|
|
|
_, ok := replica.collections[collectionID]
|
2021-11-05 16:00:55 +08:00
|
|
|
if !ok {
|
|
|
|
// collection not exist, so result segmentInfos is empty
|
2022-05-31 13:42:03 +08:00
|
|
|
return segmentInfos
|
2021-11-05 16:00:55 +08:00
|
|
|
}
|
|
|
|
|
2022-05-31 13:42:03 +08:00
|
|
|
for _, segment := range replica.growingSegments {
|
|
|
|
if segment.collectionID == collectionID {
|
|
|
|
segmentInfo := replica.getSegmentInfo(segment)
|
|
|
|
segmentInfos = append(segmentInfos, segmentInfo)
|
2021-11-05 16:00:55 +08:00
|
|
|
}
|
2022-05-31 13:42:03 +08:00
|
|
|
}
|
|
|
|
for _, segment := range replica.sealedSegments {
|
|
|
|
if segment.collectionID == collectionID {
|
2022-05-16 18:23:55 +08:00
|
|
|
segmentInfo := replica.getSegmentInfo(segment)
|
2021-11-05 16:00:55 +08:00
|
|
|
segmentInfos = append(segmentInfos, segmentInfo)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-31 13:42:03 +08:00
|
|
|
return segmentInfos
|
2021-11-05 16:00:55 +08:00
|
|
|
}
|
|
|
|
|
2020-11-09 16:27:11 +08:00
|
|
|
//----------------------------------------------------------------------------------------------------- partition
|
2021-10-13 19:12:39 +08:00
|
|
|
// addPartition adds a new partition to collection
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) addPartition(collectionID UniqueID, partitionID UniqueID) error {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
|
|
|
return replica.addPartitionPrivate(collectionID, partitionID)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
2021-01-21 15:20:23 +08:00
|
|
|
|
2021-10-13 19:14:33 +08:00
|
|
|
// addPartitionPrivate is the private function in collectionReplica, to add a new partition to collection
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) addPartitionPrivate(collectionID UniqueID, partitionID UniqueID) error {
|
|
|
|
collection, err := replica.getCollectionByIDPrivate(collectionID)
|
2021-01-21 15:20:23 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
if !replica.hasPartitionPrivate(partitionID) {
|
2021-12-10 18:47:10 +08:00
|
|
|
collection.addPartitionID(partitionID)
|
|
|
|
var newPartition = newPartition(collectionID, partitionID)
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.partitions[partitionID] = newPartition
|
2021-12-10 18:47:10 +08:00
|
|
|
}
|
2022-03-02 14:49:55 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Set(float64(len(replica.partitions)))
|
2020-11-13 17:20:13 +08:00
|
|
|
return nil
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-14 18:50:51 +08:00
|
|
|
// removePartition removes the partition from collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) removePartition(partitionID UniqueID) error {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
|
|
|
return replica.removePartitionPrivate(partitionID, false)
|
2021-01-13 10:40:46 +08:00
|
|
|
}
|
|
|
|
|
2021-10-14 18:52:41 +08:00
|
|
|
// removePartitionPrivate is the private function in collectionReplica, to remove the partition from collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
// `locked` flag indicates whether corresponding collection lock is accquired before calling this method
|
|
|
|
func (replica *metaReplica) removePartitionPrivate(partitionID UniqueID, locked bool) error {
|
|
|
|
partition, err := replica.getPartitionByIDPrivate(partitionID)
|
2020-11-13 17:20:13 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
collection, err := replica.getCollectionByIDPrivate(partition.collectionID)
|
2020-11-25 10:31:51 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
if !locked {
|
|
|
|
collection.Lock()
|
|
|
|
defer collection.Unlock()
|
|
|
|
}
|
|
|
|
|
2021-02-05 10:53:11 +08:00
|
|
|
// delete segments
|
2022-05-31 13:42:03 +08:00
|
|
|
ids, _ := partition.getSegmentIDs(segmentTypeGrowing)
|
|
|
|
for _, segmentID := range ids {
|
2022-06-01 13:18:02 +08:00
|
|
|
replica.removeSegmentPrivate(segmentID, segmentTypeGrowing)
|
2022-05-31 13:42:03 +08:00
|
|
|
}
|
|
|
|
ids, _ = partition.getSegmentIDs(segmentTypeSealed)
|
|
|
|
for _, segmentID := range ids {
|
2022-06-01 13:18:02 +08:00
|
|
|
replica.removeSegmentPrivate(segmentID, segmentTypeSealed)
|
2020-11-25 10:31:51 +08:00
|
|
|
}
|
2021-02-24 17:24:51 +08:00
|
|
|
|
|
|
|
collection.removePartitionID(partitionID)
|
2022-05-16 18:23:55 +08:00
|
|
|
delete(replica.partitions, partitionID)
|
2021-02-24 17:24:51 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
metrics.QueryNodeNumPartitions.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Set(float64(len(replica.partitions)))
|
2020-11-25 10:31:51 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-14 18:54:34 +08:00
|
|
|
// getPartitionByID returns the partition which id is partitionID
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getPartitionByID(partitionID UniqueID) (*Partition, error) {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
|
|
|
return replica.getPartitionByIDPrivate(partitionID)
|
2021-01-20 09:36:50 +08:00
|
|
|
}
|
|
|
|
|
2021-10-15 18:34:51 +08:00
|
|
|
// getPartitionByIDPrivate is the private function in collectionReplica, to get the partition
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getPartitionByIDPrivate(partitionID UniqueID) (*Partition, error) {
|
|
|
|
partition, ok := replica.partitions[partitionID]
|
2021-02-05 10:53:11 +08:00
|
|
|
if !ok {
|
2022-01-10 10:09:39 +08:00
|
|
|
return nil, fmt.Errorf("partition %d hasn't been loaded or has been released", partitionID)
|
2021-01-20 09:36:50 +08:00
|
|
|
}
|
|
|
|
|
2021-02-05 10:53:11 +08:00
|
|
|
return partition, nil
|
|
|
|
}
|
|
|
|
|
2021-10-15 18:36:42 +08:00
|
|
|
// hasPartition returns true if collectionReplica has the partition, false otherwise
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) hasPartition(partitionID UniqueID) bool {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
|
|
|
return replica.hasPartitionPrivate(partitionID)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
|
2021-10-15 18:38:36 +08:00
|
|
|
// hasPartitionPrivate is the private function in collectionReplica, to check if collectionReplica has the partition
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) hasPartitionPrivate(partitionID UniqueID) bool {
|
|
|
|
_, ok := replica.partitions[partitionID]
|
2021-02-05 10:53:11 +08:00
|
|
|
return ok
|
2021-01-20 09:36:50 +08:00
|
|
|
}
|
|
|
|
|
2021-10-16 21:47:44 +08:00
|
|
|
// getPartitionNum returns num of partitions
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getPartitionNum() int {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
|
|
|
return len(replica.partitions)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
2021-01-13 10:40:46 +08:00
|
|
|
|
2021-10-16 21:45:40 +08:00
|
|
|
// getSegmentIDs returns segment ids
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) getSegmentIDs(partitionID UniqueID, segType segmentType) ([]UniqueID, error) {
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2022-05-31 13:42:03 +08:00
|
|
|
|
|
|
|
return replica.getSegmentIDsPrivate(partitionID, segType)
|
2021-02-06 11:35:35 +08:00
|
|
|
}
|
2020-11-25 10:31:51 +08:00
|
|
|
|
2021-10-16 21:51:32 +08:00
|
|
|
// getSegmentIDsByVChannel returns segment ids which virtual channel is vChannel
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getSegmentIDsByVChannel(partitionID UniqueID, vChannel Channel) ([]UniqueID, error) {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2022-05-31 13:42:03 +08:00
|
|
|
segmentIDs, err := replica.getSegmentIDsPrivate(partitionID, segmentTypeGrowing)
|
2021-01-26 09:38:40 +08:00
|
|
|
if err != nil {
|
2021-06-15 12:41:40 +08:00
|
|
|
return nil, err
|
2021-01-26 09:38:40 +08:00
|
|
|
}
|
2021-06-15 12:41:40 +08:00
|
|
|
segmentIDsTmp := make([]UniqueID, 0)
|
|
|
|
for _, segmentID := range segmentIDs {
|
2022-05-31 13:42:03 +08:00
|
|
|
segment, err := replica.getSegmentByIDPrivate(segmentID, segmentTypeGrowing)
|
2021-06-15 12:41:40 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
if segment.vChannelID == vChannel {
|
|
|
|
segmentIDsTmp = append(segmentIDsTmp, segment.ID())
|
|
|
|
}
|
2021-01-26 09:38:40 +08:00
|
|
|
}
|
|
|
|
|
2021-06-15 12:41:40 +08:00
|
|
|
return segmentIDsTmp, nil
|
2021-01-26 09:38:40 +08:00
|
|
|
}
|
|
|
|
|
2021-10-18 20:22:56 +08:00
|
|
|
// getSegmentIDsPrivate is private function in collectionReplica, it returns segment ids
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) getSegmentIDsPrivate(partitionID UniqueID, segType segmentType) ([]UniqueID, error) {
|
2022-05-16 18:23:55 +08:00
|
|
|
partition, err2 := replica.getPartitionByIDPrivate(partitionID)
|
2021-06-15 12:41:40 +08:00
|
|
|
if err2 != nil {
|
|
|
|
return nil, err2
|
2021-01-26 09:38:40 +08:00
|
|
|
}
|
2022-05-24 21:11:59 +08:00
|
|
|
|
2022-05-31 13:42:03 +08:00
|
|
|
return partition.getSegmentIDs(segType)
|
2021-01-26 09:38:40 +08:00
|
|
|
}
|
|
|
|
|
2020-11-09 16:27:11 +08:00
|
|
|
//----------------------------------------------------------------------------------------------------- segment
|
2021-10-18 20:24:55 +08:00
|
|
|
// addSegment add a new segment to collectionReplica
|
2022-05-23 16:41:58 +08:00
|
|
|
func (replica *metaReplica) addSegment(segmentID UniqueID, partitionID UniqueID, collectionID UniqueID, vChannelID Channel, segType segmentType) error {
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2022-05-31 13:42:03 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
collection, err := replica.getCollectionByIDPrivate(collectionID)
|
2021-01-20 09:36:50 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2022-05-23 16:41:58 +08:00
|
|
|
seg, err := newSegment(collection, segmentID, partitionID, collectionID, vChannelID, segType)
|
2022-02-08 21:57:46 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2022-05-16 18:23:55 +08:00
|
|
|
return replica.addSegmentPrivate(segmentID, partitionID, seg)
|
2021-04-07 18:29:19 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
|
2021-10-18 20:26:59 +08:00
|
|
|
// addSegmentPrivate is private function in collectionReplica, to add a new segment to collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) addSegmentPrivate(segmentID UniqueID, partitionID UniqueID, segment *Segment) error {
|
|
|
|
partition, err := replica.getPartitionByIDPrivate(partitionID)
|
2021-02-05 10:53:11 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
2021-01-20 09:36:50 +08:00
|
|
|
}
|
|
|
|
|
2022-05-31 13:42:03 +08:00
|
|
|
segType := segment.getType()
|
|
|
|
ok, err := replica.hasSegmentPrivate(segmentID, segType)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if ok {
|
2021-02-23 14:13:33 +08:00
|
|
|
return nil
|
|
|
|
}
|
2022-05-31 13:42:03 +08:00
|
|
|
partition.addSegmentID(segmentID, segType)
|
|
|
|
|
|
|
|
switch segType {
|
|
|
|
case segmentTypeGrowing:
|
|
|
|
replica.growingSegments[segmentID] = segment
|
|
|
|
case segmentTypeSealed:
|
|
|
|
replica.sealedSegments[segmentID] = segment
|
|
|
|
default:
|
|
|
|
return fmt.Errorf("unexpected segment type, segmentID = %d, segmentType = %s", segmentID, segType.String())
|
|
|
|
}
|
2020-11-13 17:20:13 +08:00
|
|
|
|
2022-04-24 22:03:44 +08:00
|
|
|
metrics.QueryNodeNumSegments.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Inc()
|
2022-06-02 16:06:03 +08:00
|
|
|
rowCount := segment.getRowCount()
|
|
|
|
if rowCount > 0 {
|
|
|
|
metrics.QueryNodeNumEntities.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Add(float64(rowCount))
|
|
|
|
}
|
2020-11-13 17:20:13 +08:00
|
|
|
return nil
|
2020-11-13 16:53:55 +08:00
|
|
|
}
|
2020-11-13 16:47:47 +08:00
|
|
|
|
2021-10-19 20:27:09 +08:00
|
|
|
// setSegment adds a segment to collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) setSegment(segment *Segment) error {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2022-05-31 13:42:03 +08:00
|
|
|
|
|
|
|
if segment == nil {
|
|
|
|
return fmt.Errorf("nil segment when setSegment")
|
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
_, err := replica.getCollectionByIDPrivate(segment.collectionID)
|
2021-04-07 18:29:19 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2022-05-31 13:42:03 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
return replica.addSegmentPrivate(segment.segmentID, segment.partitionID, segment)
|
2021-04-07 18:29:19 +08:00
|
|
|
}
|
|
|
|
|
2021-10-19 20:28:59 +08:00
|
|
|
// removeSegment removes a segment from collectionReplica
|
2022-06-01 13:18:02 +08:00
|
|
|
func (replica *metaReplica) removeSegment(segmentID UniqueID, segType segmentType) {
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2022-06-01 13:18:02 +08:00
|
|
|
replica.removeSegmentPrivate(segmentID, segType)
|
2021-02-02 19:54:31 +08:00
|
|
|
}
|
|
|
|
|
2021-10-19 20:30:56 +08:00
|
|
|
// removeSegmentPrivate is private function in collectionReplica, to remove a segment from collectionReplica
|
2022-06-01 13:18:02 +08:00
|
|
|
func (replica *metaReplica) removeSegmentPrivate(segmentID UniqueID, segType segmentType) {
|
2022-06-02 16:06:03 +08:00
|
|
|
var rowCount int64
|
2022-05-31 13:42:03 +08:00
|
|
|
switch segType {
|
|
|
|
case segmentTypeGrowing:
|
2022-06-01 13:18:02 +08:00
|
|
|
if segment, ok := replica.growingSegments[segmentID]; ok {
|
|
|
|
if partition, ok := replica.partitions[segment.partitionID]; ok {
|
|
|
|
partition.removeSegmentID(segmentID, segType)
|
|
|
|
}
|
2022-06-02 16:06:03 +08:00
|
|
|
rowCount = segment.getRowCount()
|
2022-06-01 13:18:02 +08:00
|
|
|
delete(replica.growingSegments, segmentID)
|
|
|
|
deleteSegment(segment)
|
|
|
|
}
|
2022-05-31 13:42:03 +08:00
|
|
|
case segmentTypeSealed:
|
2022-06-01 13:18:02 +08:00
|
|
|
if segment, ok := replica.sealedSegments[segmentID]; ok {
|
|
|
|
if partition, ok := replica.partitions[segment.partitionID]; ok {
|
|
|
|
partition.removeSegmentID(segmentID, segType)
|
|
|
|
}
|
|
|
|
|
2022-06-02 16:06:03 +08:00
|
|
|
rowCount = segment.getRowCount()
|
2022-06-01 13:18:02 +08:00
|
|
|
delete(replica.sealedSegments, segmentID)
|
|
|
|
deleteSegment(segment)
|
|
|
|
}
|
2022-05-31 13:42:03 +08:00
|
|
|
default:
|
2022-06-01 13:18:02 +08:00
|
|
|
panic("unsupported segment type")
|
2022-05-31 13:42:03 +08:00
|
|
|
}
|
2022-03-02 14:49:55 +08:00
|
|
|
|
2022-04-24 22:03:44 +08:00
|
|
|
metrics.QueryNodeNumSegments.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Dec()
|
2022-06-02 16:06:03 +08:00
|
|
|
if rowCount > 0 {
|
|
|
|
metrics.QueryNodeNumEntities.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID())).Sub(float64(rowCount))
|
|
|
|
}
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-20 19:23:28 +08:00
|
|
|
// getSegmentByID returns the segment which id is segmentID
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) getSegmentByID(segmentID UniqueID, segType segmentType) (*Segment, error) {
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2022-05-31 13:42:03 +08:00
|
|
|
return replica.getSegmentByIDPrivate(segmentID, segType)
|
2021-01-18 10:38:41 +08:00
|
|
|
}
|
|
|
|
|
2021-10-20 19:05:04 +08:00
|
|
|
// getSegmentByIDPrivate is private function in collectionReplica, it returns the segment which id is segmentID
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) getSegmentByIDPrivate(segmentID UniqueID, segType segmentType) (*Segment, error) {
|
|
|
|
switch segType {
|
|
|
|
case segmentTypeGrowing:
|
|
|
|
segment, ok := replica.growingSegments[segmentID]
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("cannot find growing segment %d in QueryNode", segmentID)
|
|
|
|
}
|
|
|
|
return segment, nil
|
|
|
|
case segmentTypeSealed:
|
|
|
|
segment, ok := replica.sealedSegments[segmentID]
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("cannot find sealed segment %d in QueryNode", segmentID)
|
|
|
|
}
|
|
|
|
return segment, nil
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("unexpected segment type, segmentID = %d, segmentType = %s", segmentID, segType.String())
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-20 19:25:29 +08:00
|
|
|
// hasSegment returns true if collectionReplica has the segment, false otherwise
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) hasSegment(segmentID UniqueID, segType segmentType) (bool, error) {
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2022-05-31 13:42:03 +08:00
|
|
|
return replica.hasSegmentPrivate(segmentID, segType)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
2020-11-13 17:20:13 +08:00
|
|
|
|
2021-10-21 19:48:06 +08:00
|
|
|
// hasSegmentPrivate is private function in collectionReplica, to check if collectionReplica has the segment
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) hasSegmentPrivate(segmentID UniqueID, segType segmentType) (bool, error) {
|
|
|
|
switch segType {
|
|
|
|
case segmentTypeGrowing:
|
|
|
|
_, ok := replica.growingSegments[segmentID]
|
|
|
|
return ok, nil
|
|
|
|
case segmentTypeSealed:
|
|
|
|
_, ok := replica.sealedSegments[segmentID]
|
|
|
|
return ok, nil
|
|
|
|
default:
|
|
|
|
return false, fmt.Errorf("unexpected segment type, segmentID = %d, segmentType = %s", segmentID, segType.String())
|
|
|
|
}
|
2020-11-09 16:27:11 +08:00
|
|
|
}
|
2020-11-24 16:12:39 +08:00
|
|
|
|
2021-10-21 19:54:17 +08:00
|
|
|
// getSegmentNum returns num of segments in collectionReplica
|
2022-05-31 13:42:03 +08:00
|
|
|
func (replica *metaReplica) getSegmentNum(segType segmentType) int {
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2022-05-31 13:42:03 +08:00
|
|
|
|
|
|
|
switch segType {
|
|
|
|
case segmentTypeGrowing:
|
|
|
|
return len(replica.growingSegments)
|
|
|
|
case segmentTypeSealed:
|
|
|
|
return len(replica.sealedSegments)
|
|
|
|
default:
|
|
|
|
log.Error("unexpected segment type", zap.String("segmentType", segType.String()))
|
|
|
|
return 0
|
|
|
|
}
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-21 19:22:50 +08:00
|
|
|
// getSegmentStatistics returns the statistics of segments in collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getSegmentStatistics() []*internalpb.SegmentStats {
|
2022-05-31 13:42:03 +08:00
|
|
|
// TODO: deprecated
|
|
|
|
return nil
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-22 19:01:29 +08:00
|
|
|
// removeExcludedSegments will remove excludedSegments from collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) removeExcludedSegments(collectionID UniqueID) {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2020-11-25 10:31:51 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
delete(replica.excludedSegments, collectionID)
|
2021-03-22 16:36:10 +08:00
|
|
|
}
|
|
|
|
|
2021-10-22 19:03:31 +08:00
|
|
|
// addExcludedSegments will add excludedSegments to collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) addExcludedSegments(collectionID UniqueID, segmentInfos []*datapb.SegmentInfo) {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2021-03-22 16:36:10 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
if _, ok := replica.excludedSegments[collectionID]; !ok {
|
|
|
|
replica.excludedSegments[collectionID] = make([]*datapb.SegmentInfo, 0)
|
2020-11-24 16:12:39 +08:00
|
|
|
}
|
2021-02-02 19:54:31 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.excludedSegments[collectionID] = append(replica.excludedSegments[collectionID], segmentInfos...)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
|
|
|
|
2021-10-22 19:05:14 +08:00
|
|
|
// getExcludedSegments returns excludedSegments of collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getExcludedSegments(collectionID UniqueID) ([]*datapb.SegmentInfo, error) {
|
|
|
|
replica.mu.RLock()
|
|
|
|
defer replica.mu.RUnlock()
|
2021-02-05 10:53:11 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
if _, ok := replica.excludedSegments[collectionID]; !ok {
|
2021-03-22 16:36:10 +08:00
|
|
|
return nil, errors.New("getExcludedSegments failed, cannot found collection, id =" + fmt.Sprintln(collectionID))
|
|
|
|
}
|
2021-02-05 10:53:11 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
return replica.excludedSegments[collectionID], nil
|
2021-03-22 16:36:10 +08:00
|
|
|
}
|
|
|
|
|
2021-10-25 19:40:05 +08:00
|
|
|
// freeAll will free all meta info from collectionReplica
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) freeAll() {
|
|
|
|
replica.mu.Lock()
|
|
|
|
defer replica.mu.Unlock()
|
2021-03-22 16:36:10 +08:00
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
for id := range replica.collections {
|
|
|
|
_ = replica.removeCollectionPrivate(id)
|
2021-02-05 10:53:11 +08:00
|
|
|
}
|
|
|
|
|
2022-05-16 18:23:55 +08:00
|
|
|
replica.collections = make(map[UniqueID]*Collection)
|
|
|
|
replica.partitions = make(map[UniqueID]*Partition)
|
2022-05-31 13:42:03 +08:00
|
|
|
replica.growingSegments = make(map[UniqueID]*Segment)
|
|
|
|
replica.sealedSegments = make(map[UniqueID]*Segment)
|
2020-11-24 16:12:39 +08:00
|
|
|
}
|
2021-03-12 19:23:06 +08:00
|
|
|
|
2021-10-25 19:36:12 +08:00
|
|
|
// newCollectionReplica returns a new ReplicaInterface
|
2022-05-31 13:42:03 +08:00
|
|
|
func newCollectionReplica() ReplicaInterface {
|
2022-05-16 18:23:55 +08:00
|
|
|
var replica ReplicaInterface = &metaReplica{
|
2022-05-31 13:42:03 +08:00
|
|
|
collections: make(map[UniqueID]*Collection),
|
|
|
|
partitions: make(map[UniqueID]*Partition),
|
|
|
|
growingSegments: make(map[UniqueID]*Segment),
|
|
|
|
sealedSegments: make(map[UniqueID]*Segment),
|
2021-03-22 16:36:10 +08:00
|
|
|
|
2022-05-31 13:42:03 +08:00
|
|
|
excludedSegments: make(map[UniqueID][]*datapb.SegmentInfo),
|
2021-03-22 16:36:10 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
return replica
|
|
|
|
}
|
2021-11-05 16:00:55 +08:00
|
|
|
|
|
|
|
// trans segment to queryPb.segmentInfo
|
2022-05-16 18:23:55 +08:00
|
|
|
func (replica *metaReplica) getSegmentInfo(segment *Segment) *querypb.SegmentInfo {
|
2021-11-05 16:00:55 +08:00
|
|
|
var indexName string
|
|
|
|
var indexID int64
|
2022-03-30 21:11:28 +08:00
|
|
|
var indexInfos []*querypb.FieldIndexInfo
|
2021-11-05 16:00:55 +08:00
|
|
|
// TODO:: segment has multi vec column
|
2022-05-16 18:23:55 +08:00
|
|
|
indexedFieldIDs, _ := replica.getIndexedFieldIDByCollectionIDPrivate(segment.collectionID, segment)
|
2022-03-30 21:11:28 +08:00
|
|
|
for _, fieldID := range indexedFieldIDs {
|
|
|
|
fieldInfo, err := segment.getIndexedFieldInfo(fieldID)
|
|
|
|
if err == nil {
|
|
|
|
indexName = fieldInfo.indexInfo.IndexName
|
|
|
|
indexID = fieldInfo.indexInfo.IndexID
|
|
|
|
indexInfos = append(indexInfos, fieldInfo.indexInfo)
|
2022-02-08 21:57:46 +08:00
|
|
|
}
|
2021-11-05 16:00:55 +08:00
|
|
|
}
|
|
|
|
info := &querypb.SegmentInfo{
|
|
|
|
SegmentID: segment.ID(),
|
|
|
|
CollectionID: segment.collectionID,
|
|
|
|
PartitionID: segment.partitionID,
|
2022-04-24 22:03:44 +08:00
|
|
|
NodeID: Params.QueryNodeCfg.GetNodeID(),
|
2021-11-05 16:00:55 +08:00
|
|
|
MemSize: segment.getMemSize(),
|
|
|
|
NumRows: segment.getRowCount(),
|
|
|
|
IndexName: indexName,
|
|
|
|
IndexID: indexID,
|
2021-12-15 16:53:12 +08:00
|
|
|
DmChannel: segment.vChannelID,
|
2022-02-08 21:57:46 +08:00
|
|
|
SegmentState: segment.segmentType,
|
2022-03-30 21:11:28 +08:00
|
|
|
IndexInfos: indexInfos,
|
2022-04-29 14:11:47 +08:00
|
|
|
NodeIds: []UniqueID{Params.QueryNodeCfg.GetNodeID()},
|
2021-11-05 16:00:55 +08:00
|
|
|
}
|
|
|
|
return info
|
|
|
|
}
|