2022-04-06 15:43:31 +08:00
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
2022-04-02 17:35:29 +08:00
// 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"
2022-06-17 17:38:12 +08:00
"errors"
2022-04-02 17:35:29 +08:00
"fmt"
2022-07-20 15:02:30 +08:00
"runtime"
2022-04-02 17:35:29 +08:00
"sync"
2022-05-06 16:55:51 +08:00
"go.uber.org/atomic"
"go.uber.org/zap"
2022-10-08 20:26:57 +08:00
"github.com/golang/protobuf/proto"
2022-10-16 20:49:27 +08:00
"github.com/milvus-io/milvus-proto/go-api/commonpb"
2022-09-02 10:42:59 +08:00
"github.com/milvus-io/milvus/internal/common"
2022-04-02 17:35:29 +08:00
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
2022-07-06 15:06:21 +08:00
"github.com/milvus-io/milvus/internal/util/funcutil"
2022-06-30 20:14:19 +08:00
"github.com/milvus-io/milvus/internal/util/typeutil"
2022-04-02 17:35:29 +08:00
)
type shardClusterState int32
const (
available shardClusterState = 1
unavailable shardClusterState = 2
)
type nodeEventType int32
const (
nodeAdd nodeEventType = 1
nodeDel nodeEventType = 2
)
type segmentEventType int32
const (
segmentAdd segmentEventType = 1
segmentDel segmentEventType = 2
)
type segmentState int32
const (
segmentStateNone segmentState = 0
segmentStateOffline segmentState = 1
segmentStateLoading segmentState = 2
segmentStateLoaded segmentState = 3
)
type nodeEvent struct {
eventType nodeEventType
nodeID int64
nodeAddr string
2022-06-17 17:38:12 +08:00
isLeader bool
2022-04-02 17:35:29 +08:00
}
type segmentEvent struct {
2022-04-20 16:15:41 +08:00
eventType segmentEventType
segmentID int64
partitionID int64
2022-05-11 17:39:53 +08:00
nodeIDs [ ] int64 // nodes from events
2022-04-20 16:15:41 +08:00
state segmentState
2022-04-02 17:35:29 +08:00
}
type shardQueryNode interface {
2022-07-18 09:58:28 +08:00
GetStatistics ( context . Context , * querypb . GetStatisticsRequest ) ( * internalpb . GetStatisticsResponse , error )
2022-04-02 17:35:29 +08:00
Search ( context . Context , * querypb . SearchRequest ) ( * internalpb . SearchResults , error )
Query ( context . Context , * querypb . QueryRequest ) ( * internalpb . RetrieveResults , error )
2022-09-15 18:48:32 +08:00
LoadSegments ( ctx context . Context , in * querypb . LoadSegmentsRequest ) ( * commonpb . Status , error )
2022-04-27 22:23:46 +08:00
ReleaseSegments ( ctx context . Context , in * querypb . ReleaseSegmentsRequest ) ( * commonpb . Status , error )
2022-04-20 16:15:41 +08:00
Stop ( ) error
2022-04-02 17:35:29 +08:00
}
type shardNode struct {
nodeID int64
nodeAddr string
client shardQueryNode
}
type shardSegmentInfo struct {
2022-04-20 16:15:41 +08:00
segmentID int64
partitionID int64
nodeID int64
state segmentState
2022-09-28 12:10:54 +08:00
version int64
2022-04-25 11:51:46 +08:00
inUse int32
2022-04-02 17:35:29 +08:00
}
2022-08-01 13:50:33 +08:00
// Closable interface for close.
type Closable interface {
Close ( )
}
2022-04-02 17:35:29 +08:00
// ShardNodeDetector provides method to detect node events
type ShardNodeDetector interface {
2022-08-01 13:50:33 +08:00
Closable
2022-04-02 17:35:29 +08:00
watchNodes ( collectionID int64 , replicaID int64 , vchannelName string ) ( [ ] nodeEvent , <- chan nodeEvent )
}
// ShardSegmentDetector provides method to detect segment events
type ShardSegmentDetector interface {
2022-08-01 13:50:33 +08:00
Closable
2022-04-02 17:35:29 +08:00
watchSegments ( collectionID int64 , replicaID int64 , vchannelName string ) ( [ ] segmentEvent , <- chan segmentEvent )
}
// ShardNodeBuilder function type to build types.QueryNode from addr and id
type ShardNodeBuilder func ( nodeID int64 , addr string ) shardQueryNode
2022-06-17 17:38:12 +08:00
// withStreaming function type to let search detects corresponding search streaming is done.
type withStreaming func ( ctx context . Context ) error
2022-04-02 17:35:29 +08:00
// ShardCluster maintains the ShardCluster information and perform shard level operations
type ShardCluster struct {
state * atomic . Int32
collectionID int64
replicaID int64
vchannelName string
2022-09-15 18:48:32 +08:00
version int64
2022-04-02 17:35:29 +08:00
nodeDetector ShardNodeDetector
segmentDetector ShardSegmentDetector
nodeBuilder ShardNodeBuilder
2022-06-25 18:48:15 +08:00
mut sync . RWMutex
leader * shardNode // shard leader node instance
nodes map [ int64 ] * shardNode // online nodes
segments SegmentsStatus // shard segments
mutVersion sync . RWMutex
versions sync . Map // version id to version
currentVersion * ShardClusterVersion // current serving segment state version
2022-06-17 17:38:12 +08:00
nextVersionID * atomic . Int64
segmentCond * sync . Cond // segment state change condition
2022-04-02 17:35:29 +08:00
closeOnce sync . Once
closeCh chan struct { }
}
// NewShardCluster create a ShardCluster with provided information.
func NewShardCluster ( collectionID int64 , replicaID int64 , vchannelName string ,
nodeDetector ShardNodeDetector , segmentDetector ShardSegmentDetector , nodeBuilder ShardNodeBuilder ) * ShardCluster {
sc := & ShardCluster {
state : atomic . NewInt32 ( int32 ( unavailable ) ) ,
collectionID : collectionID ,
replicaID : replicaID ,
vchannelName : vchannelName ,
segmentDetector : segmentDetector ,
2022-09-29 18:02:54 +08:00
nodeDetector : nodeDetector ,
2022-04-02 17:35:29 +08:00
nodeBuilder : nodeBuilder ,
2022-06-17 17:38:12 +08:00
nodes : make ( map [ int64 ] * shardNode ) ,
segments : make ( map [ int64 ] shardSegmentInfo ) ,
nextVersionID : atomic . NewInt64 ( 0 ) ,
2022-04-02 17:35:29 +08:00
closeCh : make ( chan struct { } ) ,
}
2022-04-25 11:51:46 +08:00
m := sync . Mutex { }
sc . segmentCond = sync . NewCond ( & m )
2022-04-02 17:35:29 +08:00
sc . init ( )
return sc
}
func ( sc * ShardCluster ) Close ( ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-05-07 10:27:51 +08:00
log . Info ( "Close shard cluster" )
2022-04-02 17:35:29 +08:00
sc . closeOnce . Do ( func ( ) {
2022-07-20 15:02:30 +08:00
sc . updateShardClusterState ( unavailable )
2022-08-01 13:50:33 +08:00
if sc . nodeDetector != nil {
sc . nodeDetector . Close ( )
}
if sc . segmentDetector != nil {
sc . segmentDetector . Close ( )
}
2022-04-02 17:35:29 +08:00
close ( sc . closeCh )
} )
}
2022-09-27 19:22:54 +08:00
func ( sc * ShardCluster ) getLogger ( ) * zap . Logger {
return log . With ( zap . Int64 ( "collectionID" , sc . collectionID ) ,
zap . String ( "channel" , sc . vchannelName ) ,
zap . Int64 ( "replicaID" , sc . replicaID ) )
}
2022-06-17 17:38:12 +08:00
// serviceable returns whether shard cluster could provide query service.
func ( sc * ShardCluster ) serviceable ( ) bool {
// all segment in loaded state
if sc . state . Load ( ) != int32 ( available ) {
return false
}
2022-06-25 18:48:15 +08:00
sc . mutVersion . RLock ( )
defer sc . mutVersion . RUnlock ( )
2022-06-17 17:38:12 +08:00
// check there is a working version(SyncSegments called)
return sc . currentVersion != nil
}
2022-04-02 17:35:29 +08:00
// addNode add a node into cluster
func ( sc * ShardCluster ) addNode ( evt nodeEvent ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-05-07 10:27:51 +08:00
log . Info ( "ShardCluster add node" , zap . Int64 ( "nodeID" , evt . nodeID ) )
2022-04-02 17:35:29 +08:00
sc . mut . Lock ( )
defer sc . mut . Unlock ( )
oldNode , ok := sc . nodes [ evt . nodeID ]
if ok {
if oldNode . nodeAddr == evt . nodeAddr {
log . Warn ( "ShardCluster add same node, skip" , zap . Int64 ( "nodeID" , evt . nodeID ) , zap . String ( "addr" , evt . nodeAddr ) )
return
}
defer oldNode . client . Stop ( )
}
2022-06-17 17:38:12 +08:00
node := & shardNode {
2022-04-02 17:35:29 +08:00
nodeID : evt . nodeID ,
nodeAddr : evt . nodeAddr ,
client : sc . nodeBuilder ( evt . nodeID , evt . nodeAddr ) ,
}
2022-06-17 17:38:12 +08:00
sc . nodes [ evt . nodeID ] = node
if evt . isLeader {
sc . leader = node
}
2022-04-02 17:35:29 +08:00
}
// removeNode handles node offline and setup related segments
func ( sc * ShardCluster ) removeNode ( evt nodeEvent ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-05-07 10:27:51 +08:00
log . Info ( "ShardCluster remove node" , zap . Int64 ( "nodeID" , evt . nodeID ) )
2022-04-02 17:35:29 +08:00
sc . mut . Lock ( )
defer sc . mut . Unlock ( )
old , ok := sc . nodes [ evt . nodeID ]
if ! ok {
log . Warn ( "ShardCluster removeNode does not belong to it" , zap . Int64 ( "nodeID" , evt . nodeID ) , zap . String ( "addr" , evt . nodeAddr ) )
return
}
defer old . client . Stop ( )
delete ( sc . nodes , evt . nodeID )
2022-06-17 17:38:12 +08:00
for id , segment := range sc . segments {
2022-04-02 17:35:29 +08:00
if segment . nodeID == evt . nodeID {
segment . state = segmentStateOffline
2022-09-28 12:10:54 +08:00
segment . version = - 1
2022-06-17 17:38:12 +08:00
sc . segments [ id ] = segment
2022-07-20 15:02:30 +08:00
sc . updateShardClusterState ( unavailable )
2022-04-02 17:35:29 +08:00
}
}
2022-06-17 17:38:12 +08:00
// ignore leader process here
2022-04-02 17:35:29 +08:00
}
// updateSegment apply segment change to shard cluster
2022-05-11 17:39:53 +08:00
func ( sc * ShardCluster ) updateSegment ( evt shardSegmentInfo ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-05-07 10:27:51 +08:00
log . Info ( "ShardCluster update segment" , zap . Int64 ( "nodeID" , evt . nodeID ) , zap . Int64 ( "segmentID" , evt . segmentID ) , zap . Int32 ( "state" , int32 ( evt . state ) ) )
2022-04-25 11:51:46 +08:00
// notify handoff wait online if any
defer func ( ) {
sc . segmentCond . Broadcast ( )
} ( )
2022-04-02 17:35:29 +08:00
sc . mut . Lock ( )
defer sc . mut . Unlock ( )
old , ok := sc . segments [ evt . segmentID ]
if ! ok { // newly add
2022-09-28 12:10:54 +08:00
sc . segments [ evt . segmentID ] = evt
2022-04-02 17:35:29 +08:00
return
}
sc . transferSegment ( old , evt )
}
2022-10-20 16:35:28 +08:00
// SetupFirstVersion initialized first version for shard cluster.
func ( sc * ShardCluster ) SetupFirstVersion ( ) {
sc . mutVersion . Lock ( )
defer sc . mutVersion . Unlock ( )
version := NewShardClusterVersion ( sc . nextVersionID . Inc ( ) , make ( SegmentsStatus ) , nil )
sc . versions . Store ( version . versionID , version )
sc . currentVersion = version
}
2022-04-27 10:41:46 +08:00
// SyncSegments synchronize segment distribution in batch
func ( sc * ShardCluster ) SyncSegments ( distribution [ ] * querypb . ReplicaSegmentsInfo , state segmentState ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-05-07 10:27:51 +08:00
log . Info ( "ShardCluster sync segments" , zap . Any ( "replica segments" , distribution ) , zap . Int32 ( "state" , int32 ( state ) ) )
2022-04-27 10:41:46 +08:00
2022-10-20 16:35:28 +08:00
var currentVersion * ShardClusterVersion
sc . mutVersion . RLock ( )
currentVersion = sc . currentVersion
sc . mutVersion . RUnlock ( )
if currentVersion == nil {
log . Warn ( "received SyncSegments call before version setup" )
return
}
2022-06-25 18:48:15 +08:00
sc . mut . Lock ( )
2022-04-27 10:41:46 +08:00
for _ , line := range distribution {
2022-09-28 12:10:54 +08:00
for i , segmentID := range line . GetSegmentIds ( ) {
2022-09-02 10:42:59 +08:00
nodeID := line . GetNodeId ( )
2022-09-28 12:10:54 +08:00
version := line . GetVersions ( ) [ i ]
2022-09-02 10:42:59 +08:00
// if node id not in replica node list, this line shall be placeholder for segment offline
_ , ok := sc . nodes [ nodeID ]
if ! ok {
log . Warn ( "Sync segment with invalid nodeID" , zap . Int64 ( "segmentID" , segmentID ) , zap . Int64 ( "nodeID" , line . NodeId ) )
nodeID = common . InvalidNodeID
}
2022-04-27 10:41:46 +08:00
old , ok := sc . segments [ segmentID ]
if ! ok { // newly add
2022-06-17 17:38:12 +08:00
sc . segments [ segmentID ] = shardSegmentInfo {
2022-09-02 10:42:59 +08:00
nodeID : nodeID ,
2022-04-27 10:41:46 +08:00
partitionID : line . GetPartitionId ( ) ,
segmentID : segmentID ,
state : state ,
2022-09-28 12:10:54 +08:00
version : version ,
2022-04-27 10:41:46 +08:00
}
continue
}
2022-05-11 17:39:53 +08:00
sc . transferSegment ( old , shardSegmentInfo {
2022-09-02 10:42:59 +08:00
nodeID : nodeID ,
2022-04-27 10:41:46 +08:00
partitionID : line . GetPartitionId ( ) ,
segmentID : segmentID ,
state : state ,
2022-09-28 12:10:54 +08:00
version : version ,
2022-04-27 10:41:46 +08:00
} )
}
}
2022-07-05 13:16:20 +08:00
2022-10-20 16:35:28 +08:00
// allocations := sc.segments.Clone(filterNothing)
2022-06-25 18:48:15 +08:00
sc . mut . Unlock ( )
// notify handoff wait online if any
sc . segmentCond . Broadcast ( )
2022-06-17 17:38:12 +08:00
2022-06-25 18:48:15 +08:00
sc . mutVersion . Lock ( )
defer sc . mutVersion . Unlock ( )
2022-10-20 16:35:28 +08:00
// update shardleader allocation view
allocations := sc . currentVersion . segments . Clone ( filterNothing )
for _ , line := range distribution {
for _ , segmentID := range line . GetSegmentIds ( ) {
allocations [ segmentID ] = shardSegmentInfo { nodeID : line . GetNodeId ( ) , segmentID : segmentID , partitionID : line . GetPartitionId ( ) , state : state }
}
}
2022-09-15 18:48:32 +08:00
version := NewShardClusterVersion ( sc . nextVersionID . Inc ( ) , allocations , sc . currentVersion )
2022-06-25 18:48:15 +08:00
sc . versions . Store ( version . versionID , version )
2022-06-17 17:38:12 +08:00
sc . currentVersion = version
2022-04-27 10:41:46 +08:00
}
2022-04-02 17:35:29 +08:00
// transferSegment apply segment state transition.
2022-05-07 18:25:52 +08:00
// old\new | Offline | Loading | Loaded
// Offline | OK | OK | OK
// Loading | OK | OK | NodeID check
// Loaded | OK | OK | legacy pending
2022-06-17 17:38:12 +08:00
func ( sc * ShardCluster ) transferSegment ( old shardSegmentInfo , evt shardSegmentInfo ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-04-02 17:35:29 +08:00
switch old . state {
case segmentStateOffline : // safe to update nodeID and state
old . nodeID = evt . nodeID
old . state = evt . state
2022-09-28 12:10:54 +08:00
old . version = evt . version
2022-06-17 17:38:12 +08:00
sc . segments [ old . segmentID ] = old
2022-04-02 17:35:29 +08:00
if evt . state == segmentStateLoaded {
sc . healthCheck ( )
}
case segmentStateLoading : // to Loaded only when nodeID equal
if evt . state == segmentStateLoaded && evt . nodeID != old . nodeID {
log . Warn ( "transferSegment to loaded failed, nodeID not match" , zap . Int64 ( "segmentID" , evt . segmentID ) , zap . Int64 ( "nodeID" , old . nodeID ) , zap . Int64 ( "evtNodeID" , evt . nodeID ) )
return
}
old . nodeID = evt . nodeID
old . state = evt . state
2022-09-28 12:10:54 +08:00
old . version = evt . version
2022-06-17 17:38:12 +08:00
sc . segments [ old . segmentID ] = old
2022-04-02 17:35:29 +08:00
if evt . state == segmentStateLoaded {
sc . healthCheck ( )
}
case segmentStateLoaded :
2022-05-07 18:25:52 +08:00
// load balance
2022-04-02 17:35:29 +08:00
old . nodeID = evt . nodeID
old . state = evt . state
2022-09-28 12:10:54 +08:00
old . version = evt . version
2022-06-17 17:38:12 +08:00
sc . segments [ old . segmentID ] = old
2022-04-02 17:35:29 +08:00
if evt . state != segmentStateLoaded {
sc . healthCheck ( )
}
}
}
// removeSegment removes segment from cluster
// should only applied in hand-off or load balance procedure
2022-05-11 17:39:53 +08:00
func ( sc * ShardCluster ) removeSegment ( evt shardSegmentInfo ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-05-07 10:27:51 +08:00
log . Info ( "ShardCluster remove segment" , zap . Int64 ( "nodeID" , evt . nodeID ) , zap . Int64 ( "segmentID" , evt . segmentID ) , zap . Int32 ( "state" , int32 ( evt . state ) ) )
2022-04-25 11:51:46 +08:00
2022-04-02 17:35:29 +08:00
sc . mut . Lock ( )
defer sc . mut . Unlock ( )
old , ok := sc . segments [ evt . segmentID ]
if ! ok {
log . Warn ( "ShardCluster removeSegment does not belong to it" , zap . Int64 ( "nodeID" , evt . nodeID ) , zap . Int64 ( "segmentID" , evt . segmentID ) )
return
}
if old . nodeID != evt . nodeID {
log . Warn ( "ShardCluster removeSegment found node not match" , zap . Int64 ( "segmentID" , evt . segmentID ) , zap . Int64 ( "nodeID" , old . nodeID ) , zap . Int64 ( "evtNodeID" , evt . nodeID ) )
return
}
delete ( sc . segments , evt . segmentID )
2022-09-02 10:42:59 +08:00
sc . healthCheck ( )
2022-04-02 17:35:29 +08:00
}
// init list all nodes and semgent states ant start watching
func ( sc * ShardCluster ) init ( ) {
// list nodes
nodes , nodeEvtCh := sc . nodeDetector . watchNodes ( sc . collectionID , sc . replicaID , sc . vchannelName )
for _ , node := range nodes {
sc . addNode ( node )
}
go sc . watchNodes ( nodeEvtCh )
// list segments
segments , segmentEvtCh := sc . segmentDetector . watchSegments ( sc . collectionID , sc . replicaID , sc . vchannelName )
for _ , segment := range segments {
2022-05-11 17:39:53 +08:00
info , ok := sc . pickNode ( segment )
if ok {
sc . updateSegment ( info )
}
2022-04-02 17:35:29 +08:00
}
go sc . watchSegments ( segmentEvtCh )
sc . healthCheck ( )
}
2022-06-27 22:08:17 +08:00
// pickNode selects node in the cluster
2022-05-11 17:39:53 +08:00
func ( sc * ShardCluster ) pickNode ( evt segmentEvent ) ( shardSegmentInfo , bool ) {
2022-06-27 22:08:17 +08:00
nodeID , has := sc . selectNodeInReplica ( evt . nodeIDs )
if has { // assume one segment shall exist once in one replica
return shardSegmentInfo {
segmentID : evt . segmentID ,
partitionID : evt . partitionID ,
nodeID : nodeID ,
state : evt . state ,
} , true
}
return shardSegmentInfo { } , false
}
// selectNodeInReplica returns first node id inside the shard cluster replica.
// if there is no nodeID found, returns 0.
func ( sc * ShardCluster ) selectNodeInReplica ( nodeIDs [ ] int64 ) ( int64 , bool ) {
for _ , nodeID := range nodeIDs {
2022-05-11 17:39:53 +08:00
_ , has := sc . getNode ( nodeID )
2022-06-27 22:08:17 +08:00
if has {
return nodeID , true
2022-05-11 17:39:53 +08:00
}
}
2022-06-27 22:08:17 +08:00
return 0 , false
2022-05-11 17:39:53 +08:00
}
2022-07-20 15:02:30 +08:00
func ( sc * ShardCluster ) updateShardClusterState ( state shardClusterState ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-07-20 15:02:30 +08:00
old := sc . state . Load ( )
sc . state . Store ( int32 ( state ) )
pc , _ , _ , _ := runtime . Caller ( 1 )
callerName := runtime . FuncForPC ( pc ) . Name ( )
2022-09-27 19:22:54 +08:00
log . Info ( "Shard Cluster update state" ,
2022-07-20 15:02:30 +08:00
zap . Int32 ( "old state" , old ) , zap . Int32 ( "new state" , int32 ( state ) ) ,
zap . String ( "caller" , callerName ) )
}
2022-04-02 17:35:29 +08:00
// healthCheck iterate all segments to to check cluster could provide service.
func ( sc * ShardCluster ) healthCheck ( ) {
for _ , segment := range sc . segments {
2022-09-02 10:42:59 +08:00
if segment . state != segmentStateLoaded ||
segment . nodeID == common . InvalidNodeID { // segment in offline nodes
2022-07-20 15:02:30 +08:00
sc . updateShardClusterState ( unavailable )
2022-04-02 17:35:29 +08:00
return
}
}
2022-07-20 15:02:30 +08:00
sc . updateShardClusterState ( available )
2022-04-02 17:35:29 +08:00
}
// watchNodes handles node events.
func ( sc * ShardCluster ) watchNodes ( evtCh <- chan nodeEvent ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-04-02 17:35:29 +08:00
for {
select {
case evt , ok := <- evtCh :
if ! ok {
2022-09-27 19:22:54 +08:00
log . Warn ( "ShardCluster node channel closed" )
2022-04-02 17:35:29 +08:00
return
}
switch evt . eventType {
case nodeAdd :
sc . addNode ( evt )
case nodeDel :
sc . removeNode ( evt )
}
case <- sc . closeCh :
2022-09-27 19:22:54 +08:00
log . Info ( "ShardCluster watchNode quit" )
2022-04-02 17:35:29 +08:00
return
}
}
}
// watchSegments handles segment events.
func ( sc * ShardCluster ) watchSegments ( evtCh <- chan segmentEvent ) {
2022-09-27 19:22:54 +08:00
log := sc . getLogger ( )
2022-04-02 17:35:29 +08:00
for {
select {
case evt , ok := <- evtCh :
if ! ok {
2022-09-27 19:22:54 +08:00
log . Warn ( "ShardCluster segment channel closed" )
2022-04-02 17:35:29 +08:00
return
}
2022-05-11 17:39:53 +08:00
info , ok := sc . pickNode ( evt )
if ! ok {
2022-06-27 22:08:17 +08:00
log . Info ( "No node of event is in cluster, skip to process it" ,
zap . Int64s ( "nodes" , evt . nodeIDs ) )
2022-05-11 17:39:53 +08:00
continue
}
2022-04-02 17:35:29 +08:00
switch evt . eventType {
case segmentAdd :
2022-05-11 17:39:53 +08:00
sc . updateSegment ( info )
2022-04-02 17:35:29 +08:00
case segmentDel :
2022-05-11 17:39:53 +08:00
sc . removeSegment ( info )
2022-04-02 17:35:29 +08:00
}
case <- sc . closeCh :
2022-09-27 19:22:54 +08:00
log . Info ( "ShardCluster watchSegments quit" )
2022-04-02 17:35:29 +08:00
return
}
}
}
// getNode returns shallow copy of shardNode
func ( sc * ShardCluster ) getNode ( nodeID int64 ) ( * shardNode , bool ) {
sc . mut . RLock ( )
defer sc . mut . RUnlock ( )
node , ok := sc . nodes [ nodeID ]
if ! ok {
return nil , false
}
return & shardNode {
nodeID : node . nodeID ,
nodeAddr : node . nodeAddr ,
client : node . client , // shallow copy
} , true
}
// getSegment returns copy of shardSegmentInfo
2022-06-17 17:38:12 +08:00
func ( sc * ShardCluster ) getSegment ( segmentID int64 ) ( shardSegmentInfo , bool ) {
2022-04-02 17:35:29 +08:00
sc . mut . RLock ( )
defer sc . mut . RUnlock ( )
segment , ok := sc . segments [ segmentID ]
2022-06-17 17:38:12 +08:00
return segment , ok
2022-04-02 17:35:29 +08:00
}
// segmentAllocations returns node to segments mappings.
2022-04-25 11:51:46 +08:00
// calling this function also increases the reference count of related segments.
2022-06-17 17:38:12 +08:00
func ( sc * ShardCluster ) segmentAllocations ( partitionIDs [ ] int64 ) ( map [ int64 ] [ ] int64 , int64 ) {
// check cluster serviceable
if ! sc . serviceable ( ) {
log . Warn ( "request segment allocations when cluster is not serviceable" , zap . Int64 ( "collectionID" , sc . collectionID ) , zap . Int64 ( "replicaID" , sc . replicaID ) , zap . String ( "vchannelName" , sc . vchannelName ) )
return map [ int64 ] [ ] int64 { } , 0
2022-04-25 11:51:46 +08:00
}
2022-06-25 18:48:15 +08:00
sc . mutVersion . RLock ( )
defer sc . mutVersion . RUnlock ( )
2022-06-17 17:38:12 +08:00
// return allocation from current version and version id
return sc . currentVersion . GetAllocation ( partitionIDs ) , sc . currentVersion . versionID
2022-04-25 11:51:46 +08:00
}
// finishUsage decreases the inUse count of provided segments
2022-06-17 17:38:12 +08:00
func ( sc * ShardCluster ) finishUsage ( versionID int64 ) {
2022-06-25 18:48:15 +08:00
v , ok := sc . versions . Load ( versionID )
2022-11-07 17:07:03 +08:00
if ! ok {
return
2022-04-25 11:51:46 +08:00
}
2022-11-07 17:07:03 +08:00
version := v . ( * ShardClusterVersion )
version . FinishUsage ( )
// cleanup version if expired
sc . cleanupVersion ( version )
2022-04-25 11:51:46 +08:00
}
2022-09-27 19:22:54 +08:00
// LoadSegments loads segments with shardCluster.
// shard cluster shall try to loadSegments in the follower then update the allocation.
func ( sc * ShardCluster ) LoadSegments ( ctx context . Context , req * querypb . LoadSegmentsRequest ) error {
log := sc . getLogger ( )
2022-09-15 18:48:32 +08:00
// add common log fields
2022-09-27 19:22:54 +08:00
log = log . With ( zap . Int64 ( "dstNodeID" , req . GetDstNodeID ( ) ) )
2022-09-15 18:48:32 +08:00
segmentIDs := make ( [ ] int64 , 0 , len ( req . Infos ) )
for _ , info := range req . Infos {
segmentIDs = append ( segmentIDs , info . SegmentID )
}
log = log . With ( zap . Int64s ( "segmentIDs" , segmentIDs ) )
// notify follower to load segment
node , ok := sc . getNode ( req . GetDstNodeID ( ) )
if ! ok {
log . Warn ( "node not in cluster" )
return fmt . Errorf ( "node not in cluster %d" , req . GetDstNodeID ( ) )
}
2022-10-08 20:26:57 +08:00
req = proto . Clone ( req ) . ( * querypb . LoadSegmentsRequest )
req . Base . TargetID = req . GetDstNodeID ( )
2022-09-15 18:48:32 +08:00
resp , err := node . client . LoadSegments ( ctx , req )
if err != nil {
log . Warn ( "failed to dispatch load segment request" , zap . Error ( err ) )
return err
}
if resp . GetErrorCode ( ) != commonpb . ErrorCode_Success {
log . Warn ( "follower load segment failed" , zap . String ( "reason" , resp . GetReason ( ) ) )
return fmt . Errorf ( "follower %d failed to load segment, reason %s" , req . DstNodeID , resp . GetReason ( ) )
}
// update allocation
for _ , info := range req . Infos {
2022-09-28 12:10:54 +08:00
sc . updateSegment ( shardSegmentInfo {
nodeID : req . DstNodeID ,
segmentID : info . SegmentID ,
partitionID : info . PartitionID ,
state : segmentStateLoaded ,
version : req . GetVersion ( ) ,
} )
2022-09-15 18:48:32 +08:00
}
// notify handoff wait online if any
sc . segmentCond . Broadcast ( )
sc . mutVersion . Lock ( )
defer sc . mutVersion . Unlock ( )
// update shardleader allocation view
allocations := sc . currentVersion . segments . Clone ( filterNothing )
for _ , info := range req . Infos {
allocations [ info . SegmentID ] = shardSegmentInfo { nodeID : req . DstNodeID , segmentID : info . SegmentID , partitionID : info . PartitionID , state : segmentStateLoaded }
}
2022-11-07 17:07:03 +08:00
lastVersion := sc . currentVersion
2022-09-15 18:48:32 +08:00
version := NewShardClusterVersion ( sc . nextVersionID . Inc ( ) , allocations , sc . currentVersion )
sc . versions . Store ( version . versionID , version )
sc . currentVersion = version
2022-11-07 17:07:03 +08:00
sc . cleanupVersion ( lastVersion )
2022-09-15 18:48:32 +08:00
return nil
}
2022-09-27 19:22:54 +08:00
// ReleaseSegments releases segments via ShardCluster.
// ShardCluster will wait all on-going search until finished, update the current version,
// then release the segments through follower.
func ( sc * ShardCluster ) ReleaseSegments ( ctx context . Context , req * querypb . ReleaseSegmentsRequest , force bool ) error {
log := sc . getLogger ( )
2022-09-15 18:48:32 +08:00
// add common log fields
2022-09-27 19:22:54 +08:00
log = log . With ( zap . Int64s ( "segmentIDs" , req . GetSegmentIDs ( ) ) ,
zap . String ( "scope" , req . GetScope ( ) . String ( ) ) ,
zap . Bool ( "force" , force ) )
2022-09-15 18:48:32 +08:00
2022-09-27 19:22:54 +08:00
//shardCluster.forceRemoveSegment(action.GetSegmentID())
2022-09-20 12:04:49 +08:00
offlineSegments := make ( typeutil . UniqueSet )
if req . Scope != querypb . DataScope_Streaming {
2022-09-15 18:48:32 +08:00
offlineSegments . Insert ( req . GetSegmentIDs ( ) ... )
2022-09-20 12:04:49 +08:00
}
2022-09-15 18:48:32 +08:00
2022-11-07 17:07:03 +08:00
var lastVersion * ShardClusterVersion
2022-09-20 12:04:49 +08:00
var err error
func ( ) {
2022-09-15 18:48:32 +08:00
sc . mutVersion . Lock ( )
2022-09-20 12:04:49 +08:00
defer sc . mutVersion . Unlock ( )
2022-09-15 18:48:32 +08:00
var allocations SegmentsStatus
if sc . currentVersion != nil {
allocations = sc . currentVersion . segments . Clone ( func ( segmentID UniqueID , nodeID UniqueID ) bool {
2022-09-27 19:22:54 +08:00
return ( nodeID == req . NodeID || force ) && offlineSegments . Contain ( segmentID )
2022-09-15 18:48:32 +08:00
} )
}
// generate a new version
versionID := sc . nextVersionID . Inc ( )
// remove offline segments in next version
// so incoming request will not have allocation of these segments
version := NewShardClusterVersion ( versionID , allocations , sc . currentVersion )
sc . versions . Store ( versionID , version )
2022-09-27 19:22:54 +08:00
// force release means current distribution has error
if ! force {
// currentVersion shall be not nil
if sc . currentVersion != nil {
// wait for last version search done
<- sc . currentVersion . Expire ( )
2022-11-07 17:07:03 +08:00
lastVersion = sc . currentVersion
2022-09-27 19:22:54 +08:00
}
2022-09-15 18:48:32 +08:00
}
// set current version to new one
sc . currentVersion = version
2022-09-27 19:22:54 +08:00
// force release skips the release call
if force {
return
}
2022-09-20 12:04:49 +08:00
// try to release segments from nodes
node , ok := sc . getNode ( req . GetNodeID ( ) )
if ! ok {
log . Warn ( "node not in cluster" , zap . Int64 ( "nodeID" , req . NodeID ) )
err = fmt . Errorf ( "node %d not in cluster " , req . NodeID )
return
}
2022-10-08 20:26:57 +08:00
req = proto . Clone ( req ) . ( * querypb . ReleaseSegmentsRequest )
req . Base . TargetID = req . GetNodeID ( )
2022-09-20 12:04:49 +08:00
resp , rerr := node . client . ReleaseSegments ( ctx , req )
if err != nil {
log . Warn ( "failed to dispatch release segment request" , zap . Error ( err ) )
err = rerr
return
}
if resp . GetErrorCode ( ) != commonpb . ErrorCode_Success {
log . Warn ( "follower release segment failed" , zap . String ( "reason" , resp . GetReason ( ) ) )
err = fmt . Errorf ( "follower %d failed to release segment, reason %s" , req . NodeID , resp . GetReason ( ) )
}
} ( )
2022-11-07 17:07:03 +08:00
sc . cleanupVersion ( lastVersion )
2022-09-20 12:04:49 +08:00
sc . mut . Lock ( )
// do not delete segment if data scope is streaming
if req . GetScope ( ) != querypb . DataScope_Streaming {
2022-09-15 18:48:32 +08:00
for _ , segmentID := range req . SegmentIDs {
info , ok := sc . segments [ segmentID ]
if ok {
// otherwise, segment is on another node, do nothing
2022-09-27 19:22:54 +08:00
if force || info . nodeID == req . NodeID {
2022-09-15 18:48:32 +08:00
delete ( sc . segments , segmentID )
}
}
}
}
2022-10-11 09:17:22 +08:00
sc . healthCheck ( )
2022-09-20 12:04:49 +08:00
sc . mut . Unlock ( )
2022-09-15 18:48:32 +08:00
2022-09-20 12:04:49 +08:00
return err
2022-09-15 18:48:32 +08:00
}
2022-06-17 17:38:12 +08:00
// cleanupVersion clean up version from map
2022-11-07 17:07:03 +08:00
func ( sc * ShardCluster ) cleanupVersion ( version * ShardClusterVersion ) {
// last version nil, just return
if version == nil {
2022-06-17 17:38:12 +08:00
return
}
2022-11-07 17:07:03 +08:00
// if version is still current one or still in use, return
if version . current . Load ( ) || version . inUse . Load ( ) > 0 {
return
}
sc . versions . Delete ( version . versionID )
2022-04-25 11:51:46 +08:00
}
// waitSegmentsOnline waits until all provided segments is loaded.
2022-05-07 18:25:52 +08:00
func ( sc * ShardCluster ) waitSegmentsOnline ( segments [ ] shardSegmentInfo ) {
2022-04-25 11:51:46 +08:00
sc . segmentCond . L . Lock ( )
for ! sc . segmentsOnline ( segments ) {
sc . segmentCond . Wait ( )
}
sc . segmentCond . L . Unlock ( )
}
2022-05-07 18:25:52 +08:00
// checkOnline checks whether all segment info provided in online state.
func ( sc * ShardCluster ) segmentsOnline ( segments [ ] shardSegmentInfo ) bool {
2022-04-25 11:51:46 +08:00
sc . mut . RLock ( )
defer sc . mut . RUnlock ( )
2022-05-07 18:25:52 +08:00
for _ , segInfo := range segments {
segment , ok := sc . segments [ segInfo . segmentID ]
// check segment online on #specified Node#
if ! ok || segment . state != segmentStateLoaded || segment . nodeID != segInfo . nodeID {
2022-04-25 11:51:46 +08:00
return false
}
}
return true
}
2022-07-18 09:58:28 +08:00
// GetStatistics returns the statistics on the shard cluster.
func ( sc * ShardCluster ) GetStatistics ( ctx context . Context , req * querypb . GetStatisticsRequest , withStreaming withStreaming ) ( [ ] * internalpb . GetStatisticsResponse , error ) {
if ! sc . serviceable ( ) {
2022-10-17 19:39:26 +08:00
return nil , fmt . Errorf ( "ShardCluster for %s replicaID %d is not available" , sc . vchannelName , sc . replicaID )
2022-07-18 09:58:28 +08:00
}
if ! funcutil . SliceContain ( req . GetDmlChannels ( ) , sc . vchannelName ) {
return nil , fmt . Errorf ( "ShardCluster for %s does not match request channels :%v" , sc . vchannelName , req . GetDmlChannels ( ) )
}
// get node allocation and maintains the inUse reference count
segAllocs , versionID := sc . segmentAllocations ( req . GetReq ( ) . GetPartitionIDs ( ) )
defer sc . finishUsage ( versionID )
log . Debug ( "cluster segment distribution" , zap . Int ( "len" , len ( segAllocs ) ) )
for nodeID , segmentIDs := range segAllocs {
log . Debug ( "segments distribution" , zap . Int64 ( "nodeID" , nodeID ) , zap . Int64s ( "segments" , segmentIDs ) )
}
// concurrent visiting nodes
var wg sync . WaitGroup
reqCtx , cancel := context . WithCancel ( ctx )
defer cancel ( )
var err error
var resultMut sync . Mutex
results := make ( [ ] * internalpb . GetStatisticsResponse , 0 , len ( segAllocs ) ) // count(nodes) + 1(growing)
// detect corresponding streaming search is done
wg . Add ( 1 )
go func ( ) {
defer wg . Done ( )
streamErr := withStreaming ( reqCtx )
resultMut . Lock ( )
defer resultMut . Unlock ( )
if streamErr != nil {
cancel ( )
// not set cancel error
if ! errors . Is ( streamErr , context . Canceled ) {
err = fmt . Errorf ( "stream operation failed: %w" , streamErr )
}
}
} ( )
// dispatch request to followers
for nodeID , segments := range segAllocs {
nodeReq := & querypb . GetStatisticsRequest {
Req : req . GetReq ( ) ,
DmlChannels : req . GetDmlChannels ( ) ,
FromShardLeader : true ,
Scope : querypb . DataScope_Historical ,
SegmentIDs : segments ,
}
node , ok := sc . getNode ( nodeID )
if ! ok { // meta mismatch, report error
2022-10-17 19:39:26 +08:00
return nil , WrapErrShardNotAvailable ( sc . replicaID , sc . vchannelName )
2022-07-18 09:58:28 +08:00
}
wg . Add ( 1 )
go func ( ) {
defer wg . Done ( )
partialResult , nodeErr := node . client . GetStatistics ( reqCtx , nodeReq )
resultMut . Lock ( )
defer resultMut . Unlock ( )
if nodeErr != nil || partialResult . GetStatus ( ) . GetErrorCode ( ) != commonpb . ErrorCode_Success {
cancel ( )
// not set cancel error
if ! errors . Is ( nodeErr , context . Canceled ) {
err = fmt . Errorf ( "GetStatistic %d failed, reason %s err %w" , node . nodeID , partialResult . GetStatus ( ) . GetReason ( ) , nodeErr )
}
return
}
results = append ( results , partialResult )
} ( )
}
wg . Wait ( )
if err != nil {
log . Error ( err . Error ( ) )
return nil , err
}
return results , nil
}
2022-04-02 17:35:29 +08:00
// Search preforms search operation on shard cluster.
2022-06-17 17:38:12 +08:00
func ( sc * ShardCluster ) Search ( ctx context . Context , req * querypb . SearchRequest , withStreaming withStreaming ) ( [ ] * internalpb . SearchResults , error ) {
if ! sc . serviceable ( ) {
2022-10-17 19:39:26 +08:00
err := WrapErrShardNotAvailable ( sc . replicaID , sc . vchannelName )
2022-10-25 19:29:36 +08:00
log . Warn ( "failed to search on shard" ,
2022-10-17 19:39:26 +08:00
zap . Int64 ( "replicaID" , sc . replicaID ) ,
zap . String ( "channel" , sc . vchannelName ) ,
zap . Int32 ( "state" , sc . state . Load ( ) ) ,
zap . Any ( "version" , sc . currentVersion ) ,
zap . Error ( err ) ,
)
return nil , err
2022-04-02 17:35:29 +08:00
}
2022-07-06 15:06:21 +08:00
if ! funcutil . SliceContain ( req . GetDmlChannels ( ) , sc . vchannelName ) {
return nil , fmt . Errorf ( "ShardCluster for %s does not match request channels :%v" , sc . vchannelName , req . GetDmlChannels ( ) )
2022-04-02 17:35:29 +08:00
}
2022-04-25 11:51:46 +08:00
// get node allocation and maintains the inUse reference count
2022-06-17 17:38:12 +08:00
segAllocs , versionID := sc . segmentAllocations ( req . GetReq ( ) . GetPartitionIDs ( ) )
defer sc . finishUsage ( versionID )
2022-04-02 17:35:29 +08:00
2022-09-15 18:48:32 +08:00
log . Debug ( "cluster segment distribution" , zap . Int ( "len" , len ( segAllocs ) ) , zap . Int64s ( "partitionIDs" , req . GetReq ( ) . GetPartitionIDs ( ) ) )
2022-04-20 16:15:41 +08:00
for nodeID , segmentIDs := range segAllocs {
log . Debug ( "segments distribution" , zap . Int64 ( "nodeID" , nodeID ) , zap . Int64s ( "segments" , segmentIDs ) )
}
2022-04-02 17:35:29 +08:00
// concurrent visiting nodes
var wg sync . WaitGroup
reqCtx , cancel := context . WithCancel ( ctx )
defer cancel ( )
var err error
var resultMut sync . Mutex
2022-06-17 17:38:12 +08:00
results := make ( [ ] * internalpb . SearchResults , 0 , len ( segAllocs ) ) // count(nodes) + 1(growing)
// detect corresponding streaming search is done
wg . Add ( 1 )
go func ( ) {
defer wg . Done ( )
streamErr := withStreaming ( reqCtx )
resultMut . Lock ( )
defer resultMut . Unlock ( )
if streamErr != nil {
2022-09-28 12:08:59 +08:00
if err == nil {
2022-06-17 17:38:12 +08:00
err = fmt . Errorf ( "stream operation failed: %w" , streamErr )
}
2022-09-28 12:08:59 +08:00
cancel ( )
2022-06-17 17:38:12 +08:00
}
} ( )
2022-04-02 17:35:29 +08:00
2022-06-17 17:38:12 +08:00
// dispatch request to followers
2022-04-02 17:35:29 +08:00
for nodeID , segments := range segAllocs {
2022-07-06 15:06:21 +08:00
nodeReq := & querypb . SearchRequest {
Req : req . Req ,
DmlChannels : req . DmlChannels ,
FromShardLeader : true ,
Scope : querypb . DataScope_Historical ,
SegmentIDs : segments ,
}
2022-04-02 17:35:29 +08:00
node , ok := sc . getNode ( nodeID )
if ! ok { // meta dismatch, report error
2022-10-17 19:39:26 +08:00
return nil , fmt . Errorf ( "%w, node %d not found" ,
WrapErrShardNotAvailable ( sc . replicaID , sc . vchannelName ) ,
nodeID ,
)
2022-04-02 17:35:29 +08:00
}
wg . Add ( 1 )
go func ( ) {
defer wg . Done ( )
partialResult , nodeErr := node . client . Search ( reqCtx , nodeReq )
resultMut . Lock ( )
defer resultMut . Unlock ( )
if nodeErr != nil || partialResult . GetStatus ( ) . GetErrorCode ( ) != commonpb . ErrorCode_Success {
2022-09-28 12:08:59 +08:00
if err == nil {
2022-06-17 17:38:12 +08:00
err = fmt . Errorf ( "Search %d failed, reason %s err %w" , node . nodeID , partialResult . GetStatus ( ) . GetReason ( ) , nodeErr )
}
2022-09-28 12:08:59 +08:00
cancel ( )
2022-04-02 17:35:29 +08:00
return
}
results = append ( results , partialResult )
} ( )
}
wg . Wait ( )
if err != nil {
2022-10-25 17:53:32 +08:00
log . Error ( "failed to do search" ,
zap . Int64 ( "msgID" , req . GetReq ( ) . GetBase ( ) . GetMsgID ( ) ) ,
zap . Int64 ( "sourceID" , req . GetReq ( ) . GetBase ( ) . GetSourceID ( ) ) ,
zap . Strings ( "channels" , req . GetDmlChannels ( ) ) ,
zap . Int64s ( "segmentIDs" , req . GetSegmentIDs ( ) ) ,
zap . Error ( err ) )
2022-04-02 17:35:29 +08:00
return nil , err
}
return results , nil
}
// Query performs query operation on shard cluster.
2022-06-17 17:38:12 +08:00
func ( sc * ShardCluster ) Query ( ctx context . Context , req * querypb . QueryRequest , withStreaming withStreaming ) ( [ ] * internalpb . RetrieveResults , error ) {
if ! sc . serviceable ( ) {
2022-10-17 19:39:26 +08:00
return nil , WrapErrShardNotAvailable ( sc . replicaID , sc . vchannelName )
2022-04-02 17:35:29 +08:00
}
// handles only the dml channel part, segment ids is dispatch by cluster itself
2022-07-06 15:06:21 +08:00
if ! funcutil . SliceContain ( req . GetDmlChannels ( ) , sc . vchannelName ) {
return nil , fmt . Errorf ( "ShardCluster for %s does not match to request channels :%v" , sc . vchannelName , req . GetDmlChannels ( ) )
2022-04-02 17:35:29 +08:00
}
2022-04-25 11:51:46 +08:00
// get node allocation and maintains the inUse reference count
2022-06-17 17:38:12 +08:00
segAllocs , versionID := sc . segmentAllocations ( req . GetReq ( ) . GetPartitionIDs ( ) )
defer sc . finishUsage ( versionID )
2022-04-02 17:35:29 +08:00
// concurrent visiting nodes
var wg sync . WaitGroup
reqCtx , cancel := context . WithCancel ( ctx )
defer cancel ( )
var err error
var resultMut sync . Mutex
results := make ( [ ] * internalpb . RetrieveResults , 0 , len ( segAllocs ) + 1 ) // count(nodes) + 1(growing)
2022-06-17 17:38:12 +08:00
// detect corresponding streaming query is done
wg . Add ( 1 )
go func ( ) {
defer wg . Done ( )
streamErr := withStreaming ( reqCtx )
if streamErr != nil {
2022-09-28 12:08:59 +08:00
if err == nil {
2022-06-17 17:38:12 +08:00
err = fmt . Errorf ( "stream operation failed: %w" , streamErr )
}
2022-09-28 12:08:59 +08:00
cancel ( )
2022-06-17 17:38:12 +08:00
}
} ( )
// dispatch request to followers
2022-04-02 17:35:29 +08:00
for nodeID , segments := range segAllocs {
2022-07-06 15:06:21 +08:00
nodeReq := & querypb . QueryRequest {
Req : req . Req ,
FromShardLeader : true ,
SegmentIDs : segments ,
Scope : querypb . DataScope_Historical ,
DmlChannels : req . DmlChannels ,
}
2022-04-02 17:35:29 +08:00
node , ok := sc . getNode ( nodeID )
if ! ok { // meta dismatch, report error
2022-10-17 19:39:26 +08:00
return nil , WrapErrShardNotAvailable ( sc . replicaID , sc . vchannelName )
2022-04-02 17:35:29 +08:00
}
wg . Add ( 1 )
go func ( ) {
defer wg . Done ( )
partialResult , nodeErr := node . client . Query ( reqCtx , nodeReq )
resultMut . Lock ( )
defer resultMut . Unlock ( )
if nodeErr != nil || partialResult . GetStatus ( ) . GetErrorCode ( ) != commonpb . ErrorCode_Success {
err = fmt . Errorf ( "Query %d failed, reason %s err %w" , node . nodeID , partialResult . GetStatus ( ) . GetReason ( ) , nodeErr )
2022-09-28 12:08:59 +08:00
cancel ( )
2022-04-02 17:35:29 +08:00
return
}
results = append ( results , partialResult )
} ( )
}
wg . Wait ( )
if err != nil {
2022-05-07 10:27:51 +08:00
log . Error ( err . Error ( ) )
2022-04-02 17:35:29 +08:00
return nil , err
}
return results , nil
}
2022-09-15 18:48:32 +08:00
func ( sc * ShardCluster ) GetSegmentInfos ( ) [ ] shardSegmentInfo {
2022-10-18 19:47:36 +08:00
sc . mut . RLock ( )
defer sc . mut . RUnlock ( )
2022-09-15 18:48:32 +08:00
ret := make ( [ ] shardSegmentInfo , 0 , len ( sc . segments ) )
for _ , info := range sc . segments {
ret = append ( ret , info )
}
return ret
}
func ( sc * ShardCluster ) getVersion ( ) int64 {
2022-10-18 19:47:36 +08:00
sc . mutVersion . RLock ( )
defer sc . mutVersion . RUnlock ( )
2022-09-15 18:48:32 +08:00
return sc . version
}