milvus/internal/querynode/historical.go
bigsheeper dbaca0a8c7
[skip ci] Add comment for historical close (#13373)
Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
2021-12-14 21:55:06 +08:00

318 lines
8.9 KiB
Go

// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package querynode
import (
"context"
"errors"
"fmt"
"path/filepath"
"strconv"
"sync"
"github.com/golang/protobuf/proto"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"go.etcd.io/etcd/api/v3/mvccpb"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/segcorepb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util"
)
// historical is in charge of historical data in query node
type historical struct {
ctx context.Context
replica ReplicaInterface
tSafeReplica TSafeReplicaInterface
mu sync.Mutex // guards globalSealedSegments
globalSealedSegments map[UniqueID]*querypb.SegmentInfo
etcdKV *etcdkv.EtcdKV
}
// newHistorical returns a new historical
func newHistorical(ctx context.Context,
replica ReplicaInterface,
etcdKV *etcdkv.EtcdKV,
tSafeReplica TSafeReplicaInterface) *historical {
return &historical{
ctx: ctx,
replica: replica,
globalSealedSegments: make(map[UniqueID]*querypb.SegmentInfo),
etcdKV: etcdKV,
tSafeReplica: tSafeReplica,
}
}
func (h *historical) start() {
go h.watchGlobalSegmentMeta()
}
// close would release all resources in historical
func (h *historical) close() {
// free collectionReplica
h.replica.freeAll()
}
func (h *historical) watchGlobalSegmentMeta() {
log.Debug("query node watchGlobalSegmentMeta start")
watchChan := h.etcdKV.WatchWithPrefix(util.SegmentMetaPrefix)
for {
select {
case <-h.ctx.Done():
log.Debug("query node watchGlobalSegmentMeta close")
return
case resp := <-watchChan:
for _, event := range resp.Events {
segmentID, err := strconv.ParseInt(filepath.Base(string(event.Kv.Key)), 10, 64)
if err != nil {
log.Warn("watchGlobalSegmentMeta failed", zap.Any("error", err.Error()))
continue
}
switch event.Type {
case mvccpb.PUT:
log.Debug("globalSealedSegments add segment",
zap.Any("segmentID", segmentID),
)
segmentInfo := &querypb.SegmentInfo{}
err = proto.Unmarshal(event.Kv.Value, segmentInfo)
if err != nil {
log.Warn("watchGlobalSegmentMeta failed", zap.Any("error", err.Error()))
continue
}
h.addGlobalSegmentInfo(segmentID, segmentInfo)
case mvccpb.DELETE:
log.Debug("globalSealedSegments delete segment",
zap.Any("segmentID", segmentID),
)
h.removeGlobalSegmentInfo(segmentID)
}
}
}
}
}
func (h *historical) addGlobalSegmentInfo(segmentID UniqueID, segmentInfo *querypb.SegmentInfo) {
h.mu.Lock()
defer h.mu.Unlock()
h.globalSealedSegments[segmentID] = segmentInfo
}
func (h *historical) removeGlobalSegmentInfo(segmentID UniqueID) {
h.mu.Lock()
defer h.mu.Unlock()
delete(h.globalSealedSegments, segmentID)
}
func (h *historical) getGlobalSegmentIDsByCollectionID(collectionID UniqueID) []UniqueID {
h.mu.Lock()
defer h.mu.Unlock()
resIDs := make([]UniqueID, 0)
for _, v := range h.globalSealedSegments {
if v.CollectionID == collectionID {
resIDs = append(resIDs, v.SegmentID)
}
}
return resIDs
}
func (h *historical) getGlobalSegmentIDsByPartitionIds(partitionIDs []UniqueID) []UniqueID {
h.mu.Lock()
defer h.mu.Unlock()
resIDs := make([]UniqueID, 0)
for _, v := range h.globalSealedSegments {
for _, partitionID := range partitionIDs {
if v.PartitionID == partitionID {
resIDs = append(resIDs, v.SegmentID)
}
}
}
return resIDs
}
func (h *historical) removeGlobalSegmentIDsByCollectionID(collectionID UniqueID) {
h.mu.Lock()
defer h.mu.Unlock()
for _, v := range h.globalSealedSegments {
if v.CollectionID == collectionID {
delete(h.globalSealedSegments, v.SegmentID)
}
}
}
func (h *historical) removeGlobalSegmentIDsByPartitionIds(partitionIDs []UniqueID) {
h.mu.Lock()
defer h.mu.Unlock()
for _, v := range h.globalSealedSegments {
for _, partitionID := range partitionIDs {
if v.PartitionID == partitionID {
delete(h.globalSealedSegments, v.SegmentID)
}
}
}
}
func (h *historical) retrieve(collID UniqueID, partIDs []UniqueID, vcm storage.ChunkManager,
plan *RetrievePlan) ([]*segcorepb.RetrieveResults, []UniqueID, []UniqueID, error) {
retrieveResults := make([]*segcorepb.RetrieveResults, 0)
retrieveSegmentIDs := make([]UniqueID, 0)
// get historical partition ids
var retrievePartIDs []UniqueID
if len(partIDs) == 0 {
hisPartIDs, err := h.replica.getPartitionIDs(collID)
if err != nil {
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err
}
retrievePartIDs = hisPartIDs
} else {
for _, id := range partIDs {
_, err := h.replica.getPartitionByID(id)
if err == nil {
retrievePartIDs = append(retrievePartIDs, id)
}
}
}
for _, partID := range retrievePartIDs {
segIDs, err := h.replica.getSegmentIDs(partID)
if err != nil {
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err
}
for _, segID := range segIDs {
seg, err := h.replica.getSegmentByID(segID)
if err != nil {
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err
}
result, err := seg.retrieve(plan)
if err != nil {
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err
}
if err = seg.fillVectorFieldsData(collID, vcm, result); err != nil {
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err
}
retrieveResults = append(retrieveResults, result)
retrieveSegmentIDs = append(retrieveSegmentIDs, segID)
}
}
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, nil
}
// search will search all the target segments in historical
func (h *historical) search(searchReqs []*searchRequest, collID UniqueID, partIDs []UniqueID, plan *SearchPlan,
searchTs Timestamp) ([]*SearchResult, []UniqueID, []UniqueID, error) {
searchResults := make([]*SearchResult, 0)
searchSegmentIDs := make([]UniqueID, 0)
// get historical partition ids
var searchPartIDs []UniqueID
if len(partIDs) == 0 {
hisPartIDs, err := h.replica.getPartitionIDs(collID)
if err != nil {
return searchResults, searchSegmentIDs, searchPartIDs, err
}
log.Debug("no partition specified, search all partitions",
zap.Any("collectionID", collID),
zap.Any("all partitions", hisPartIDs),
)
searchPartIDs = hisPartIDs
} else {
for _, id := range partIDs {
_, err := h.replica.getPartitionByID(id)
if err == nil {
log.Debug("append search partition id",
zap.Any("collectionID", collID),
zap.Any("partitionID", id),
)
searchPartIDs = append(searchPartIDs, id)
}
}
}
col, err := h.replica.getCollectionByID(collID)
if err != nil {
return searchResults, searchSegmentIDs, searchPartIDs, err
}
// all partitions have been released
if len(searchPartIDs) == 0 && col.getLoadType() == loadTypePartition {
return searchResults, searchSegmentIDs, searchPartIDs, errors.New("partitions have been released , collectionID = " +
fmt.Sprintln(collID) + "target partitionIDs = " + fmt.Sprintln(partIDs))
}
if len(searchPartIDs) == 0 && col.getLoadType() == loadTypeCollection {
if err = col.checkReleasedPartitions(partIDs); err != nil {
return searchResults, searchSegmentIDs, searchPartIDs, err
}
return searchResults, searchSegmentIDs, searchPartIDs, nil
}
var segmentLock sync.RWMutex
for _, partID := range searchPartIDs {
segIDs, err := h.replica.getSegmentIDs(partID)
if err != nil {
return searchResults, searchSegmentIDs, searchPartIDs, err
}
var err2 error
var wg sync.WaitGroup
for _, segID := range segIDs {
segID2 := segID
wg.Add(1)
go func() {
defer wg.Done()
seg, err := h.replica.getSegmentByID(segID2)
if err != nil {
err2 = err
return
}
if !seg.getOnService() {
return
}
searchResult, err := seg.search(plan, searchReqs, []Timestamp{searchTs})
if err != nil {
err2 = err
return
}
segmentLock.Lock()
searchResults = append(searchResults, searchResult)
searchSegmentIDs = append(searchSegmentIDs, seg.segmentID)
segmentLock.Unlock()
}()
}
wg.Wait()
if err2 != nil {
return searchResults, searchSegmentIDs, searchPartIDs, err2
}
}
return searchResults, searchSegmentIDs, searchPartIDs, nil
}