2021-04-19 10:09:43 +08:00
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed 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.
2021-01-15 14:38:36 +08:00
package proxynode
2020-11-03 14:53:36 +08:00
import (
2021-06-03 15:03:34 +08:00
"bytes"
2021-02-04 19:34:35 +08:00
"context"
2021-06-03 15:03:34 +08:00
"encoding/binary"
2021-03-08 19:39:36 +08:00
"errors"
2021-02-08 14:20:29 +08:00
"fmt"
2020-11-26 16:01:31 +08:00
"math"
2021-06-03 15:03:34 +08:00
"reflect"
2021-03-22 16:36:10 +08:00
"regexp"
2021-03-25 10:14:09 +08:00
"runtime"
2021-05-25 19:53:15 +08:00
"sort"
2020-11-26 16:01:31 +08:00
"strconv"
2021-03-26 11:19:02 +08:00
"time"
2021-05-25 19:53:15 +08:00
"unsafe"
2021-03-26 11:19:02 +08:00
2021-04-29 16:48:06 +08:00
"github.com/milvus-io/milvus/internal/proto/planpb"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/util/funcutil"
2020-11-26 16:01:31 +08:00
2021-03-08 19:39:36 +08:00
"go.uber.org/zap"
2021-03-05 10:15:27 +08:00
2020-11-26 16:01:31 +08:00
"github.com/golang/protobuf/proto"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/schemapb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/typeutil"
2020-11-03 14:53:36 +08:00
)
2021-02-23 09:58:06 +08:00
const (
InsertTaskName = "InsertTask"
CreateCollectionTaskName = "CreateCollectionTask"
DropCollectionTaskName = "DropCollectionTask"
SearchTaskName = "SearchTask"
2021-05-19 18:45:15 +08:00
RetrieveTaskName = "RetrieveTask"
2021-04-29 16:48:06 +08:00
AnnsFieldKey = "anns_field"
TopKKey = "topk"
MetricTypeKey = "metric_type"
SearchParamsKey = "params"
2021-02-23 09:58:06 +08:00
HasCollectionTaskName = "HasCollectionTask"
DescribeCollectionTaskName = "DescribeCollectionTask"
GetCollectionStatisticsTaskName = "GetCollectionStatisticsTask"
2021-05-10 17:39:08 +08:00
GetPartitionStatisticsTaskName = "GetPartitionStatisticsTask"
2021-02-23 09:58:06 +08:00
ShowCollectionTaskName = "ShowCollectionTask"
CreatePartitionTaskName = "CreatePartitionTask"
DropPartitionTaskName = "DropPartitionTask"
HasPartitionTaskName = "HasPartitionTask"
ShowPartitionTaskName = "ShowPartitionTask"
CreateIndexTaskName = "CreateIndexTask"
DescribeIndexTaskName = "DescribeIndexTask"
DropIndexTaskName = "DropIndexTask"
GetIndexStateTaskName = "GetIndexStateTask"
2021-04-28 11:15:28 +08:00
GetIndexBuildProgressTaskName = "GetIndexBuildProgressTask"
2021-02-23 09:58:06 +08:00
FlushTaskName = "FlushTask"
LoadCollectionTaskName = "LoadCollectionTask"
ReleaseCollectionTaskName = "ReleaseCollectionTask"
LoadPartitionTaskName = "LoadPartitionTask"
ReleasePartitionTaskName = "ReleasePartitionTask"
)
2020-11-03 14:53:36 +08:00
type task interface {
2021-03-25 14:41:46 +08:00
TraceCtx ( ) context . Context
2020-11-23 16:52:17 +08:00
ID ( ) UniqueID // return ReqID
SetID ( uid UniqueID ) // set ReqID
2021-02-23 09:58:06 +08:00
Name ( ) string
2021-01-16 15:06:19 +08:00
Type ( ) commonpb . MsgType
2020-11-05 18:01:33 +08:00
BeginTs ( ) Timestamp
EndTs ( ) Timestamp
2020-11-04 17:58:43 +08:00
SetTs ( ts Timestamp )
2021-01-22 09:36:18 +08:00
OnEnqueue ( ) error
2021-02-23 09:58:06 +08:00
PreExecute ( ctx context . Context ) error
Execute ( ctx context . Context ) error
PostExecute ( ctx context . Context ) error
2020-11-03 14:53:36 +08:00
WaitToFinish ( ) error
2020-11-05 18:01:33 +08:00
Notify ( err error )
2020-11-03 14:53:36 +08:00
}
2021-05-31 11:40:31 +08:00
type dmlTask interface {
task
2021-06-15 10:19:38 +08:00
getChannels ( ) ( [ ] vChan , error )
getPChanStats ( ) ( map [ pChan ] pChanStatistics , error )
2021-06-02 10:17:32 +08:00
}
2020-11-07 16:18:23 +08:00
type BaseInsertTask = msgstream . InsertMsg
2020-11-05 18:01:33 +08:00
type InsertTask struct {
2020-11-07 16:18:23 +08:00
BaseInsertTask
2021-06-03 15:03:34 +08:00
req * milvuspb . InsertRequest
2020-11-17 20:00:23 +08:00
Condition
2021-03-08 10:09:48 +08:00
ctx context . Context
dataService types . DataService
result * milvuspb . InsertResponse
rowIDAllocator * allocator . IDAllocator
2021-05-25 19:53:15 +08:00
segIDAssigner * SegIDAssigner
2021-05-27 17:09:50 +08:00
chMgr channelsMgr
2021-05-31 17:28:31 +08:00
chTicker channelsTimeTicker
2021-06-15 10:19:38 +08:00
vChannels [ ] vChan
pChannels [ ] pChan
2020-11-05 18:01:33 +08:00
}
2021-03-25 14:41:46 +08:00
func ( it * InsertTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return it . ctx
}
func ( it * InsertTask ) ID ( ) UniqueID {
return it . Base . MsgID
2021-01-22 09:36:18 +08:00
}
2020-11-23 16:52:17 +08:00
func ( it * InsertTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
it . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( it * InsertTask ) Name ( ) string {
return InsertTaskName
}
func ( it * InsertTask ) Type ( ) commonpb . MsgType {
return it . Base . MsgType
}
func ( it * InsertTask ) BeginTs ( ) Timestamp {
return it . BeginTimestamp
}
2020-11-05 18:01:33 +08:00
func ( it * InsertTask ) SetTs ( ts Timestamp ) {
2020-11-26 16:01:31 +08:00
it . BeginTimestamp = ts
it . EndTimestamp = ts
2020-11-05 18:01:33 +08:00
}
func ( it * InsertTask ) EndTs ( ) Timestamp {
2020-11-26 16:01:31 +08:00
return it . EndTimestamp
2020-11-05 18:01:33 +08:00
}
2021-06-15 10:19:38 +08:00
func ( it * InsertTask ) getPChanStats ( ) ( map [ pChan ] pChanStatistics , error ) {
ret := make ( map [ pChan ] pChanStatistics )
channels , err := it . getChannels ( )
2021-05-31 11:40:31 +08:00
if err != nil {
2021-06-15 10:19:38 +08:00
return ret , err
2021-05-31 11:40:31 +08:00
}
2021-06-15 10:19:38 +08:00
beginTs := it . BeginTs ( )
endTs := it . EndTs ( )
for _ , channel := range channels {
ret [ channel ] = pChanStatistics {
minTs : beginTs ,
maxTs : endTs ,
2021-05-31 11:40:31 +08:00
}
}
2021-06-15 10:19:38 +08:00
return ret , nil
}
func ( it * InsertTask ) getChannels ( ) ( [ ] pChan , error ) {
collID , err := globalMetaCache . GetCollectionID ( it . ctx , it . CollectionName )
2021-05-31 11:40:31 +08:00
if err != nil {
2021-06-15 10:19:38 +08:00
return nil , err
2021-05-31 11:40:31 +08:00
}
2021-06-15 10:19:38 +08:00
var channels [ ] pChan
channels , err = it . chMgr . getChannels ( collID )
if err != nil {
err = it . chMgr . createDMLMsgStream ( collID )
if err != nil {
return nil , err
2021-05-31 11:40:31 +08:00
}
2021-06-15 10:19:38 +08:00
channels , err = it . chMgr . getChannels ( collID )
2021-05-31 11:40:31 +08:00
}
2021-06-15 10:19:38 +08:00
return channels , err
2021-05-31 11:40:31 +08:00
}
2021-02-23 09:58:06 +08:00
func ( it * InsertTask ) OnEnqueue ( ) error {
2021-03-22 19:28:43 +08:00
it . BaseInsertTask . InsertRequest . Base = & commonpb . MsgBase { }
2021-02-23 09:58:06 +08:00
return nil
2020-11-05 18:01:33 +08:00
}
2021-06-03 15:03:34 +08:00
func ( it * InsertTask ) transferColumnBasedRequestToRowBasedData ( ) error {
dTypes := make ( [ ] schemapb . DataType , 0 , len ( it . req . FieldsData ) )
datas := make ( [ ] [ ] interface { } , 0 , len ( it . req . FieldsData ) )
rowNum := 0
appendScalarField := func ( getDataFunc func ( ) interface { } ) error {
fieldDatas := reflect . ValueOf ( getDataFunc ( ) )
if rowNum != 0 && rowNum != fieldDatas . Len ( ) {
return errors . New ( "the row num of different column is not equal" )
}
rowNum = fieldDatas . Len ( )
datas = append ( datas , make ( [ ] interface { } , 0 , rowNum ) )
idx := len ( datas ) - 1
for i := 0 ; i < rowNum ; i ++ {
datas [ idx ] = append ( datas [ idx ] , fieldDatas . Index ( i ) . Interface ( ) )
}
return nil
}
appendFloatVectorField := func ( fDatas [ ] float32 , dim int64 ) error {
l := len ( fDatas )
if int64 ( l ) % dim != 0 {
return errors . New ( "invalid vectors" )
}
r := int64 ( l ) / dim
if rowNum != 0 && rowNum != int ( r ) {
return errors . New ( "the row num of different column is not equal" )
}
rowNum = int ( r )
datas = append ( datas , make ( [ ] interface { } , 0 , rowNum ) )
idx := len ( datas ) - 1
vector := make ( [ ] float32 , 0 , dim )
for i := 0 ; i < l ; i ++ {
vector = append ( vector , fDatas [ i ] )
if int64 ( i + 1 ) % dim == 0 {
datas [ idx ] = append ( datas [ idx ] , vector )
vector = make ( [ ] float32 , 0 , dim )
}
}
return nil
}
appendBinaryVectorField := func ( bDatas [ ] byte , dim int64 ) error {
l := len ( bDatas )
if dim % 8 != 0 {
return errors . New ( "invalid dim" )
}
if ( 8 * int64 ( l ) ) % dim != 0 {
return errors . New ( "invalid vectors" )
}
r := ( 8 * int64 ( l ) ) / dim
if rowNum != 0 && rowNum != int ( r ) {
return errors . New ( "the row num of different column is not equal" )
}
rowNum = int ( r )
datas = append ( datas , make ( [ ] interface { } , 0 , rowNum ) )
idx := len ( datas ) - 1
vector := make ( [ ] byte , 0 , dim )
for i := 0 ; i < l ; i ++ {
vector = append ( vector , bDatas [ i ] )
if ( 8 * int64 ( i + 1 ) ) % dim == 0 {
datas [ idx ] = append ( datas [ idx ] , vector )
vector = make ( [ ] byte , 0 , dim )
}
}
return nil
}
for _ , field := range it . req . FieldsData {
switch field . Field . ( type ) {
case * schemapb . FieldData_Scalars :
scalarField := field . GetScalars ( )
switch scalarField . Data . ( type ) {
case * schemapb . ScalarField_BoolData :
err := appendScalarField ( func ( ) interface { } {
return scalarField . GetBoolData ( ) . Data
} )
if err != nil {
return err
}
case * schemapb . ScalarField_IntData :
err := appendScalarField ( func ( ) interface { } {
return scalarField . GetIntData ( ) . Data
} )
if err != nil {
return err
}
case * schemapb . ScalarField_LongData :
err := appendScalarField ( func ( ) interface { } {
return scalarField . GetLongData ( ) . Data
} )
if err != nil {
return err
}
case * schemapb . ScalarField_FloatData :
err := appendScalarField ( func ( ) interface { } {
return scalarField . GetFloatData ( ) . Data
} )
if err != nil {
return err
}
case * schemapb . ScalarField_DoubleData :
err := appendScalarField ( func ( ) interface { } {
return scalarField . GetDoubleData ( ) . Data
} )
if err != nil {
return err
}
case * schemapb . ScalarField_BytesData :
return errors . New ( "bytes field is not supported now" )
case * schemapb . ScalarField_StringData :
return errors . New ( "string field is not supported now" )
case nil :
continue
default :
continue
}
case * schemapb . FieldData_Vectors :
vectorField := field . GetVectors ( )
switch vectorField . Data . ( type ) {
case * schemapb . VectorField_FloatVector :
floatVectorFieldData := vectorField . GetFloatVector ( ) . Data
dim := vectorField . GetDim ( )
err := appendFloatVectorField ( floatVectorFieldData , dim )
if err != nil {
return err
}
case * schemapb . VectorField_BinaryVector :
binaryVectorFieldData := vectorField . GetBinaryVector ( )
dim := vectorField . GetDim ( )
err := appendBinaryVectorField ( binaryVectorFieldData , dim )
if err != nil {
return err
}
case nil :
continue
default :
continue
}
case nil :
continue
default :
continue
}
dTypes = append ( dTypes , field . Type )
}
it . RowData = make ( [ ] * commonpb . Blob , 0 , rowNum )
l := len ( dTypes )
// TODO(dragondriver): big endian or little endian?
endian := binary . LittleEndian
2021-06-05 16:21:36 +08:00
printed := false
2021-06-03 15:03:34 +08:00
for i := 0 ; i < rowNum ; i ++ {
blob := & commonpb . Blob {
Value : make ( [ ] byte , 0 ) ,
}
for j := 0 ; j < l ; j ++ {
var buffer bytes . Buffer
switch dTypes [ j ] {
case schemapb . DataType_Bool :
d := datas [ j ] [ i ] . ( bool )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_Int8 :
d := datas [ j ] [ i ] . ( int8 )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_Int16 :
d := datas [ j ] [ i ] . ( int16 )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_Int32 :
d := datas [ j ] [ i ] . ( int32 )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_Int64 :
d := datas [ j ] [ i ] . ( int64 )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_Float :
d := datas [ j ] [ i ] . ( float32 )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_Double :
d := datas [ j ] [ i ] . ( float64 )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_FloatVector :
d := datas [ j ] [ i ] . ( [ ] float32 )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
case schemapb . DataType_BinaryVector :
d := datas [ j ] [ i ] . ( [ ] byte )
err := binary . Write ( & buffer , endian , d )
if err != nil {
log . Warn ( "ConvertData" , zap . Error ( err ) )
}
blob . Value = append ( blob . Value , buffer . Bytes ( ) ... )
default :
log . Warn ( "unsupported data type" )
}
}
2021-06-05 16:21:36 +08:00
if ! printed {
log . Debug ( "ProxyNode, transform" , zap . Any ( "ID" , it . ID ( ) ) , zap . Any ( "BlobLen" , len ( blob . Value ) ) , zap . Any ( "dTypes" , dTypes ) )
printed = true
}
2021-06-03 15:03:34 +08:00
it . RowData = append ( it . RowData , blob )
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( it * InsertTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
it . Base . MsgType = commonpb . MsgType_Insert
2021-01-28 20:51:44 +08:00
it . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2020-11-26 16:01:31 +08:00
collectionName := it . BaseInsertTask . CollectionName
if err := ValidateCollectionName ( collectionName ) ; err != nil {
return err
}
2021-01-18 19:32:08 +08:00
partitionTag := it . BaseInsertTask . PartitionName
2020-11-26 16:01:31 +08:00
if err := ValidatePartitionTag ( partitionTag , true ) ; err != nil {
return err
}
2021-06-03 15:03:34 +08:00
err := it . transferColumnBasedRequestToRowBasedData ( )
if err != nil {
return err
}
rowNum := len ( it . RowData )
it . Timestamps = make ( [ ] uint64 , rowNum )
for index := range it . Timestamps {
it . Timestamps [ index ] = it . BeginTimestamp
}
2020-11-05 18:01:33 +08:00
return nil
}
2021-05-25 19:53:15 +08:00
func ( it * InsertTask ) _assignSegmentID ( stream msgstream . MsgStream , pack * msgstream . MsgPack ) ( * msgstream . MsgPack , error ) {
newPack := & msgstream . MsgPack {
BeginTs : pack . BeginTs ,
EndTs : pack . EndTs ,
StartPositions : pack . StartPositions ,
EndPositions : pack . EndPositions ,
Msgs : nil ,
}
tsMsgs := pack . Msgs
hashKeys := stream . ComputeProduceChannelIndexes ( tsMsgs )
reqID := it . Base . MsgID
channelCountMap := make ( map [ int32 ] uint32 ) // channelID to count
channelMaxTSMap := make ( map [ int32 ] Timestamp ) // channelID to max Timestamp
channelNames := stream . GetProduceChannels ( )
log . Debug ( "_assignSemgentID, produceChannels:" , zap . Any ( "Channels" , channelNames ) )
for i , request := range tsMsgs {
if request . Type ( ) != commonpb . MsgType_Insert {
return nil , fmt . Errorf ( "msg's must be Insert" )
}
insertRequest , ok := request . ( * msgstream . InsertMsg )
if ! ok {
return nil , fmt . Errorf ( "msg's must be Insert" )
}
keys := hashKeys [ i ]
timestampLen := len ( insertRequest . Timestamps )
rowIDLen := len ( insertRequest . RowIDs )
rowDataLen := len ( insertRequest . RowData )
keysLen := len ( keys )
if keysLen != timestampLen || keysLen != rowIDLen || keysLen != rowDataLen {
return nil , fmt . Errorf ( "the length of hashValue, timestamps, rowIDs, RowData are not equal" )
}
for idx , channelID := range keys {
channelCountMap [ channelID ] ++
if _ , ok := channelMaxTSMap [ channelID ] ; ! ok {
channelMaxTSMap [ channelID ] = typeutil . ZeroTimestamp
}
ts := insertRequest . Timestamps [ idx ]
if channelMaxTSMap [ channelID ] < ts {
channelMaxTSMap [ channelID ] = ts
}
}
}
reqSegCountMap := make ( map [ int32 ] map [ UniqueID ] uint32 )
for channelID , count := range channelCountMap {
ts , ok := channelMaxTSMap [ channelID ]
if ! ok {
ts = typeutil . ZeroTimestamp
log . Debug ( "Warning: did not get max Timestamp!" )
}
channelName := channelNames [ channelID ]
if channelName == "" {
return nil , fmt . Errorf ( "ProxyNode, repack_func, can not found channelName" )
}
mapInfo , err := it . segIDAssigner . GetSegmentID ( it . CollectionID , it . PartitionID , channelName , count , ts )
if err != nil {
return nil , err
}
reqSegCountMap [ channelID ] = make ( map [ UniqueID ] uint32 )
reqSegCountMap [ channelID ] = mapInfo
log . Debug ( "ProxyNode" , zap . Int64 ( "repackFunc, reqSegCountMap, reqID" , reqID ) , zap . Any ( "mapinfo" , mapInfo ) )
}
reqSegAccumulateCountMap := make ( map [ int32 ] [ ] uint32 )
reqSegIDMap := make ( map [ int32 ] [ ] UniqueID )
reqSegAllocateCounter := make ( map [ int32 ] uint32 )
for channelID , segInfo := range reqSegCountMap {
reqSegAllocateCounter [ channelID ] = 0
keys := make ( [ ] UniqueID , len ( segInfo ) )
i := 0
for key := range segInfo {
keys [ i ] = key
i ++
}
sort . Slice ( keys , func ( i , j int ) bool { return keys [ i ] < keys [ j ] } )
accumulate := uint32 ( 0 )
for _ , key := range keys {
accumulate += segInfo [ key ]
if _ , ok := reqSegAccumulateCountMap [ channelID ] ; ! ok {
reqSegAccumulateCountMap [ channelID ] = make ( [ ] uint32 , 0 )
}
reqSegAccumulateCountMap [ channelID ] = append (
reqSegAccumulateCountMap [ channelID ] ,
accumulate ,
)
if _ , ok := reqSegIDMap [ channelID ] ; ! ok {
reqSegIDMap [ channelID ] = make ( [ ] UniqueID , 0 )
}
reqSegIDMap [ channelID ] = append (
reqSegIDMap [ channelID ] ,
key ,
)
}
}
var getSegmentID = func ( channelID int32 ) UniqueID {
reqSegAllocateCounter [ channelID ] ++
cur := reqSegAllocateCounter [ channelID ]
accumulateSlice := reqSegAccumulateCountMap [ channelID ]
segIDSlice := reqSegIDMap [ channelID ]
for index , count := range accumulateSlice {
if cur <= count {
return segIDSlice [ index ]
}
}
log . Warn ( "Can't Found SegmentID" )
return 0
}
factor := 10
threshold := Params . PulsarMaxMessageSize / factor
log . Debug ( "ProxyNode" , zap . Int ( "threshold of message size: " , threshold ) )
// not accurate
getFixedSizeOfInsertMsg := func ( msg * msgstream . InsertMsg ) int {
size := 0
size += int ( unsafe . Sizeof ( * msg . Base ) )
size += int ( unsafe . Sizeof ( msg . DbName ) )
size += int ( unsafe . Sizeof ( msg . CollectionName ) )
size += int ( unsafe . Sizeof ( msg . PartitionName ) )
size += int ( unsafe . Sizeof ( msg . DbID ) )
size += int ( unsafe . Sizeof ( msg . CollectionID ) )
size += int ( unsafe . Sizeof ( msg . PartitionID ) )
size += int ( unsafe . Sizeof ( msg . SegmentID ) )
size += int ( unsafe . Sizeof ( msg . ChannelID ) )
size += int ( unsafe . Sizeof ( msg . Timestamps ) )
size += int ( unsafe . Sizeof ( msg . RowIDs ) )
return size
}
result := make ( map [ int32 ] msgstream . TsMsg )
curMsgSizeMap := make ( map [ int32 ] int )
for i , request := range tsMsgs {
insertRequest := request . ( * msgstream . InsertMsg )
keys := hashKeys [ i ]
collectionName := insertRequest . CollectionName
collectionID := insertRequest . CollectionID
partitionID := insertRequest . PartitionID
partitionName := insertRequest . PartitionName
proxyID := insertRequest . Base . SourceID
for index , key := range keys {
ts := insertRequest . Timestamps [ index ]
rowID := insertRequest . RowIDs [ index ]
row := insertRequest . RowData [ index ]
segmentID := getSegmentID ( key )
_ , ok := result [ key ]
if ! ok {
sliceRequest := internalpb . InsertRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_Insert ,
MsgID : reqID ,
Timestamp : ts ,
SourceID : proxyID ,
} ,
CollectionID : collectionID ,
PartitionID : partitionID ,
CollectionName : collectionName ,
PartitionName : partitionName ,
SegmentID : segmentID ,
// todo rename to ChannelName
ChannelID : channelNames [ key ] ,
}
insertMsg := & msgstream . InsertMsg {
BaseMsg : msgstream . BaseMsg {
Ctx : request . TraceCtx ( ) ,
} ,
InsertRequest : sliceRequest ,
}
result [ key ] = insertMsg
curMsgSizeMap [ key ] = getFixedSizeOfInsertMsg ( insertMsg )
}
curMsg := result [ key ] . ( * msgstream . InsertMsg )
curMsgSize := curMsgSizeMap [ key ]
curMsg . HashValues = append ( curMsg . HashValues , insertRequest . HashValues [ index ] )
curMsg . Timestamps = append ( curMsg . Timestamps , ts )
curMsg . RowIDs = append ( curMsg . RowIDs , rowID )
curMsg . RowData = append ( curMsg . RowData , row )
curMsgSize += 4 + 8 + int ( unsafe . Sizeof ( row . Value ) )
curMsgSize += len ( row . Value )
if curMsgSize >= threshold {
newPack . Msgs = append ( newPack . Msgs , curMsg )
delete ( result , key )
curMsgSize = 0
}
curMsgSizeMap [ key ] = curMsgSize
}
}
for _ , msg := range result {
if msg != nil {
newPack . Msgs = append ( newPack . Msgs , msg )
}
}
return newPack , nil
}
2021-02-23 09:58:06 +08:00
func ( it * InsertTask ) Execute ( ctx context . Context ) error {
2020-11-20 17:53:31 +08:00
collectionName := it . BaseInsertTask . CollectionName
2021-02-26 17:44:24 +08:00
collSchema , err := globalMetaCache . GetCollectionSchema ( ctx , collectionName )
2021-06-05 16:21:36 +08:00
log . Debug ( "ProxyNode Insert" , zap . Any ( "collSchema" , collSchema ) )
2021-01-31 14:55:36 +08:00
if err != nil {
return err
2020-11-20 17:53:31 +08:00
}
2021-01-31 14:55:36 +08:00
autoID := collSchema . AutoID
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , collectionName )
2021-01-31 14:55:36 +08:00
if err != nil {
2020-11-20 17:53:31 +08:00
return err
}
2021-01-31 14:55:36 +08:00
it . CollectionID = collID
2021-02-03 17:30:10 +08:00
var partitionID UniqueID
if len ( it . PartitionName ) > 0 {
2021-02-26 17:44:24 +08:00
partitionID , err = globalMetaCache . GetPartitionID ( ctx , collectionName , it . PartitionName )
2021-02-03 17:30:10 +08:00
if err != nil {
return err
}
} else {
2021-04-13 10:04:39 +08:00
partitionID , err = globalMetaCache . GetPartitionID ( ctx , collectionName , Params . DefaultPartitionName )
2021-02-03 17:30:10 +08:00
if err != nil {
return err
}
2021-01-31 14:55:36 +08:00
}
it . PartitionID = partitionID
2020-11-30 12:08:47 +08:00
var rowIDBegin UniqueID
var rowIDEnd UniqueID
2020-12-03 19:00:11 +08:00
rowNums := len ( it . BaseInsertTask . RowData )
rowIDBegin , rowIDEnd , _ = it . rowIDAllocator . Alloc ( uint32 ( rowNums ) )
2021-02-03 17:30:10 +08:00
2020-12-03 19:00:11 +08:00
it . BaseInsertTask . RowIDs = make ( [ ] UniqueID , rowNums )
for i := rowIDBegin ; i < rowIDEnd ; i ++ {
offset := i - rowIDBegin
it . BaseInsertTask . RowIDs [ offset ] = i
}
if autoID {
2020-11-30 19:38:23 +08:00
if it . HashValues == nil || len ( it . HashValues ) == 0 {
it . HashValues = make ( [ ] uint32 , 0 )
}
2020-12-03 19:00:11 +08:00
for _ , rowID := range it . RowIDs {
hashValue , _ := typeutil . Hash32Int64 ( rowID )
2020-11-30 19:38:23 +08:00
it . HashValues = append ( it . HashValues , hashValue )
2020-11-20 17:53:31 +08:00
}
}
2020-11-07 16:18:23 +08:00
var tsMsg msgstream . TsMsg = & it . BaseInsertTask
2021-03-25 14:41:46 +08:00
it . BaseMsg . Ctx = ctx
msgPack := msgstream . MsgPack {
2020-11-05 18:01:33 +08:00
BeginTs : it . BeginTs ( ) ,
EndTs : it . EndTs ( ) ,
2020-11-17 14:10:07 +08:00
Msgs : make ( [ ] msgstream . TsMsg , 1 ) ,
2020-11-05 18:01:33 +08:00
}
2020-12-03 19:00:11 +08:00
2021-01-22 09:36:18 +08:00
it . result = & milvuspb . InsertResponse {
2020-11-14 18:18:10 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2020-11-14 18:18:10 +08:00
} ,
2021-01-22 09:36:18 +08:00
RowIDBegin : rowIDBegin ,
RowIDEnd : rowIDEnd ,
2020-11-14 18:18:10 +08:00
}
2021-01-30 15:30:38 +08:00
msgPack . Msgs [ 0 ] = tsMsg
2021-05-27 17:09:50 +08:00
stream , err := it . chMgr . getDMLStream ( collID )
2021-01-30 15:30:38 +08:00
if err != nil {
2021-05-27 17:09:50 +08:00
err = it . chMgr . createDMLMsgStream ( collID )
2021-01-30 15:30:38 +08:00
if err != nil {
2021-06-15 10:19:38 +08:00
it . result . Status . ErrorCode = commonpb . ErrorCode_UnexpectedError
it . result . Status . Reason = err . Error ( )
return err
}
stream , err = it . chMgr . getDMLStream ( collID )
if err != nil {
it . result . Status . ErrorCode = commonpb . ErrorCode_UnexpectedError
it . result . Status . Reason = err . Error ( )
2021-01-30 15:30:38 +08:00
return err
}
}
2021-05-31 17:28:31 +08:00
pchans , err := it . chMgr . getChannels ( collID )
if err != nil {
return err
}
for _ , pchan := range pchans {
2021-06-15 10:19:38 +08:00
log . Debug ( "ProxyNode InsertTask add pchan" , zap . Any ( "pchan" , pchan ) )
2021-05-31 17:28:31 +08:00
_ = it . chTicker . addPChan ( pchan )
}
2021-05-25 19:53:15 +08:00
// Assign SegmentID
var pack * msgstream . MsgPack
pack , err = it . _assignSegmentID ( stream , & msgPack )
if err != nil {
return err
}
err = stream . Produce ( pack )
2020-11-14 18:18:10 +08:00
if err != nil {
2021-03-10 22:06:22 +08:00
it . result . Status . ErrorCode = commonpb . ErrorCode_UnexpectedError
2020-11-14 18:18:10 +08:00
it . result . Status . Reason = err . Error ( )
2021-01-30 15:30:38 +08:00
return err
2020-11-14 18:18:10 +08:00
}
2021-01-30 15:30:38 +08:00
2020-11-05 18:01:33 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( it * InsertTask ) PostExecute ( ctx context . Context ) error {
2020-11-05 18:01:33 +08:00
return nil
}
type CreateCollectionTask struct {
2020-11-17 20:00:23 +08:00
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . CreateCollectionRequest
2021-02-23 09:58:06 +08:00
ctx context . Context
2021-03-08 10:09:48 +08:00
masterService types . MasterService
dataServiceClient types . DataService
2021-01-30 15:30:38 +08:00
result * commonpb . Status
schema * schemapb . CollectionSchema
2020-11-05 18:01:33 +08:00
}
2021-03-25 14:41:46 +08:00
func ( cct * CreateCollectionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return cct . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-12 12:04:12 +08:00
func ( cct * CreateCollectionTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return cct . Base . MsgID
2020-11-05 18:01:33 +08:00
}
2020-11-23 16:52:17 +08:00
func ( cct * CreateCollectionTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
cct . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cct * CreateCollectionTask ) Name ( ) string {
return CreateCollectionTaskName
}
2021-01-16 15:06:19 +08:00
func ( cct * CreateCollectionTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return cct . Base . MsgType
2020-11-05 18:01:33 +08:00
}
func ( cct * CreateCollectionTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return cct . Base . Timestamp
2020-11-05 18:01:33 +08:00
}
func ( cct * CreateCollectionTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return cct . Base . Timestamp
2020-11-05 18:01:33 +08:00
}
func ( cct * CreateCollectionTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
cct . Base . Timestamp = ts
2020-11-05 18:01:33 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cct * CreateCollectionTask ) OnEnqueue ( ) error {
cct . Base = & commonpb . MsgBase { }
return nil
}
func ( cct * CreateCollectionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
cct . Base . MsgType = commonpb . MsgType_CreateCollection
2021-01-28 20:51:44 +08:00
cct . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
cct . schema = & schemapb . CollectionSchema { }
err := proto . Unmarshal ( cct . Schema , cct . schema )
if err != nil {
return err
}
2021-01-28 20:51:44 +08:00
if int64 ( len ( cct . schema . Fields ) ) > Params . MaxFieldNum {
2021-03-05 10:15:27 +08:00
return fmt . Errorf ( "maximum field's number should be limited to %d" , Params . MaxFieldNum )
2020-11-26 16:01:31 +08:00
}
// validate collection name
if err := ValidateCollectionName ( cct . schema . Name ) ; err != nil {
return err
}
2020-11-30 19:38:23 +08:00
if err := ValidateDuplicatedFieldName ( cct . schema . Fields ) ; err != nil {
return err
}
if err := ValidatePrimaryKey ( cct . schema ) ; err != nil {
return err
}
2020-11-26 16:01:31 +08:00
// validate field name
for _ , field := range cct . schema . Fields {
if err := ValidateFieldName ( field . Name ) ; err != nil {
return err
}
2021-03-12 14:22:09 +08:00
if field . DataType == schemapb . DataType_FloatVector || field . DataType == schemapb . DataType_BinaryVector {
2020-11-26 16:01:31 +08:00
exist := false
var dim int64 = 0
for _ , param := range field . TypeParams {
if param . Key == "dim" {
exist = true
tmp , err := strconv . ParseInt ( param . Value , 10 , 64 )
if err != nil {
return err
}
dim = tmp
break
}
}
if ! exist {
return errors . New ( "dimension is not defined in field type params" )
}
2021-03-12 14:22:09 +08:00
if field . DataType == schemapb . DataType_FloatVector {
2020-11-26 16:01:31 +08:00
if err := ValidateDimension ( dim , false ) ; err != nil {
return err
}
} else {
if err := ValidateDimension ( dim , true ) ; err != nil {
return err
}
}
}
}
2020-11-05 18:01:33 +08:00
return nil
2020-11-03 14:53:36 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cct * CreateCollectionTask ) Execute ( ctx context . Context ) error {
2021-01-22 09:36:18 +08:00
var err error
2021-03-08 10:09:48 +08:00
cct . result , err = cct . masterService . CreateCollection ( ctx , cct . CreateCollectionRequest )
2021-05-27 17:09:50 +08:00
return err
2020-11-03 14:53:36 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cct * CreateCollectionTask ) PostExecute ( ctx context . Context ) error {
2020-11-05 18:01:33 +08:00
return nil
2020-11-03 14:53:36 +08:00
}
2020-11-09 17:25:53 +08:00
type DropCollectionTask struct {
2020-11-17 20:00:23 +08:00
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . DropCollectionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * commonpb . Status
2021-05-27 17:09:50 +08:00
chMgr channelsMgr
2021-06-08 19:25:37 +08:00
chTicker channelsTimeTicker
2020-11-09 17:25:53 +08:00
}
2021-03-25 14:41:46 +08:00
func ( dct * DropCollectionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return dct . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-12 12:04:12 +08:00
func ( dct * DropCollectionTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return dct . Base . MsgID
2020-11-09 17:25:53 +08:00
}
2020-11-23 16:52:17 +08:00
func ( dct * DropCollectionTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
dct . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dct * DropCollectionTask ) Name ( ) string {
return DropCollectionTaskName
}
2021-01-16 15:06:19 +08:00
func ( dct * DropCollectionTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return dct . Base . MsgType
2020-11-09 17:25:53 +08:00
}
func ( dct * DropCollectionTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( dct * DropCollectionTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( dct * DropCollectionTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
dct . Base . Timestamp = ts
2020-11-09 17:25:53 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dct * DropCollectionTask ) OnEnqueue ( ) error {
dct . Base = & commonpb . MsgBase { }
return nil
}
func ( dct * DropCollectionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
dct . Base . MsgType = commonpb . MsgType_DropCollection
2021-01-28 20:51:44 +08:00
dct . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
if err := ValidateCollectionName ( dct . CollectionName ) ; err != nil {
2020-11-26 16:01:31 +08:00
return err
}
2020-11-09 17:25:53 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( dct * DropCollectionTask ) Execute ( ctx context . Context ) error {
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , dct . CollectionName )
2021-02-01 10:53:13 +08:00
if err != nil {
return err
}
2021-02-07 13:53:40 +08:00
2021-03-08 10:09:48 +08:00
dct . result , err = dct . masterService . DropCollection ( ctx , dct . DropCollectionRequest )
2021-02-07 13:53:40 +08:00
if err != nil {
return err
2021-01-30 15:30:38 +08:00
}
2021-02-07 13:53:40 +08:00
2021-06-08 19:25:37 +08:00
pchans , _ := dct . chMgr . getChannels ( collID )
for _ , pchan := range pchans {
_ = dct . chTicker . removePChan ( pchan )
2021-02-04 19:34:35 +08:00
}
2021-02-07 13:53:40 +08:00
2021-06-08 19:25:37 +08:00
_ = dct . chMgr . removeDMLStream ( collID )
2021-02-04 19:34:35 +08:00
return nil
2020-11-09 17:25:53 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dct * DropCollectionTask ) PostExecute ( ctx context . Context ) error {
2021-02-26 17:44:24 +08:00
globalMetaCache . RemoveCollection ( ctx , dct . CollectionName )
2020-11-30 22:14:19 +08:00
return nil
2020-11-09 17:25:53 +08:00
}
2021-01-18 19:32:08 +08:00
type SearchTask struct {
2020-11-17 20:00:23 +08:00
Condition
2021-03-12 14:22:09 +08:00
* internalpb . SearchRequest
2021-02-23 09:58:06 +08:00
ctx context . Context
2021-02-04 14:37:12 +08:00
queryMsgStream msgstream . MsgStream
2021-03-12 14:22:09 +08:00
resultBuf chan [ ] * internalpb . SearchResults
2021-01-22 09:36:18 +08:00
result * milvuspb . SearchResults
query * milvuspb . SearchRequest
2021-06-02 10:17:32 +08:00
chMgr channelsMgr
2021-01-22 09:36:18 +08:00
}
2021-03-25 14:41:46 +08:00
func ( st * SearchTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return st . ctx
2020-11-07 16:18:23 +08:00
}
2021-01-18 19:32:08 +08:00
func ( st * SearchTask ) ID ( ) UniqueID {
return st . Base . MsgID
2020-11-07 16:18:23 +08:00
}
2021-01-18 19:32:08 +08:00
func ( st * SearchTask ) SetID ( uid UniqueID ) {
st . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( st * SearchTask ) Name ( ) string {
return SearchTaskName
}
2021-01-18 19:32:08 +08:00
func ( st * SearchTask ) Type ( ) commonpb . MsgType {
return st . Base . MsgType
2020-11-07 16:18:23 +08:00
}
2021-01-18 19:32:08 +08:00
func ( st * SearchTask ) BeginTs ( ) Timestamp {
return st . Base . Timestamp
2020-11-07 16:18:23 +08:00
}
2021-01-18 19:32:08 +08:00
func ( st * SearchTask ) EndTs ( ) Timestamp {
return st . Base . Timestamp
2020-11-07 16:18:23 +08:00
}
2021-01-18 19:32:08 +08:00
func ( st * SearchTask ) SetTs ( ts Timestamp ) {
st . Base . Timestamp = ts
2020-11-07 16:18:23 +08:00
}
2021-02-23 09:58:06 +08:00
func ( st * SearchTask ) OnEnqueue ( ) error {
2021-03-22 19:28:43 +08:00
st . Base = & commonpb . MsgBase { }
2021-02-23 09:58:06 +08:00
return nil
}
2021-06-15 10:19:38 +08:00
func ( st * SearchTask ) getChannels ( ) ( [ ] pChan , error ) {
2021-06-11 09:50:34 +08:00
collID , err := globalMetaCache . GetCollectionID ( st . ctx , st . query . CollectionName )
if err != nil {
return nil , err
}
return st . chMgr . getChannels ( collID )
}
2021-06-02 10:17:32 +08:00
func ( st * SearchTask ) getVChannels ( ) ( [ ] vChan , error ) {
collID , err := globalMetaCache . GetCollectionID ( st . ctx , st . query . CollectionName )
if err != nil {
return nil , err
}
_ , err = st . chMgr . getChannels ( collID )
if err != nil {
err := st . chMgr . createDMLMsgStream ( collID )
if err != nil {
return nil , err
}
}
return st . chMgr . getVChannels ( collID )
}
2021-02-23 09:58:06 +08:00
func ( st * SearchTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
st . Base . MsgType = commonpb . MsgType_Search
2021-01-28 20:51:44 +08:00
st . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
collectionName := st . query . CollectionName
2021-02-26 17:44:24 +08:00
_ , err := globalMetaCache . GetCollectionID ( ctx , collectionName )
2020-11-30 19:08:32 +08:00
if err != nil { // err is not nil if collection not exists
return err
}
2021-01-18 19:32:08 +08:00
if err := ValidateCollectionName ( st . query . CollectionName ) ; err != nil {
2020-11-26 16:01:31 +08:00
return err
}
2021-01-22 09:36:18 +08:00
for _ , tag := range st . query . PartitionNames {
2020-11-26 16:01:31 +08:00
if err := ValidatePartitionTag ( tag , false ) ; err != nil {
return err
}
}
2021-03-10 14:45:35 +08:00
st . Base . MsgType = commonpb . MsgType_Search
2021-04-29 16:48:06 +08:00
if st . query . GetDslType ( ) == commonpb . DslType_BoolExprV1 {
schema , err := globalMetaCache . GetCollectionSchema ( ctx , collectionName )
if err != nil { // err is not nil if collection not exists
return err
}
annsField , err := GetAttrByKeyFromRepeatedKV ( AnnsFieldKey , st . query . SearchParams )
if err != nil {
return errors . New ( AnnsFieldKey + " not found in search_params" )
}
topKStr , err := GetAttrByKeyFromRepeatedKV ( TopKKey , st . query . SearchParams )
if err != nil {
return errors . New ( TopKKey + " not found in search_params" )
}
topK , err := strconv . Atoi ( topKStr )
if err != nil {
return errors . New ( TopKKey + " " + topKStr + " is not invalid" )
}
metricType , err := GetAttrByKeyFromRepeatedKV ( MetricTypeKey , st . query . SearchParams )
if err != nil {
return errors . New ( MetricTypeKey + " not found in search_params" )
}
searchParams , err := GetAttrByKeyFromRepeatedKV ( SearchParamsKey , st . query . SearchParams )
if err != nil {
return errors . New ( SearchParamsKey + " not found in search_params" )
}
queryInfo := & planpb . QueryInfo {
Topk : int64 ( topK ) ,
MetricType : metricType ,
SearchParams : searchParams ,
}
2021-06-05 14:57:34 +08:00
plan , err := CreateQueryPlan ( schema , st . query . Dsl , annsField , queryInfo )
2021-04-29 16:48:06 +08:00
if err != nil {
return errors . New ( "invalid expression: " + st . query . Dsl )
}
2021-05-07 15:20:47 +08:00
st . SearchRequest . DslType = commonpb . DslType_BoolExprV1
2021-05-07 19:27:17 +08:00
st . SearchRequest . SerializedExprPlan , err = proto . Marshal ( plan )
if err != nil {
return err
}
2021-04-29 16:48:06 +08:00
}
2021-05-07 15:20:47 +08:00
st . SearchRequest . ResultChannelID = Params . SearchResultChannelNames [ 0 ]
st . SearchRequest . DbID = 0 // todo
2021-02-26 17:44:24 +08:00
collectionID , err := globalMetaCache . GetCollectionID ( ctx , collectionName )
2021-02-03 17:30:10 +08:00
if err != nil { // err is not nil if collection not exists
return err
}
2021-05-07 15:20:47 +08:00
st . SearchRequest . CollectionID = collectionID
st . SearchRequest . PartitionIDs = make ( [ ] UniqueID , 0 )
2021-03-22 16:36:10 +08:00
partitionsMap , err := globalMetaCache . GetPartitions ( ctx , collectionName )
if err != nil {
return err
}
partitionsRecord := make ( map [ UniqueID ] bool )
2021-02-03 17:30:10 +08:00
for _ , partitionName := range st . query . PartitionNames {
2021-03-22 16:36:10 +08:00
pattern := fmt . Sprintf ( "^%s$" , partitionName )
re , err := regexp . Compile ( pattern )
2021-02-03 17:30:10 +08:00
if err != nil {
2021-03-22 16:36:10 +08:00
return errors . New ( "invalid partition names" )
}
found := false
for name , pID := range partitionsMap {
if re . MatchString ( name ) {
if _ , exist := partitionsRecord [ pID ] ; ! exist {
st . PartitionIDs = append ( st . PartitionIDs , pID )
partitionsRecord [ pID ] = true
}
found = true
}
}
if ! found {
errMsg := fmt . Sprintf ( "PartitonName: %s not found" , partitionName )
return errors . New ( errMsg )
2021-02-03 17:30:10 +08:00
}
}
2021-03-22 16:36:10 +08:00
2021-05-07 15:20:47 +08:00
st . SearchRequest . Dsl = st . query . Dsl
st . SearchRequest . PlaceholderGroup = st . query . PlaceholderGroup
2021-02-03 17:30:10 +08:00
2020-11-07 16:18:23 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( st * SearchTask ) Execute ( ctx context . Context ) error {
2020-11-07 16:18:23 +08:00
var tsMsg msgstream . TsMsg = & msgstream . SearchMsg {
2021-02-23 09:58:06 +08:00
SearchRequest : * st . SearchRequest ,
2020-11-07 16:18:23 +08:00
BaseMsg : msgstream . BaseMsg {
2021-03-25 14:41:46 +08:00
Ctx : ctx ,
2021-01-28 20:51:44 +08:00
HashValues : [ ] uint32 { uint32 ( Params . ProxyID ) } ,
2021-01-18 19:32:08 +08:00
BeginTimestamp : st . Base . Timestamp ,
EndTimestamp : st . Base . Timestamp ,
2020-11-07 16:18:23 +08:00
} ,
}
2021-03-25 14:41:46 +08:00
msgPack := msgstream . MsgPack {
2021-01-18 19:32:08 +08:00
BeginTs : st . Base . Timestamp ,
EndTs : st . Base . Timestamp ,
2020-11-17 14:10:07 +08:00
Msgs : make ( [ ] msgstream . TsMsg , 1 ) ,
2020-11-07 16:18:23 +08:00
}
2020-11-17 14:10:07 +08:00
msgPack . Msgs [ 0 ] = tsMsg
2021-03-25 14:41:46 +08:00
err := st . queryMsgStream . Produce ( & msgPack )
2021-03-08 19:39:36 +08:00
log . Debug ( "proxynode" , zap . Int ( "length of searchMsg" , len ( msgPack . Msgs ) ) )
2021-06-03 14:58:34 +08:00
log . Debug ( "proxy node sent one searchMsg" ,
2021-06-15 12:41:40 +08:00
zap . Any ( "collectionID" , st . CollectionID ) ,
2021-06-03 14:58:34 +08:00
zap . Any ( "msgID" , tsMsg . ID ( ) ) ,
2021-06-15 12:41:40 +08:00
)
2020-11-28 19:06:48 +08:00
if err != nil {
2021-03-08 19:39:36 +08:00
log . Debug ( "proxynode" , zap . String ( "send search request failed" , err . Error ( ) ) )
2020-11-28 19:06:48 +08:00
}
return err
2020-11-07 16:18:23 +08:00
}
2021-03-26 11:19:02 +08:00
// TODO: add benchmark to compare with serial implementation
func decodeSearchResultsParallel ( searchResults [ ] * internalpb . SearchResults , maxParallel int ) ( [ ] [ ] * milvuspb . Hits , error ) {
log . Debug ( "decodeSearchResultsParallel" , zap . Any ( "NumOfGoRoutines" , maxParallel ) )
2021-03-25 10:14:09 +08:00
hits := make ( [ ] [ ] * milvuspb . Hits , 0 )
2021-03-26 11:19:02 +08:00
// necessary to parallel this?
2021-03-25 10:14:09 +08:00
for _ , partialSearchResult := range searchResults {
if partialSearchResult . Hits == nil || len ( partialSearchResult . Hits ) <= 0 {
continue
}
2021-03-26 11:19:02 +08:00
nq := len ( partialSearchResult . Hits )
partialHits := make ( [ ] * milvuspb . Hits , nq )
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
f := func ( idx int ) error {
partialHit := & milvuspb . Hits { }
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
err := proto . Unmarshal ( partialSearchResult . Hits [ idx ] , partialHit )
if err != nil {
return err
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
partialHits [ idx ] = partialHit
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
return nil
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
err := funcutil . ProcessFuncParallel ( nq , maxParallel , f , "decodePartialSearchResult" )
2021-03-25 10:14:09 +08:00
if err != nil {
return nil , err
}
hits = append ( hits , partialHits )
}
return hits , nil
}
2021-03-26 11:19:02 +08:00
func decodeSearchResultsSerial ( searchResults [ ] * internalpb . SearchResults ) ( [ ] [ ] * milvuspb . Hits , error ) {
return decodeSearchResultsParallel ( searchResults , 1 )
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
// TODO: add benchmark to compare with serial implementation
func decodeSearchResultsParallelByNq ( searchResults [ ] * internalpb . SearchResults ) ( [ ] [ ] * milvuspb . Hits , error ) {
if len ( searchResults ) <= 0 {
return nil , errors . New ( "no need to decode empty search results" )
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
nq := len ( searchResults [ 0 ] . Hits )
return decodeSearchResultsParallel ( searchResults , nq )
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
// TODO: add benchmark to compare with serial implementation
func decodeSearchResultsParallelByCPU ( searchResults [ ] * internalpb . SearchResults ) ( [ ] [ ] * milvuspb . Hits , error ) {
return decodeSearchResultsParallel ( searchResults , runtime . NumCPU ( ) )
2021-03-25 10:14:09 +08:00
}
func decodeSearchResults ( searchResults [ ] * internalpb . SearchResults ) ( [ ] [ ] * milvuspb . Hits , error ) {
2021-03-26 11:19:02 +08:00
t := time . Now ( )
defer func ( ) {
log . Debug ( "decodeSearchResults" , zap . Any ( "time cost" , time . Since ( t ) ) )
} ( )
return decodeSearchResultsParallelByCPU ( searchResults )
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
func reduceSearchResultsParallel ( hits [ ] [ ] * milvuspb . Hits , nq , availableQueryNodeNum , topk int , metricType string , maxParallel int ) * milvuspb . SearchResults {
log . Debug ( "reduceSearchResultsParallel" , zap . Any ( "NumOfGoRoutines" , maxParallel ) )
2021-03-25 10:14:09 +08:00
ret := & milvuspb . SearchResults {
Status : & commonpb . Status {
ErrorCode : 0 ,
} ,
Hits : make ( [ ] [ ] byte , nq ) ,
}
const minFloat32 = - 1 * float32 ( math . MaxFloat32 )
2021-03-26 11:19:02 +08:00
f := func ( idx int ) error {
locs := make ( [ ] int , availableQueryNodeNum )
reducedHits := & milvuspb . Hits {
IDs : make ( [ ] int64 , 0 ) ,
RowData : make ( [ ] [ ] byte , 0 ) ,
Scores : make ( [ ] float32 , 0 ) ,
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
for j := 0 ; j < topk ; j ++ {
valid := false
choice , maxDistance := 0 , minFloat32
for q , loc := range locs { // query num, the number of ways to merge
if loc >= len ( hits [ q ] [ idx ] . IDs ) {
continue
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
distance := hits [ q ] [ idx ] . Scores [ loc ]
if distance > maxDistance || ( math . Abs ( float64 ( distance - maxDistance ) ) < math . SmallestNonzeroFloat32 && choice != q ) {
choice = q
maxDistance = distance
valid = true
2021-03-25 10:14:09 +08:00
}
}
2021-03-26 11:19:02 +08:00
if ! valid {
break
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
choiceOffset := locs [ choice ]
// check if distance is valid, `invalid` here means very very big,
// in this process, distance here is the smallest, so the rest of distance are all invalid
if hits [ choice ] [ idx ] . Scores [ choiceOffset ] <= minFloat32 {
break
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
reducedHits . IDs = append ( reducedHits . IDs , hits [ choice ] [ idx ] . IDs [ choiceOffset ] )
if hits [ choice ] [ idx ] . RowData != nil && len ( hits [ choice ] [ idx ] . RowData ) > 0 {
reducedHits . RowData = append ( reducedHits . RowData , hits [ choice ] [ idx ] . RowData [ choiceOffset ] )
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
reducedHits . Scores = append ( reducedHits . Scores , hits [ choice ] [ idx ] . Scores [ choiceOffset ] )
locs [ choice ] ++
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
if metricType != "IP" {
for k := range reducedHits . Scores {
reducedHits . Scores [ k ] *= - 1
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
reducedHitsBs , err := proto . Marshal ( reducedHits )
if err != nil {
return err
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
ret . Hits [ idx ] = reducedHitsBs
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
return nil
2021-03-25 10:14:09 +08:00
}
2021-03-26 11:19:02 +08:00
err := funcutil . ProcessFuncParallel ( nq , maxParallel , f , "reduceSearchResults" )
if err != nil {
return nil
}
2021-03-25 10:14:09 +08:00
return ret
}
2021-03-26 11:19:02 +08:00
func reduceSearchResultsSerial ( hits [ ] [ ] * milvuspb . Hits , nq , availableQueryNodeNum , topk int , metricType string ) * milvuspb . SearchResults {
return reduceSearchResultsParallel ( hits , nq , availableQueryNodeNum , topk , metricType , 1 )
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
// TODO: add benchmark to compare with serial implementation
func reduceSearchResultsParallelByNq ( hits [ ] [ ] * milvuspb . Hits , nq , availableQueryNodeNum , topk int , metricType string ) * milvuspb . SearchResults {
return reduceSearchResultsParallel ( hits , nq , availableQueryNodeNum , topk , metricType , nq )
}
2021-03-25 10:14:09 +08:00
2021-03-26 11:19:02 +08:00
// TODO: add benchmark to compare with serial implementation
func reduceSearchResultsParallelByCPU ( hits [ ] [ ] * milvuspb . Hits , nq , availableQueryNodeNum , topk int , metricType string ) * milvuspb . SearchResults {
return reduceSearchResultsParallel ( hits , nq , availableQueryNodeNum , topk , metricType , runtime . NumCPU ( ) )
2021-03-25 10:14:09 +08:00
}
func reduceSearchResults ( hits [ ] [ ] * milvuspb . Hits , nq , availableQueryNodeNum , topk int , metricType string ) * milvuspb . SearchResults {
2021-03-26 11:19:02 +08:00
t := time . Now ( )
defer func ( ) {
log . Debug ( "reduceSearchResults" , zap . Any ( "time cost" , time . Since ( t ) ) )
} ( )
return reduceSearchResultsParallelByCPU ( hits , nq , availableQueryNodeNum , topk , metricType )
2021-03-25 10:14:09 +08:00
}
func printSearchResult ( partialSearchResult * internalpb . SearchResults ) {
for i := 0 ; i < len ( partialSearchResult . Hits ) ; i ++ {
testHits := milvuspb . Hits { }
err := proto . Unmarshal ( partialSearchResult . Hits [ i ] , & testHits )
if err != nil {
panic ( err )
}
fmt . Println ( testHits . IDs )
fmt . Println ( testHits . Scores )
}
}
2021-02-23 09:58:06 +08:00
func ( st * SearchTask ) PostExecute ( ctx context . Context ) error {
2021-03-26 11:19:02 +08:00
t0 := time . Now ( )
defer func ( ) {
log . Debug ( "WaitAndPostExecute" , zap . Any ( "time cost" , time . Since ( t0 ) ) )
} ( )
2020-11-07 16:18:23 +08:00
for {
select {
2021-03-25 14:41:46 +08:00
case <- st . TraceCtx ( ) . Done ( ) :
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode" , zap . Int64 ( "SearchTask PostExecute Loop exit caused by ctx.Done" , st . ID ( ) ) )
2021-03-05 10:15:27 +08:00
return fmt . Errorf ( "SearchTask:wait to finish failed, timeout: %d" , st . ID ( ) )
2021-01-18 19:32:08 +08:00
case searchResults := <- st . resultBuf :
2021-01-23 20:58:46 +08:00
// fmt.Println("searchResults: ", searchResults)
2021-03-12 14:22:09 +08:00
filterSearchResult := make ( [ ] * internalpb . SearchResults , 0 )
2020-11-30 22:14:19 +08:00
var filterReason string
2020-11-30 19:08:32 +08:00
for _ , partialSearchResult := range searchResults {
2021-03-10 22:06:22 +08:00
if partialSearchResult . Status . ErrorCode == commonpb . ErrorCode_Success {
2020-11-30 19:08:32 +08:00
filterSearchResult = append ( filterSearchResult , partialSearchResult )
2021-01-18 19:32:08 +08:00
// For debugging, please don't delete.
2021-03-25 10:14:09 +08:00
// printSearchResult(partialSearchResult)
2020-11-30 22:14:19 +08:00
} else {
filterReason += partialSearchResult . Status . Reason + "\n"
2020-11-30 19:08:32 +08:00
}
}
2020-12-10 16:31:09 +08:00
availableQueryNodeNum := len ( filterSearchResult )
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode Search PostExecute stage1" , zap . Any ( "availableQueryNodeNum" , availableQueryNodeNum ) )
2020-12-10 16:31:09 +08:00
if availableQueryNodeNum <= 0 {
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode Search PostExecute failed" , zap . Any ( "filterReason" , filterReason ) )
2021-01-22 09:36:18 +08:00
st . result = & milvuspb . SearchResults {
2020-11-30 22:14:19 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
2020-11-30 22:14:19 +08:00
Reason : filterReason ,
} ,
}
return errors . New ( filterReason )
2020-11-07 16:18:23 +08:00
}
2020-11-28 19:06:48 +08:00
2021-03-25 10:14:09 +08:00
availableQueryNodeNum = 0
2020-12-10 16:31:09 +08:00
for _ , partialSearchResult := range filterSearchResult {
2021-01-12 18:03:24 +08:00
if partialSearchResult . Hits == nil || len ( partialSearchResult . Hits ) <= 0 {
2020-12-10 16:31:09 +08:00
filterReason += "nq is zero\n"
continue
}
2021-03-25 10:14:09 +08:00
availableQueryNodeNum ++
2020-12-10 16:31:09 +08:00
}
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode Search PostExecute stage2" , zap . Any ( "availableQueryNodeNum" , availableQueryNodeNum ) )
2020-12-10 16:31:09 +08:00
if availableQueryNodeNum <= 0 {
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode Search PostExecute stage2 failed" , zap . Any ( "filterReason" , filterReason ) )
2021-01-22 09:36:18 +08:00
st . result = & milvuspb . SearchResults {
2020-12-10 16:31:09 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2020-12-10 16:31:09 +08:00
Reason : filterReason ,
} ,
}
return nil
}
2021-03-25 10:14:09 +08:00
hits , err := decodeSearchResults ( filterSearchResult )
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode Search PostExecute decodeSearchResults" , zap . Error ( err ) )
2021-03-25 10:14:09 +08:00
if err != nil {
return err
}
2020-12-10 16:31:09 +08:00
nq := len ( hits [ 0 ] )
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode Search PostExecute" , zap . Any ( "nq" , nq ) )
2020-12-10 16:31:09 +08:00
if nq <= 0 {
2021-01-22 09:36:18 +08:00
st . result = & milvuspb . SearchResults {
2020-12-10 16:31:09 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2020-12-10 16:31:09 +08:00
Reason : filterReason ,
} ,
2020-11-26 16:01:31 +08:00
}
2020-12-10 16:31:09 +08:00
return nil
2020-11-26 16:01:31 +08:00
}
2020-11-28 19:06:48 +08:00
2021-01-12 18:03:24 +08:00
topk := 0
for _ , hit := range hits {
topk = getMax ( topk , len ( hit [ 0 ] . IDs ) )
}
2020-11-28 19:06:48 +08:00
2021-03-25 10:14:09 +08:00
st . result = reduceSearchResults ( hits , nq , availableQueryNodeNum , topk , searchResults [ 0 ] . MetricType )
2021-06-11 09:50:34 +08:00
log . Debug ( "ProxyNode Search PostExecute Done" )
2020-11-28 19:06:48 +08:00
return nil
2020-11-07 16:18:23 +08:00
}
}
2020-11-16 17:01:10 +08:00
}
2021-05-20 15:02:31 +08:00
type RetrieveTask struct {
Condition
* internalpb . RetrieveRequest
ctx context . Context
queryMsgStream msgstream . MsgStream
resultBuf chan [ ] * internalpb . RetrieveResults
result * milvuspb . RetrieveResults
retrieve * milvuspb . RetrieveRequest
}
func ( rt * RetrieveTask ) TraceCtx ( ) context . Context {
return rt . ctx
}
func ( rt * RetrieveTask ) ID ( ) UniqueID {
return rt . Base . MsgID
}
func ( rt * RetrieveTask ) SetID ( uid UniqueID ) {
rt . Base . MsgID = uid
}
func ( rt * RetrieveTask ) Name ( ) string {
return RetrieveTaskName
}
func ( rt * RetrieveTask ) Type ( ) commonpb . MsgType {
return rt . Base . MsgType
}
func ( rt * RetrieveTask ) BeginTs ( ) Timestamp {
return rt . Base . Timestamp
}
func ( rt * RetrieveTask ) EndTs ( ) Timestamp {
return rt . Base . Timestamp
}
func ( rt * RetrieveTask ) SetTs ( ts Timestamp ) {
rt . Base . Timestamp = ts
}
func ( rt * RetrieveTask ) OnEnqueue ( ) error {
rt . Base . MsgType = commonpb . MsgType_Retrieve
return nil
}
func ( rt * RetrieveTask ) PreExecute ( ctx context . Context ) error {
rt . Base . MsgType = commonpb . MsgType_Retrieve
rt . Base . SourceID = Params . ProxyID
collectionName := rt . retrieve . CollectionName
collectionID , err := globalMetaCache . GetCollectionID ( ctx , collectionName )
if err != nil {
2021-05-29 11:19:30 +08:00
log . Debug ( "Failed to get collection id." , zap . Any ( "collectionName" , collectionName ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
return err
}
2021-05-29 11:19:30 +08:00
log . Info ( "Get collection id by name." , zap . Any ( "collectionName" , collectionName ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
if err := ValidateCollectionName ( rt . retrieve . CollectionName ) ; err != nil {
2021-05-29 11:19:30 +08:00
log . Debug ( "Invalid collection name." , zap . Any ( "collectionName" , collectionName ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
return err
}
2021-05-29 11:19:30 +08:00
log . Info ( "Validate collection name." , zap . Any ( "collectionName" , collectionName ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
for _ , tag := range rt . retrieve . PartitionNames {
if err := ValidatePartitionTag ( tag , false ) ; err != nil {
2021-05-29 11:19:30 +08:00
log . Debug ( "Invalid partition name." , zap . Any ( "partitionName" , tag ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
return err
}
}
2021-05-29 11:19:30 +08:00
log . Info ( "Validate partition names." ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
rt . Base . MsgType = commonpb . MsgType_Retrieve
2021-06-05 10:57:34 +08:00
if rt . retrieve . Ids == nil {
errMsg := "Retrieve ids is nil"
return errors . New ( errMsg )
}
2021-05-20 15:02:31 +08:00
rt . Ids = rt . retrieve . Ids
2021-06-05 10:57:34 +08:00
if len ( rt . retrieve . OutputFields ) == 0 {
schema , err := globalMetaCache . GetCollectionSchema ( ctx , collectionName )
if err != nil {
return err
}
for _ , field := range schema . Fields {
rt . OutputFields = append ( rt . OutputFields , field . Name )
}
} else {
rt . OutputFields = rt . retrieve . OutputFields
}
2021-05-20 15:02:31 +08:00
rt . ResultChannelID = Params . RetrieveChannelNames [ 0 ]
rt . DbID = 0 // todo(yukun)
rt . CollectionID = collectionID
rt . PartitionIDs = make ( [ ] UniqueID , 0 )
partitionsMap , err := globalMetaCache . GetPartitions ( ctx , collectionName )
if err != nil {
2021-05-29 11:19:30 +08:00
log . Debug ( "Failed to get partitions in collection." , zap . Any ( "collectionName" , collectionName ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
return err
}
2021-05-29 11:19:30 +08:00
log . Info ( "Get partitions in collection." , zap . Any ( "collectionName" , collectionName ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
partitionsRecord := make ( map [ UniqueID ] bool )
for _ , partitionName := range rt . retrieve . PartitionNames {
pattern := fmt . Sprintf ( "^%s$" , partitionName )
re , err := regexp . Compile ( pattern )
if err != nil {
2021-05-29 11:19:30 +08:00
log . Debug ( "Failed to compile partition name regex expression." , zap . Any ( "partitionName" , partitionName ) ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
return errors . New ( "invalid partition names" )
}
found := false
for name , pID := range partitionsMap {
if re . MatchString ( name ) {
if _ , exist := partitionsRecord [ pID ] ; ! exist {
rt . PartitionIDs = append ( rt . PartitionIDs , pID )
partitionsRecord [ pID ] = true
}
found = true
}
}
if ! found {
2021-05-29 11:19:30 +08:00
// FIXME(wxyu): undefined behavior
2021-05-20 15:02:31 +08:00
errMsg := fmt . Sprintf ( "PartitonName: %s not found" , partitionName )
return errors . New ( errMsg )
}
}
2021-05-29 11:19:30 +08:00
log . Info ( "Retrieve PreExecute done." ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
return nil
}
func ( rt * RetrieveTask ) Execute ( ctx context . Context ) error {
var tsMsg msgstream . TsMsg = & msgstream . RetrieveMsg {
RetrieveRequest : * rt . RetrieveRequest ,
BaseMsg : msgstream . BaseMsg {
Ctx : ctx ,
HashValues : [ ] uint32 { uint32 ( Params . ProxyID ) } ,
BeginTimestamp : rt . Base . Timestamp ,
EndTimestamp : rt . Base . Timestamp ,
} ,
}
msgPack := msgstream . MsgPack {
BeginTs : rt . Base . Timestamp ,
EndTs : rt . Base . Timestamp ,
Msgs : make ( [ ] msgstream . TsMsg , 1 ) ,
}
msgPack . Msgs [ 0 ] = tsMsg
err := rt . queryMsgStream . Produce ( & msgPack )
log . Debug ( "proxynode" , zap . Int ( "length of retrieveMsg" , len ( msgPack . Msgs ) ) )
if err != nil {
2021-05-29 11:19:30 +08:00
log . Debug ( "Failed to send retrieve request." ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
}
2021-05-29 11:19:30 +08:00
log . Info ( "Retrieve Execute done." ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
2021-05-20 15:02:31 +08:00
return err
}
2021-05-21 16:39:28 +08:00
func ( rt * RetrieveTask ) PostExecute ( ctx context . Context ) error {
t0 := time . Now ( )
defer func ( ) {
log . Debug ( "WaitAndPostExecute" , zap . Any ( "time cost" , time . Since ( t0 ) ) )
} ( )
2021-05-29 11:19:30 +08:00
select {
case <- rt . TraceCtx ( ) . Done ( ) :
log . Debug ( "proxynode" , zap . Int64 ( "Retrieve: wait to finish failed, timeout!, taskID:" , rt . ID ( ) ) )
return fmt . Errorf ( "RetrieveTask:wait to finish failed, timeout : %d" , rt . ID ( ) )
case retrieveResults := <- rt . resultBuf :
retrieveResult := make ( [ ] * internalpb . RetrieveResults , 0 )
var reason string
for _ , partialRetrieveResult := range retrieveResults {
if partialRetrieveResult . Status . ErrorCode == commonpb . ErrorCode_Success {
retrieveResult = append ( retrieveResult , partialRetrieveResult )
} else {
reason += partialRetrieveResult . Status . Reason + "\n"
2021-05-21 16:39:28 +08:00
}
2021-05-29 11:19:30 +08:00
}
2021-05-21 16:39:28 +08:00
2021-05-29 11:19:30 +08:00
if len ( retrieveResult ) == 0 {
rt . result = & milvuspb . RetrieveResults {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
Reason : reason ,
} ,
2021-05-21 16:39:28 +08:00
}
2021-05-29 11:19:30 +08:00
log . Debug ( "Retrieve failed on all querynodes." ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
return errors . New ( reason )
}
2021-05-21 16:39:28 +08:00
2021-05-29 11:19:30 +08:00
availableQueryNodeNum := 0
2021-06-03 19:41:33 +08:00
rt . result = & milvuspb . RetrieveResults {
Status : & commonpb . Status {
2021-06-08 16:26:36 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-06-03 19:41:33 +08:00
} ,
Ids : & schemapb . IDs { } ,
FieldsData : make ( [ ] * schemapb . FieldData , 0 ) ,
}
for idx , partialRetrieveResult := range retrieveResult {
log . Debug ( "Index-" + strconv . Itoa ( idx ) )
2021-05-29 11:19:30 +08:00
if partialRetrieveResult . Ids == nil {
reason += "ids is nil\n"
continue
} else {
intIds , intOk := partialRetrieveResult . Ids . IdField . ( * schemapb . IDs_IntId )
strIds , strOk := partialRetrieveResult . Ids . IdField . ( * schemapb . IDs_StrId )
if ! intOk && ! strOk {
reason += "ids is empty\n"
2021-05-21 16:39:28 +08:00
continue
2021-05-29 11:19:30 +08:00
}
2021-05-21 16:39:28 +08:00
2021-05-29 11:19:30 +08:00
if ! intOk {
2021-06-03 19:41:33 +08:00
if idsStr , ok := rt . result . Ids . IdField . ( * schemapb . IDs_StrId ) ; ok {
idsStr . StrId . Data = append ( idsStr . StrId . Data , strIds . StrId . Data ... )
} else {
rt . result . Ids . IdField = & schemapb . IDs_StrId {
StrId : & schemapb . StringArray {
Data : strIds . StrId . Data ,
} ,
}
}
2021-05-29 11:19:30 +08:00
} else {
2021-06-03 19:41:33 +08:00
if idsInt , ok := rt . result . Ids . IdField . ( * schemapb . IDs_IntId ) ; ok {
idsInt . IntId . Data = append ( idsInt . IntId . Data , intIds . IntId . Data ... )
} else {
rt . result . Ids . IdField = & schemapb . IDs_IntId {
IntId : & schemapb . LongArray {
Data : intIds . IntId . Data ,
} ,
}
}
}
2021-06-08 16:26:36 +08:00
2021-06-03 19:41:33 +08:00
rt . result . FieldsData = append ( rt . result . FieldsData , partialRetrieveResult . FieldsData ... )
2021-05-21 16:39:28 +08:00
}
2021-05-29 11:19:30 +08:00
availableQueryNodeNum ++
}
2021-05-21 16:39:28 +08:00
2021-05-29 11:19:30 +08:00
if availableQueryNodeNum == 0 {
log . Info ( "Not any valid result found." ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
rt . result = & milvuspb . RetrieveResults {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
Reason : reason ,
} ,
2021-05-21 16:39:28 +08:00
}
return nil
}
2021-06-08 16:26:36 +08:00
schema , err := globalMetaCache . GetCollectionSchema ( ctx , rt . retrieve . CollectionName )
if err != nil {
return err
}
for i := 0 ; i < len ( rt . result . FieldsData ) ; i ++ {
for _ , field := range schema . Fields {
if field . Name == rt . OutputFields [ i ] {
rt . result . FieldsData [ i ] . FieldName = field . Name
rt . result . FieldsData [ i ] . Type = field . DataType
}
}
}
2021-05-21 16:39:28 +08:00
}
2021-05-29 11:19:30 +08:00
log . Info ( "Retrieve PostExecute done." ,
zap . Any ( "requestID" , rt . Base . MsgID ) , zap . Any ( "requestType" , "retrieve" ) )
return nil
2021-05-21 16:39:28 +08:00
}
2020-11-09 17:25:53 +08:00
type HasCollectionTask struct {
2020-11-17 20:00:23 +08:00
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . HasCollectionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * milvuspb . BoolResponse
2020-11-09 17:25:53 +08:00
}
2021-03-25 14:41:46 +08:00
func ( hct * HasCollectionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return hct . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-12 12:04:12 +08:00
func ( hct * HasCollectionTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return hct . Base . MsgID
2020-11-09 17:25:53 +08:00
}
2020-11-23 16:52:17 +08:00
func ( hct * HasCollectionTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
hct . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( hct * HasCollectionTask ) Name ( ) string {
return HasCollectionTaskName
}
2021-01-16 15:06:19 +08:00
func ( hct * HasCollectionTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return hct . Base . MsgType
2020-11-09 17:25:53 +08:00
}
func ( hct * HasCollectionTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return hct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( hct * HasCollectionTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return hct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( hct * HasCollectionTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
hct . Base . Timestamp = ts
2020-11-09 17:25:53 +08:00
}
2021-02-23 09:58:06 +08:00
func ( hct * HasCollectionTask ) OnEnqueue ( ) error {
hct . Base = & commonpb . MsgBase { }
return nil
}
func ( hct * HasCollectionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
hct . Base . MsgType = commonpb . MsgType_HasCollection
2021-01-28 20:51:44 +08:00
hct . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
if err := ValidateCollectionName ( hct . CollectionName ) ; err != nil {
2020-11-26 16:01:31 +08:00
return err
}
2020-11-09 17:25:53 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( hct * HasCollectionTask ) Execute ( ctx context . Context ) error {
2021-01-22 09:36:18 +08:00
var err error
2021-03-08 10:09:48 +08:00
hct . result , err = hct . masterService . HasCollection ( ctx , hct . HasCollectionRequest )
2021-02-04 19:34:35 +08:00
if hct . result == nil {
return errors . New ( "has collection resp is nil" )
}
2021-03-10 22:06:22 +08:00
if hct . result . Status . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( hct . result . Status . Reason )
}
2020-11-09 17:25:53 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( hct * HasCollectionTask ) PostExecute ( ctx context . Context ) error {
2020-11-09 17:25:53 +08:00
return nil
}
type DescribeCollectionTask struct {
2020-11-17 20:00:23 +08:00
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . DescribeCollectionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * milvuspb . DescribeCollectionResponse
2020-11-09 17:25:53 +08:00
}
2021-03-25 14:41:46 +08:00
func ( dct * DescribeCollectionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return dct . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-12 12:04:12 +08:00
func ( dct * DescribeCollectionTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return dct . Base . MsgID
2020-11-09 17:25:53 +08:00
}
2020-11-23 16:52:17 +08:00
func ( dct * DescribeCollectionTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
dct . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dct * DescribeCollectionTask ) Name ( ) string {
return DescribeCollectionTaskName
}
2021-01-16 15:06:19 +08:00
func ( dct * DescribeCollectionTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return dct . Base . MsgType
2020-11-09 17:25:53 +08:00
}
func ( dct * DescribeCollectionTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( dct * DescribeCollectionTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( dct * DescribeCollectionTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
dct . Base . Timestamp = ts
2020-11-09 17:25:53 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dct * DescribeCollectionTask ) OnEnqueue ( ) error {
dct . Base = & commonpb . MsgBase { }
return nil
}
func ( dct * DescribeCollectionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
dct . Base . MsgType = commonpb . MsgType_DescribeCollection
2021-01-28 20:51:44 +08:00
dct . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
if err := ValidateCollectionName ( dct . CollectionName ) ; err != nil {
2020-11-26 16:01:31 +08:00
return err
}
2020-11-09 17:25:53 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( dct * DescribeCollectionTask ) Execute ( ctx context . Context ) error {
2021-01-22 09:36:18 +08:00
var err error
2021-06-08 19:25:37 +08:00
dct . result = & milvuspb . DescribeCollectionResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
} ,
Schema : & schemapb . CollectionSchema {
Name : "" ,
Description : "" ,
AutoID : false ,
Fields : make ( [ ] * schemapb . FieldSchema , 0 ) ,
} ,
CollectionID : 0 ,
VirtualChannelNames : nil ,
PhysicalChannelNames : nil ,
}
result , err := dct . masterService . DescribeCollection ( ctx , dct . DescribeCollectionRequest )
2021-06-11 15:33:18 +08:00
if err != nil {
return err
2021-02-04 19:34:35 +08:00
}
2021-06-08 19:25:37 +08:00
2021-06-11 15:33:18 +08:00
if result . Status . ErrorCode != commonpb . ErrorCode_Success {
dct . result . Status = result . Status
} else {
dct . result . Schema . Name = result . Schema . Name
dct . result . Schema . Description = result . Schema . Description
dct . result . Schema . AutoID = result . Schema . AutoID
dct . result . CollectionID = result . CollectionID
dct . result . VirtualChannelNames = result . VirtualChannelNames
dct . result . PhysicalChannelNames = result . PhysicalChannelNames
for _ , field := range result . Schema . Fields {
if field . FieldID >= 100 { // TODO(dragondriver): use StartOfUserFieldID replacing 100
dct . result . Schema . Fields = append ( dct . result . Schema . Fields , & schemapb . FieldSchema {
FieldID : field . FieldID ,
Name : field . Name ,
IsPrimaryKey : field . IsPrimaryKey ,
Description : field . Description ,
DataType : field . DataType ,
TypeParams : field . TypeParams ,
IndexParams : field . IndexParams ,
} )
}
2021-06-08 19:25:37 +08:00
}
}
2021-06-11 15:33:18 +08:00
return nil
2020-11-09 17:25:53 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dct * DescribeCollectionTask ) PostExecute ( ctx context . Context ) error {
2021-02-02 19:54:31 +08:00
return nil
}
2021-05-10 17:39:08 +08:00
type GetCollectionStatisticsTask struct {
2021-02-02 19:54:31 +08:00
Condition
2021-03-12 14:22:09 +08:00
* milvuspb . GetCollectionStatisticsRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
dataService types . DataService
2021-03-12 14:22:09 +08:00
result * milvuspb . GetCollectionStatisticsResponse
2021-02-02 19:54:31 +08:00
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return g . ctx
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) ID ( ) UniqueID {
2021-02-02 19:54:31 +08:00
return g . Base . MsgID
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) SetID ( uid UniqueID ) {
2021-02-02 19:54:31 +08:00
g . Base . MsgID = uid
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) Name ( ) string {
2021-02-23 09:58:06 +08:00
return GetCollectionStatisticsTaskName
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) Type ( ) commonpb . MsgType {
2021-02-02 19:54:31 +08:00
return g . Base . MsgType
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) BeginTs ( ) Timestamp {
2021-02-02 19:54:31 +08:00
return g . Base . Timestamp
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) EndTs ( ) Timestamp {
2021-02-02 19:54:31 +08:00
return g . Base . Timestamp
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) SetTs ( ts Timestamp ) {
2021-02-02 19:54:31 +08:00
g . Base . Timestamp = ts
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) OnEnqueue ( ) error {
2021-02-02 19:54:31 +08:00
g . Base = & commonpb . MsgBase { }
return nil
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
g . Base . MsgType = commonpb . MsgType_GetCollectionStatistics
2021-02-02 19:54:31 +08:00
g . Base . SourceID = Params . ProxyID
return nil
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) Execute ( ctx context . Context ) error {
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , g . CollectionName )
2021-02-02 19:54:31 +08:00
if err != nil {
return err
}
2021-03-12 14:22:09 +08:00
req := & datapb . GetCollectionStatisticsRequest {
2021-02-02 19:54:31 +08:00
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_GetCollectionStatistics ,
2021-02-02 19:54:31 +08:00
MsgID : g . Base . MsgID ,
Timestamp : g . Base . Timestamp ,
SourceID : g . Base . SourceID ,
} ,
CollectionID : collID ,
}
2021-03-08 10:09:48 +08:00
result , _ := g . dataService . GetCollectionStatistics ( ctx , req )
2021-02-04 19:34:35 +08:00
if result == nil {
return errors . New ( "get collection statistics resp is nil" )
}
2021-03-10 22:06:22 +08:00
if result . Status . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( result . Status . Reason )
2021-02-02 19:54:31 +08:00
}
2021-03-12 14:22:09 +08:00
g . result = & milvuspb . GetCollectionStatisticsResponse {
2021-02-02 19:54:31 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-02-02 19:54:31 +08:00
Reason : "" ,
} ,
Stats : result . Stats ,
}
return nil
}
2021-05-10 17:39:08 +08:00
func ( g * GetCollectionStatisticsTask ) PostExecute ( ctx context . Context ) error {
return nil
}
type GetPartitionStatisticsTask struct {
Condition
* milvuspb . GetPartitionStatisticsRequest
ctx context . Context
dataService types . DataService
result * milvuspb . GetPartitionStatisticsResponse
}
func ( g * GetPartitionStatisticsTask ) TraceCtx ( ) context . Context {
return g . ctx
}
func ( g * GetPartitionStatisticsTask ) ID ( ) UniqueID {
return g . Base . MsgID
}
func ( g * GetPartitionStatisticsTask ) SetID ( uid UniqueID ) {
g . Base . MsgID = uid
}
func ( g * GetPartitionStatisticsTask ) Name ( ) string {
return GetPartitionStatisticsTaskName
}
func ( g * GetPartitionStatisticsTask ) Type ( ) commonpb . MsgType {
return g . Base . MsgType
}
func ( g * GetPartitionStatisticsTask ) BeginTs ( ) Timestamp {
return g . Base . Timestamp
}
func ( g * GetPartitionStatisticsTask ) EndTs ( ) Timestamp {
return g . Base . Timestamp
}
func ( g * GetPartitionStatisticsTask ) SetTs ( ts Timestamp ) {
g . Base . Timestamp = ts
}
func ( g * GetPartitionStatisticsTask ) OnEnqueue ( ) error {
g . Base = & commonpb . MsgBase { }
return nil
}
func ( g * GetPartitionStatisticsTask ) PreExecute ( ctx context . Context ) error {
g . Base . MsgType = commonpb . MsgType_GetPartitionStatistics
g . Base . SourceID = Params . ProxyID
return nil
}
func ( g * GetPartitionStatisticsTask ) Execute ( ctx context . Context ) error {
collID , err := globalMetaCache . GetCollectionID ( ctx , g . CollectionName )
if err != nil {
return err
}
partitionID , err := globalMetaCache . GetPartitionID ( ctx , g . CollectionName , g . PartitionName )
if err != nil {
return err
}
req := & datapb . GetPartitionStatisticsRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_GetPartitionStatistics ,
MsgID : g . Base . MsgID ,
Timestamp : g . Base . Timestamp ,
SourceID : g . Base . SourceID ,
} ,
CollectionID : collID ,
PartitionID : partitionID ,
}
result , _ := g . dataService . GetPartitionStatistics ( ctx , req )
if result == nil {
return errors . New ( "get partition statistics resp is nil" )
}
if result . Status . ErrorCode != commonpb . ErrorCode_Success {
return errors . New ( result . Status . Reason )
}
g . result = & milvuspb . GetPartitionStatisticsResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
Reason : "" ,
} ,
Stats : result . Stats ,
}
return nil
}
func ( g * GetPartitionStatisticsTask ) PostExecute ( ctx context . Context ) error {
2020-11-09 17:25:53 +08:00
return nil
}
type ShowCollectionsTask struct {
2020-11-17 20:00:23 +08:00
Condition
2021-03-12 14:22:09 +08:00
* milvuspb . ShowCollectionsRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
2021-06-03 19:09:33 +08:00
queryService types . QueryService
2021-03-12 14:22:09 +08:00
result * milvuspb . ShowCollectionsResponse
2020-11-09 17:25:53 +08:00
}
2021-03-25 14:41:46 +08:00
func ( sct * ShowCollectionsTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return sct . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-12 12:04:12 +08:00
func ( sct * ShowCollectionsTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return sct . Base . MsgID
2020-11-09 17:25:53 +08:00
}
2020-11-23 16:52:17 +08:00
func ( sct * ShowCollectionsTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
sct . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( sct * ShowCollectionsTask ) Name ( ) string {
return ShowCollectionTaskName
}
2021-01-16 15:06:19 +08:00
func ( sct * ShowCollectionsTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return sct . Base . MsgType
2020-11-09 17:25:53 +08:00
}
func ( sct * ShowCollectionsTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return sct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( sct * ShowCollectionsTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return sct . Base . Timestamp
2020-11-09 17:25:53 +08:00
}
func ( sct * ShowCollectionsTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
sct . Base . Timestamp = ts
2020-11-09 17:25:53 +08:00
}
2021-02-23 09:58:06 +08:00
func ( sct * ShowCollectionsTask ) OnEnqueue ( ) error {
sct . Base = & commonpb . MsgBase { }
return nil
}
func ( sct * ShowCollectionsTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
sct . Base . MsgType = commonpb . MsgType_ShowCollections
2021-01-28 20:51:44 +08:00
sct . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2020-11-09 17:25:53 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( sct * ShowCollectionsTask ) Execute ( ctx context . Context ) error {
2021-01-22 09:36:18 +08:00
var err error
2021-06-03 19:09:33 +08:00
respFromMaster , err := sct . masterService . ShowCollections ( ctx , sct . ShowCollectionsRequest )
if err != nil {
return err
2021-02-04 19:34:35 +08:00
}
2021-06-03 19:09:33 +08:00
if respFromMaster == nil {
return errors . New ( "failed to show collections" )
2021-02-04 19:34:35 +08:00
}
2021-06-03 19:09:33 +08:00
if respFromMaster . Status . ErrorCode != commonpb . ErrorCode_Success {
return errors . New ( respFromMaster . Status . Reason )
}
if sct . ShowCollectionsRequest . Type == milvuspb . ShowCollectionsType_InMemory {
resp , err := sct . queryService . ShowCollections ( ctx , & querypb . ShowCollectionsRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_ShowCollections ,
MsgID : sct . ShowCollectionsRequest . Base . MsgID ,
Timestamp : sct . ShowCollectionsRequest . Base . Timestamp ,
SourceID : sct . ShowCollectionsRequest . Base . SourceID ,
} ,
//DbID: sct.ShowCollectionsRequest.DbName,
} )
if err != nil {
return err
}
if resp == nil {
return errors . New ( "failed to show collections" )
}
if resp . Status . ErrorCode != commonpb . ErrorCode_Success {
return errors . New ( resp . Status . Reason )
}
sct . result = & milvuspb . ShowCollectionsResponse {
Status : resp . Status ,
CollectionNames : make ( [ ] string , 0 , len ( resp . CollectionIDs ) ) ,
CollectionIds : make ( [ ] int64 , 0 , len ( resp . CollectionIDs ) ) ,
}
idMap := make ( map [ int64 ] string )
for i , name := range respFromMaster . CollectionNames {
idMap [ respFromMaster . CollectionIds [ i ] ] = name
}
for _ , id := range resp . CollectionIDs {
sct . result . CollectionIds = append ( sct . result . CollectionIds , id )
sct . result . CollectionNames = append ( sct . result . CollectionNames , idMap [ id ] )
}
}
sct . result = respFromMaster
return nil
2020-11-09 17:25:53 +08:00
}
2021-02-23 09:58:06 +08:00
func ( sct * ShowCollectionsTask ) PostExecute ( ctx context . Context ) error {
2020-11-09 17:25:53 +08:00
return nil
}
2020-11-19 17:09:22 +08:00
type CreatePartitionTask struct {
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . CreatePartitionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * commonpb . Status
2020-11-19 17:09:22 +08:00
}
2021-03-25 14:41:46 +08:00
func ( cpt * CreatePartitionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return cpt . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-19 17:09:22 +08:00
func ( cpt * CreatePartitionTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return cpt . Base . MsgID
2020-11-19 17:09:22 +08:00
}
2020-11-23 16:52:17 +08:00
func ( cpt * CreatePartitionTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
cpt . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cpt * CreatePartitionTask ) Name ( ) string {
return CreatePartitionTaskName
}
2021-01-16 15:06:19 +08:00
func ( cpt * CreatePartitionTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return cpt . Base . MsgType
2020-11-19 17:09:22 +08:00
}
func ( cpt * CreatePartitionTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return cpt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( cpt * CreatePartitionTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return cpt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( cpt * CreatePartitionTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
cpt . Base . Timestamp = ts
2020-11-19 17:09:22 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cpt * CreatePartitionTask ) OnEnqueue ( ) error {
cpt . Base = & commonpb . MsgBase { }
return nil
}
func ( cpt * CreatePartitionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
cpt . Base . MsgType = commonpb . MsgType_CreatePartition
2021-01-28 20:51:44 +08:00
cpt . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
collName , partitionTag := cpt . CollectionName , cpt . PartitionName
2020-11-26 16:01:31 +08:00
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
if err := ValidatePartitionTag ( partitionTag , true ) ; err != nil {
return err
}
2020-11-19 17:09:22 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( cpt * CreatePartitionTask ) Execute ( ctx context . Context ) ( err error ) {
2021-03-08 10:09:48 +08:00
cpt . result , err = cpt . masterService . CreatePartition ( ctx , cpt . CreatePartitionRequest )
2021-02-04 19:34:35 +08:00
if cpt . result == nil {
return errors . New ( "get collection statistics resp is nil" )
}
2021-03-10 22:06:22 +08:00
if cpt . result . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( cpt . result . Reason )
}
2020-11-19 17:09:22 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( cpt * CreatePartitionTask ) PostExecute ( ctx context . Context ) error {
2020-11-19 17:09:22 +08:00
return nil
}
type DropPartitionTask struct {
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . DropPartitionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * commonpb . Status
2020-11-19 17:09:22 +08:00
}
2021-03-25 14:41:46 +08:00
func ( dpt * DropPartitionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return dpt . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-19 17:09:22 +08:00
func ( dpt * DropPartitionTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return dpt . Base . MsgID
2020-11-19 17:09:22 +08:00
}
2020-11-23 16:52:17 +08:00
func ( dpt * DropPartitionTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
dpt . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dpt * DropPartitionTask ) Name ( ) string {
return DropPartitionTaskName
}
2021-01-16 15:06:19 +08:00
func ( dpt * DropPartitionTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return dpt . Base . MsgType
2020-11-19 17:09:22 +08:00
}
func ( dpt * DropPartitionTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dpt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( dpt * DropPartitionTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dpt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( dpt * DropPartitionTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
dpt . Base . Timestamp = ts
2020-11-19 17:09:22 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dpt * DropPartitionTask ) OnEnqueue ( ) error {
dpt . Base = & commonpb . MsgBase { }
return nil
}
func ( dpt * DropPartitionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
dpt . Base . MsgType = commonpb . MsgType_DropPartition
2021-01-28 20:51:44 +08:00
dpt . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
collName , partitionTag := dpt . CollectionName , dpt . PartitionName
2020-11-26 16:01:31 +08:00
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
if err := ValidatePartitionTag ( partitionTag , true ) ; err != nil {
return err
}
2020-11-19 17:09:22 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( dpt * DropPartitionTask ) Execute ( ctx context . Context ) ( err error ) {
2021-03-08 10:09:48 +08:00
dpt . result , err = dpt . masterService . DropPartition ( ctx , dpt . DropPartitionRequest )
2021-02-04 19:34:35 +08:00
if dpt . result == nil {
return errors . New ( "get collection statistics resp is nil" )
}
2021-03-10 22:06:22 +08:00
if dpt . result . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( dpt . result . Reason )
}
2020-11-19 17:09:22 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( dpt * DropPartitionTask ) PostExecute ( ctx context . Context ) error {
2020-11-19 17:09:22 +08:00
return nil
}
type HasPartitionTask struct {
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . HasPartitionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * milvuspb . BoolResponse
2020-11-19 17:09:22 +08:00
}
2021-03-25 14:41:46 +08:00
func ( hpt * HasPartitionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return hpt . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-19 17:09:22 +08:00
func ( hpt * HasPartitionTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return hpt . Base . MsgID
2020-11-19 17:09:22 +08:00
}
2020-11-23 16:52:17 +08:00
func ( hpt * HasPartitionTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
hpt . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( hpt * HasPartitionTask ) Name ( ) string {
return HasPartitionTaskName
}
2021-01-16 15:06:19 +08:00
func ( hpt * HasPartitionTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return hpt . Base . MsgType
2020-11-19 17:09:22 +08:00
}
func ( hpt * HasPartitionTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return hpt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( hpt * HasPartitionTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return hpt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( hpt * HasPartitionTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
hpt . Base . Timestamp = ts
2020-11-19 17:09:22 +08:00
}
2021-02-23 09:58:06 +08:00
func ( hpt * HasPartitionTask ) OnEnqueue ( ) error {
hpt . Base = & commonpb . MsgBase { }
return nil
}
func ( hpt * HasPartitionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
hpt . Base . MsgType = commonpb . MsgType_HasPartition
2021-01-28 20:51:44 +08:00
hpt . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
collName , partitionTag := hpt . CollectionName , hpt . PartitionName
2020-11-26 16:01:31 +08:00
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
if err := ValidatePartitionTag ( partitionTag , true ) ; err != nil {
return err
}
2020-11-19 17:09:22 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( hpt * HasPartitionTask ) Execute ( ctx context . Context ) ( err error ) {
2021-03-08 10:09:48 +08:00
hpt . result , err = hpt . masterService . HasPartition ( ctx , hpt . HasPartitionRequest )
2021-02-04 19:34:35 +08:00
if hpt . result == nil {
return errors . New ( "get collection statistics resp is nil" )
}
2021-03-10 22:06:22 +08:00
if hpt . result . Status . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( hpt . result . Status . Reason )
}
2020-11-19 17:09:22 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( hpt * HasPartitionTask ) PostExecute ( ctx context . Context ) error {
2020-11-19 17:09:22 +08:00
return nil
}
type ShowPartitionsTask struct {
Condition
2021-03-12 14:22:09 +08:00
* milvuspb . ShowPartitionsRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
2021-03-12 14:22:09 +08:00
result * milvuspb . ShowPartitionsResponse
2020-11-19 17:09:22 +08:00
}
2021-03-25 14:41:46 +08:00
func ( spt * ShowPartitionsTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return spt . ctx
2021-01-22 09:36:18 +08:00
}
2020-11-19 17:09:22 +08:00
func ( spt * ShowPartitionsTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return spt . Base . MsgID
2020-11-19 17:09:22 +08:00
}
2020-11-23 16:52:17 +08:00
func ( spt * ShowPartitionsTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
spt . Base . MsgID = uid
2020-11-23 16:52:17 +08:00
}
2021-02-23 09:58:06 +08:00
func ( spt * ShowPartitionsTask ) Name ( ) string {
return ShowPartitionTaskName
}
2021-01-16 15:06:19 +08:00
func ( spt * ShowPartitionsTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return spt . Base . MsgType
2020-11-19 17:09:22 +08:00
}
func ( spt * ShowPartitionsTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return spt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( spt * ShowPartitionsTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return spt . Base . Timestamp
2020-11-19 17:09:22 +08:00
}
func ( spt * ShowPartitionsTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
spt . Base . Timestamp = ts
2020-11-19 17:09:22 +08:00
}
2021-02-23 09:58:06 +08:00
func ( spt * ShowPartitionsTask ) OnEnqueue ( ) error {
spt . Base = & commonpb . MsgBase { }
return nil
}
func ( spt * ShowPartitionsTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
spt . Base . MsgType = commonpb . MsgType_ShowPartitions
2021-01-28 20:51:44 +08:00
spt . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-01-18 19:32:08 +08:00
if err := ValidateCollectionName ( spt . CollectionName ) ; err != nil {
2020-11-26 16:01:31 +08:00
return err
}
2020-11-19 17:09:22 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( spt * ShowPartitionsTask ) Execute ( ctx context . Context ) error {
2021-01-22 09:36:18 +08:00
var err error
2021-03-12 14:22:09 +08:00
spt . result , err = spt . masterService . ShowPartitions ( ctx , spt . ShowPartitionsRequest )
2021-02-04 19:34:35 +08:00
if spt . result == nil {
return errors . New ( "get collection statistics resp is nil" )
2021-01-31 14:55:36 +08:00
}
2021-03-10 22:06:22 +08:00
if spt . result . Status . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( spt . result . Status . Reason )
}
return err
2020-11-19 17:09:22 +08:00
}
2021-02-23 09:58:06 +08:00
func ( spt * ShowPartitionsTask ) PostExecute ( ctx context . Context ) error {
2020-11-19 17:09:22 +08:00
return nil
}
2020-12-22 15:39:10 +08:00
type CreateIndexTask struct {
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . CreateIndexRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * commonpb . Status
2020-12-22 15:39:10 +08:00
}
2021-03-25 14:41:46 +08:00
func ( cit * CreateIndexTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return cit . ctx
2021-01-22 09:36:18 +08:00
}
2020-12-22 15:39:10 +08:00
func ( cit * CreateIndexTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return cit . Base . MsgID
2020-12-22 15:39:10 +08:00
}
func ( cit * CreateIndexTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
cit . Base . MsgID = uid
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cit * CreateIndexTask ) Name ( ) string {
return CreateIndexTaskName
}
2021-01-16 15:06:19 +08:00
func ( cit * CreateIndexTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return cit . Base . MsgType
2020-12-22 15:39:10 +08:00
}
func ( cit * CreateIndexTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return cit . Base . Timestamp
2020-12-22 15:39:10 +08:00
}
func ( cit * CreateIndexTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return cit . Base . Timestamp
2020-12-22 15:39:10 +08:00
}
func ( cit * CreateIndexTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
cit . Base . Timestamp = ts
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( cit * CreateIndexTask ) OnEnqueue ( ) error {
cit . Base = & commonpb . MsgBase { }
return nil
}
func ( cit * CreateIndexTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
cit . Base . MsgType = commonpb . MsgType_CreateIndex
2021-01-28 20:51:44 +08:00
cit . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2020-12-22 15:39:10 +08:00
collName , fieldName := cit . CollectionName , cit . FieldName
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
if err := ValidateFieldName ( fieldName ) ; err != nil {
return err
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( cit * CreateIndexTask ) Execute ( ctx context . Context ) error {
2021-02-04 19:34:35 +08:00
var err error
2021-03-08 10:09:48 +08:00
cit . result , err = cit . masterService . CreateIndex ( ctx , cit . CreateIndexRequest )
2021-02-04 19:34:35 +08:00
if cit . result == nil {
return errors . New ( "get collection statistics resp is nil" )
}
2021-03-10 22:06:22 +08:00
if cit . result . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( cit . result . Reason )
}
2020-12-22 15:39:10 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( cit * CreateIndexTask ) PostExecute ( ctx context . Context ) error {
2020-12-22 15:39:10 +08:00
return nil
}
type DescribeIndexTask struct {
Condition
2021-01-22 09:36:18 +08:00
* milvuspb . DescribeIndexRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
masterService types . MasterService
result * milvuspb . DescribeIndexResponse
2020-12-22 15:39:10 +08:00
}
2021-03-25 14:41:46 +08:00
func ( dit * DescribeIndexTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return dit . ctx
2021-01-22 09:36:18 +08:00
}
2020-12-22 15:39:10 +08:00
func ( dit * DescribeIndexTask ) ID ( ) UniqueID {
2021-01-18 19:32:08 +08:00
return dit . Base . MsgID
2020-12-22 15:39:10 +08:00
}
func ( dit * DescribeIndexTask ) SetID ( uid UniqueID ) {
2021-01-18 19:32:08 +08:00
dit . Base . MsgID = uid
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dit * DescribeIndexTask ) Name ( ) string {
return DescribeIndexTaskName
}
2021-01-16 15:06:19 +08:00
func ( dit * DescribeIndexTask ) Type ( ) commonpb . MsgType {
2021-01-18 19:32:08 +08:00
return dit . Base . MsgType
2020-12-22 15:39:10 +08:00
}
func ( dit * DescribeIndexTask ) BeginTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dit . Base . Timestamp
2020-12-22 15:39:10 +08:00
}
func ( dit * DescribeIndexTask ) EndTs ( ) Timestamp {
2021-01-18 19:32:08 +08:00
return dit . Base . Timestamp
2020-12-22 15:39:10 +08:00
}
func ( dit * DescribeIndexTask ) SetTs ( ts Timestamp ) {
2021-01-18 19:32:08 +08:00
dit . Base . Timestamp = ts
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( dit * DescribeIndexTask ) OnEnqueue ( ) error {
dit . Base = & commonpb . MsgBase { }
return nil
}
func ( dit * DescribeIndexTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
dit . Base . MsgType = commonpb . MsgType_DescribeIndex
2021-01-28 20:51:44 +08:00
dit . Base . SourceID = Params . ProxyID
2021-01-22 09:36:18 +08:00
2021-04-28 17:48:38 +08:00
if err := ValidateCollectionName ( dit . CollectionName ) ; err != nil {
2020-12-22 15:39:10 +08:00
return err
}
2021-02-19 09:52:06 +08:00
// only support default index name for now. @2021.02.18
if dit . IndexName == "" {
dit . IndexName = Params . DefaultIndexName
}
2020-12-22 15:39:10 +08:00
return nil
}
2021-02-23 09:58:06 +08:00
func ( dit * DescribeIndexTask ) Execute ( ctx context . Context ) error {
2021-01-22 09:36:18 +08:00
var err error
2021-03-08 10:09:48 +08:00
dit . result , err = dit . masterService . DescribeIndex ( ctx , dit . DescribeIndexRequest )
2021-02-04 19:34:35 +08:00
if dit . result == nil {
return errors . New ( "get collection statistics resp is nil" )
}
2021-03-10 22:06:22 +08:00
if dit . result . Status . ErrorCode != commonpb . ErrorCode_Success {
2021-02-04 19:34:35 +08:00
return errors . New ( dit . result . Status . Reason )
}
2020-12-22 15:39:10 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( dit * DescribeIndexTask ) PostExecute ( ctx context . Context ) error {
2020-12-22 15:39:10 +08:00
return nil
}
2021-02-20 18:30:37 +08:00
type DropIndexTask struct {
Condition
2021-02-23 09:58:06 +08:00
ctx context . Context
2021-02-20 18:30:37 +08:00
* milvuspb . DropIndexRequest
2021-03-08 10:09:48 +08:00
masterService types . MasterService
result * commonpb . Status
2021-02-20 18:30:37 +08:00
}
2021-03-25 14:41:46 +08:00
func ( dit * DropIndexTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return dit . ctx
2021-02-20 18:30:37 +08:00
}
func ( dit * DropIndexTask ) ID ( ) UniqueID {
return dit . Base . MsgID
}
func ( dit * DropIndexTask ) SetID ( uid UniqueID ) {
dit . Base . MsgID = uid
}
2021-02-23 09:58:06 +08:00
func ( dit * DropIndexTask ) Name ( ) string {
return DropIndexTaskName
}
2021-02-20 18:30:37 +08:00
func ( dit * DropIndexTask ) Type ( ) commonpb . MsgType {
return dit . Base . MsgType
}
func ( dit * DropIndexTask ) BeginTs ( ) Timestamp {
return dit . Base . Timestamp
}
func ( dit * DropIndexTask ) EndTs ( ) Timestamp {
return dit . Base . Timestamp
}
func ( dit * DropIndexTask ) SetTs ( ts Timestamp ) {
dit . Base . Timestamp = ts
}
2021-02-23 09:58:06 +08:00
func ( dit * DropIndexTask ) OnEnqueue ( ) error {
dit . Base = & commonpb . MsgBase { }
return nil
}
func ( dit * DropIndexTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
dit . Base . MsgType = commonpb . MsgType_DropIndex
2021-02-20 18:30:37 +08:00
dit . Base . SourceID = Params . ProxyID
collName , fieldName := dit . CollectionName , dit . FieldName
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
if err := ValidateFieldName ( fieldName ) ; err != nil {
return err
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( dit * DropIndexTask ) Execute ( ctx context . Context ) error {
2021-02-20 18:30:37 +08:00
var err error
2021-03-08 10:09:48 +08:00
dit . result , err = dit . masterService . DropIndex ( ctx , dit . DropIndexRequest )
2021-02-20 18:30:37 +08:00
if dit . result == nil {
return errors . New ( "drop index resp is nil" )
}
2021-03-10 22:06:22 +08:00
if dit . result . ErrorCode != commonpb . ErrorCode_Success {
2021-02-20 18:30:37 +08:00
return errors . New ( dit . result . Reason )
}
return err
}
2021-02-23 09:58:06 +08:00
func ( dit * DropIndexTask ) PostExecute ( ctx context . Context ) error {
2021-02-20 18:30:37 +08:00
return nil
}
2021-04-28 11:15:28 +08:00
type GetIndexBuildProgressTask struct {
Condition
* milvuspb . GetIndexBuildProgressRequest
ctx context . Context
indexService types . IndexService
masterService types . MasterService
dataService types . DataService
result * milvuspb . GetIndexBuildProgressResponse
}
func ( gibpt * GetIndexBuildProgressTask ) TraceCtx ( ) context . Context {
return gibpt . ctx
}
func ( gibpt * GetIndexBuildProgressTask ) ID ( ) UniqueID {
return gibpt . Base . MsgID
}
func ( gibpt * GetIndexBuildProgressTask ) SetID ( uid UniqueID ) {
gibpt . Base . MsgID = uid
}
func ( gibpt * GetIndexBuildProgressTask ) Name ( ) string {
return GetIndexBuildProgressTaskName
}
func ( gibpt * GetIndexBuildProgressTask ) Type ( ) commonpb . MsgType {
return gibpt . Base . MsgType
}
func ( gibpt * GetIndexBuildProgressTask ) BeginTs ( ) Timestamp {
return gibpt . Base . Timestamp
}
func ( gibpt * GetIndexBuildProgressTask ) EndTs ( ) Timestamp {
return gibpt . Base . Timestamp
}
func ( gibpt * GetIndexBuildProgressTask ) SetTs ( ts Timestamp ) {
gibpt . Base . Timestamp = ts
}
func ( gibpt * GetIndexBuildProgressTask ) OnEnqueue ( ) error {
gibpt . Base = & commonpb . MsgBase { }
return nil
}
func ( gibpt * GetIndexBuildProgressTask ) PreExecute ( ctx context . Context ) error {
gibpt . Base . MsgType = commonpb . MsgType_GetIndexBuildProgress
gibpt . Base . SourceID = Params . ProxyID
if err := ValidateCollectionName ( gibpt . CollectionName ) ; err != nil {
return err
}
return nil
}
func ( gibpt * GetIndexBuildProgressTask ) Execute ( ctx context . Context ) error {
collectionName := gibpt . CollectionName
collectionID , err := globalMetaCache . GetCollectionID ( ctx , collectionName )
if err != nil { // err is not nil if collection not exists
return err
}
showPartitionRequest := & milvuspb . ShowPartitionsRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_ShowPartitions ,
MsgID : gibpt . Base . MsgID ,
Timestamp : gibpt . Base . Timestamp ,
SourceID : Params . ProxyID ,
} ,
DbName : gibpt . DbName ,
CollectionName : collectionName ,
CollectionID : collectionID ,
}
partitions , err := gibpt . masterService . ShowPartitions ( ctx , showPartitionRequest )
if err != nil {
return err
}
if gibpt . IndexName == "" {
gibpt . IndexName = Params . DefaultIndexName
}
describeIndexReq := milvuspb . DescribeIndexRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_DescribeIndex ,
MsgID : gibpt . Base . MsgID ,
Timestamp : gibpt . Base . Timestamp ,
SourceID : Params . ProxyID ,
} ,
DbName : gibpt . DbName ,
CollectionName : gibpt . CollectionName ,
// IndexName: gibpt.IndexName,
}
indexDescriptionResp , err2 := gibpt . masterService . DescribeIndex ( ctx , & describeIndexReq )
if err2 != nil {
return err2
}
matchIndexID := int64 ( - 1 )
foundIndexID := false
for _ , desc := range indexDescriptionResp . IndexDescriptions {
if desc . IndexName == gibpt . IndexName {
matchIndexID = desc . IndexID
foundIndexID = true
break
}
}
if ! foundIndexID {
return errors . New ( fmt . Sprint ( "Can't found IndexID for indexName" , gibpt . IndexName ) )
}
var allSegmentIDs [ ] UniqueID
for _ , partitionID := range partitions . PartitionIDs {
showSegmentsRequest := & milvuspb . ShowSegmentsRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_ShowSegments ,
MsgID : gibpt . Base . MsgID ,
Timestamp : gibpt . Base . Timestamp ,
SourceID : Params . ProxyID ,
} ,
CollectionID : collectionID ,
PartitionID : partitionID ,
}
segments , err := gibpt . masterService . ShowSegments ( ctx , showSegmentsRequest )
if err != nil {
return err
}
if segments . Status . ErrorCode != commonpb . ErrorCode_Success {
return errors . New ( segments . Status . Reason )
}
allSegmentIDs = append ( allSegmentIDs , segments . SegmentIDs ... )
}
getIndexStatesRequest := & indexpb . GetIndexStatesRequest {
IndexBuildIDs : make ( [ ] UniqueID , 0 ) ,
}
buildIndexMap := make ( map [ int64 ] int64 )
for _ , segmentID := range allSegmentIDs {
describeSegmentRequest := & milvuspb . DescribeSegmentRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_DescribeSegment ,
MsgID : gibpt . Base . MsgID ,
Timestamp : gibpt . Base . Timestamp ,
SourceID : Params . ProxyID ,
} ,
CollectionID : collectionID ,
SegmentID : segmentID ,
}
segmentDesc , err := gibpt . masterService . DescribeSegment ( ctx , describeSegmentRequest )
if err != nil {
return err
}
if segmentDesc . IndexID == matchIndexID {
if segmentDesc . EnableIndex {
getIndexStatesRequest . IndexBuildIDs = append ( getIndexStatesRequest . IndexBuildIDs , segmentDesc . BuildID )
buildIndexMap [ segmentID ] = segmentDesc . BuildID
}
}
}
states , err := gibpt . indexService . GetIndexStates ( ctx , getIndexStatesRequest )
if err != nil {
return err
}
if states . Status . ErrorCode != commonpb . ErrorCode_Success {
gibpt . result = & milvuspb . GetIndexBuildProgressResponse {
Status : states . Status ,
}
}
buildFinishMap := make ( map [ int64 ] bool )
for _ , state := range states . States {
if state . State == commonpb . IndexState_Finished {
buildFinishMap [ state . IndexBuildID ] = true
}
}
infoResp , err := gibpt . dataService . GetSegmentInfo ( ctx , & datapb . GetSegmentInfoRequest {
Base : & commonpb . MsgBase {
MsgType : commonpb . MsgType_SegmentInfo ,
MsgID : 0 ,
Timestamp : 0 ,
SourceID : Params . ProxyID ,
} ,
SegmentIDs : allSegmentIDs ,
} )
if err != nil {
return err
}
total := int64 ( 0 )
indexed := int64 ( 0 )
for _ , info := range infoResp . Infos {
2021-06-04 11:45:45 +08:00
total += info . NumOfRows
2021-04-28 11:15:28 +08:00
if buildFinishMap [ buildIndexMap [ info . ID ] ] {
2021-06-04 11:45:45 +08:00
indexed += info . NumOfRows
2021-04-28 11:15:28 +08:00
}
}
gibpt . result = & milvuspb . GetIndexBuildProgressResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
Reason : "" ,
} ,
TotalRows : total ,
IndexedRows : indexed ,
}
return nil
}
func ( gibpt * GetIndexBuildProgressTask ) PostExecute ( ctx context . Context ) error {
return nil
}
2021-01-22 09:36:18 +08:00
type GetIndexStateTask struct {
2020-12-22 15:39:10 +08:00
Condition
2021-03-12 14:22:09 +08:00
* milvuspb . GetIndexStateRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
indexService types . IndexService
masterService types . MasterService
2021-03-12 14:22:09 +08:00
result * milvuspb . GetIndexStateResponse
2020-12-22 15:39:10 +08:00
}
2021-03-25 14:41:46 +08:00
func ( gist * GetIndexStateTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return gist . ctx
2021-01-22 09:36:18 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) ID ( ) UniqueID {
return gist . Base . MsgID
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) SetID ( uid UniqueID ) {
gist . Base . MsgID = uid
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) Name ( ) string {
return GetIndexStateTaskName
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) Type ( ) commonpb . MsgType {
return gist . Base . MsgType
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) BeginTs ( ) Timestamp {
return gist . Base . Timestamp
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) EndTs ( ) Timestamp {
return gist . Base . Timestamp
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) SetTs ( ts Timestamp ) {
gist . Base . Timestamp = ts
}
func ( gist * GetIndexStateTask ) OnEnqueue ( ) error {
gist . Base = & commonpb . MsgBase { }
return nil
}
2021-01-22 09:36:18 +08:00
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
gist . Base . MsgType = commonpb . MsgType_GetIndexState
2021-02-23 09:58:06 +08:00
gist . Base . SourceID = Params . ProxyID
2021-04-28 11:15:28 +08:00
if err := ValidateCollectionName ( gist . CollectionName ) ; err != nil {
2020-12-22 15:39:10 +08:00
return err
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) Execute ( ctx context . Context ) error {
collectionName := gist . CollectionName
2021-02-26 17:44:24 +08:00
collectionID , err := globalMetaCache . GetCollectionID ( ctx , collectionName )
2021-02-04 14:37:12 +08:00
if err != nil { // err is not nil if collection not exists
return err
}
2021-03-12 14:22:09 +08:00
showPartitionRequest := & milvuspb . ShowPartitionsRequest {
2021-02-04 14:37:12 +08:00
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_ShowPartitions ,
2021-02-23 09:58:06 +08:00
MsgID : gist . Base . MsgID ,
Timestamp : gist . Base . Timestamp ,
2021-02-04 14:37:12 +08:00
SourceID : Params . ProxyID ,
} ,
2021-02-23 09:58:06 +08:00
DbName : gist . DbName ,
2021-02-04 14:37:12 +08:00
CollectionName : collectionName ,
CollectionID : collectionID ,
}
2021-03-08 10:09:48 +08:00
partitions , err := gist . masterService . ShowPartitions ( ctx , showPartitionRequest )
2021-02-04 14:37:12 +08:00
if err != nil {
return err
}
2021-02-23 09:58:06 +08:00
if gist . IndexName == "" {
gist . IndexName = Params . DefaultIndexName
2021-02-08 14:20:29 +08:00
}
describeIndexReq := milvuspb . DescribeIndexRequest {
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_DescribeIndex ,
2021-02-23 09:58:06 +08:00
MsgID : gist . Base . MsgID ,
Timestamp : gist . Base . Timestamp ,
2021-02-08 14:20:29 +08:00
SourceID : Params . ProxyID ,
} ,
2021-02-23 09:58:06 +08:00
DbName : gist . DbName ,
CollectionName : gist . CollectionName ,
IndexName : gist . IndexName ,
2021-02-08 14:20:29 +08:00
}
2021-03-08 10:09:48 +08:00
indexDescriptionResp , err2 := gist . masterService . DescribeIndex ( ctx , & describeIndexReq )
2021-02-08 14:20:29 +08:00
if err2 != nil {
return err2
}
matchIndexID := int64 ( - 1 )
foundIndexID := false
for _ , desc := range indexDescriptionResp . IndexDescriptions {
2021-02-23 09:58:06 +08:00
if desc . IndexName == gist . IndexName {
2021-02-08 14:20:29 +08:00
matchIndexID = desc . IndexID
foundIndexID = true
break
}
}
if ! foundIndexID {
2021-02-23 09:58:06 +08:00
return errors . New ( fmt . Sprint ( "Can't found IndexID for indexName" , gist . IndexName ) )
2021-02-08 14:20:29 +08:00
}
2021-02-19 09:52:06 +08:00
var allSegmentIDs [ ] UniqueID
2021-02-04 14:37:12 +08:00
for _ , partitionID := range partitions . PartitionIDs {
2021-03-12 14:22:09 +08:00
showSegmentsRequest := & milvuspb . ShowSegmentsRequest {
2021-02-04 14:37:12 +08:00
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_ShowSegments ,
2021-02-23 09:58:06 +08:00
MsgID : gist . Base . MsgID ,
Timestamp : gist . Base . Timestamp ,
2021-02-04 14:37:12 +08:00
SourceID : Params . ProxyID ,
} ,
CollectionID : collectionID ,
PartitionID : partitionID ,
}
2021-03-08 10:09:48 +08:00
segments , err := gist . masterService . ShowSegments ( ctx , showSegmentsRequest )
2021-02-04 14:37:12 +08:00
if err != nil {
return err
}
2021-03-10 22:06:22 +08:00
if segments . Status . ErrorCode != commonpb . ErrorCode_Success {
2021-02-19 09:52:06 +08:00
return errors . New ( segments . Status . Reason )
2021-02-04 14:37:12 +08:00
}
2021-02-19 09:52:06 +08:00
allSegmentIDs = append ( allSegmentIDs , segments . SegmentIDs ... )
}
2021-03-12 14:22:09 +08:00
getIndexStatesRequest := & indexpb . GetIndexStatesRequest {
2021-02-19 09:52:06 +08:00
IndexBuildIDs : make ( [ ] UniqueID , 0 ) ,
}
2021-03-08 15:46:51 +08:00
enableIndexBitMap := make ( [ ] bool , 0 )
indexBuildIDs := make ( [ ] UniqueID , 0 )
2021-02-04 14:37:12 +08:00
2021-02-19 09:52:06 +08:00
for _ , segmentID := range allSegmentIDs {
describeSegmentRequest := & milvuspb . DescribeSegmentRequest {
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_DescribeSegment ,
2021-02-23 09:58:06 +08:00
MsgID : gist . Base . MsgID ,
Timestamp : gist . Base . Timestamp ,
2021-02-19 09:52:06 +08:00
SourceID : Params . ProxyID ,
} ,
CollectionID : collectionID ,
SegmentID : segmentID ,
}
2021-03-08 10:09:48 +08:00
segmentDesc , err := gist . masterService . DescribeSegment ( ctx , describeSegmentRequest )
2021-02-04 14:37:12 +08:00
if err != nil {
return err
}
2021-02-19 09:52:06 +08:00
if segmentDesc . IndexID == matchIndexID {
2021-03-08 15:46:51 +08:00
indexBuildIDs = append ( indexBuildIDs , segmentDesc . BuildID )
if segmentDesc . EnableIndex {
enableIndexBitMap = append ( enableIndexBitMap , true )
} else {
enableIndexBitMap = append ( enableIndexBitMap , false )
}
2021-02-19 09:52:06 +08:00
}
}
2021-02-04 14:37:12 +08:00
2021-03-08 19:39:36 +08:00
log . Debug ( "proxynode" , zap . Int ( "GetIndexState:: len of allSegmentIDs" , len ( allSegmentIDs ) ) )
log . Debug ( "proxynode" , zap . Int ( "GetIndexState:: len of IndexBuildIDs" , len ( indexBuildIDs ) ) )
2021-03-08 15:46:51 +08:00
if len ( allSegmentIDs ) != len ( indexBuildIDs ) {
2021-03-12 14:22:09 +08:00
gist . result = & milvuspb . GetIndexStateResponse {
2021-02-19 09:52:06 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-02-19 09:52:06 +08:00
Reason : "" ,
} ,
2021-03-11 14:14:29 +08:00
State : commonpb . IndexState_InProgress ,
2021-02-19 09:52:06 +08:00
}
return err
}
2021-03-08 15:46:51 +08:00
for idx , enableIndex := range enableIndexBitMap {
if enableIndex {
getIndexStatesRequest . IndexBuildIDs = append ( getIndexStatesRequest . IndexBuildIDs , indexBuildIDs [ idx ] )
}
}
2021-03-08 10:09:48 +08:00
states , err := gist . indexService . GetIndexStates ( ctx , getIndexStatesRequest )
2021-02-19 09:52:06 +08:00
if err != nil {
return err
}
2021-03-10 22:06:22 +08:00
if states . Status . ErrorCode != commonpb . ErrorCode_Success {
2021-03-12 14:22:09 +08:00
gist . result = & milvuspb . GetIndexStateResponse {
2021-02-19 09:52:06 +08:00
Status : states . Status ,
2021-03-11 14:14:29 +08:00
State : commonpb . IndexState_Failed ,
2021-02-19 09:52:06 +08:00
}
return nil
}
for _ , state := range states . States {
2021-03-11 14:14:29 +08:00
if state . State != commonpb . IndexState_Finished {
2021-03-12 14:22:09 +08:00
gist . result = & milvuspb . GetIndexStateResponse {
2021-02-04 14:37:12 +08:00
Status : states . Status ,
2021-02-19 09:52:06 +08:00
State : state . State ,
2021-02-04 14:37:12 +08:00
}
2021-02-19 09:52:06 +08:00
return nil
2021-02-04 14:37:12 +08:00
}
}
2021-03-12 14:22:09 +08:00
gist . result = & milvuspb . GetIndexStateResponse {
2021-01-28 20:51:44 +08:00
Status : & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-01-28 20:51:44 +08:00
Reason : "" ,
} ,
2021-03-11 14:14:29 +08:00
State : commonpb . IndexState_Finished ,
2021-01-28 20:51:44 +08:00
}
2021-02-04 14:37:12 +08:00
2021-01-28 20:51:44 +08:00
return nil
2020-12-22 15:39:10 +08:00
}
2021-02-23 09:58:06 +08:00
func ( gist * GetIndexStateTask ) PostExecute ( ctx context . Context ) error {
2020-12-22 15:39:10 +08:00
return nil
}
2021-02-02 10:58:39 +08:00
type FlushTask struct {
Condition
* milvuspb . FlushRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
dataService types . DataService
result * commonpb . Status
2021-02-02 10:58:39 +08:00
}
2021-03-25 14:41:46 +08:00
func ( ft * FlushTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return ft . ctx
2021-02-02 10:58:39 +08:00
}
func ( ft * FlushTask ) ID ( ) UniqueID {
return ft . Base . MsgID
}
func ( ft * FlushTask ) SetID ( uid UniqueID ) {
ft . Base . MsgID = uid
}
2021-02-23 09:58:06 +08:00
func ( ft * FlushTask ) Name ( ) string {
return FlushTaskName
}
2021-02-02 10:58:39 +08:00
func ( ft * FlushTask ) Type ( ) commonpb . MsgType {
return ft . Base . MsgType
}
func ( ft * FlushTask ) BeginTs ( ) Timestamp {
return ft . Base . Timestamp
}
func ( ft * FlushTask ) EndTs ( ) Timestamp {
return ft . Base . Timestamp
}
func ( ft * FlushTask ) SetTs ( ts Timestamp ) {
ft . Base . Timestamp = ts
}
2021-02-23 09:58:06 +08:00
func ( ft * FlushTask ) OnEnqueue ( ) error {
ft . Base = & commonpb . MsgBase { }
return nil
}
func ( ft * FlushTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
ft . Base . MsgType = commonpb . MsgType_Flush
2021-02-02 10:58:39 +08:00
ft . Base . SourceID = Params . ProxyID
return nil
}
2021-02-23 09:58:06 +08:00
func ( ft * FlushTask ) Execute ( ctx context . Context ) error {
2021-02-03 17:30:10 +08:00
for _ , collName := range ft . CollectionNames {
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , collName )
2021-02-03 17:30:10 +08:00
if err != nil {
return err
}
flushReq := & datapb . FlushRequest {
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_Flush ,
2021-02-03 17:30:10 +08:00
MsgID : ft . Base . MsgID ,
Timestamp : ft . Base . Timestamp ,
SourceID : ft . Base . SourceID ,
} ,
DbID : 0 ,
CollectionID : collID ,
}
var status * commonpb . Status
2021-03-08 10:09:48 +08:00
status , _ = ft . dataService . Flush ( ctx , flushReq )
2021-02-04 19:34:35 +08:00
if status == nil {
return errors . New ( "flush resp is nil" )
2021-02-03 17:30:10 +08:00
}
2021-03-10 22:06:22 +08:00
if status . ErrorCode != commonpb . ErrorCode_Success {
2021-02-03 17:30:10 +08:00
return errors . New ( status . Reason )
}
2021-02-02 10:58:39 +08:00
}
2021-02-03 17:30:10 +08:00
ft . result = & commonpb . Status {
2021-03-10 22:06:22 +08:00
ErrorCode : commonpb . ErrorCode_Success ,
2021-02-02 10:58:39 +08:00
}
2021-02-03 17:30:10 +08:00
return nil
2021-02-02 10:58:39 +08:00
}
2021-02-23 09:58:06 +08:00
func ( ft * FlushTask ) PostExecute ( ctx context . Context ) error {
2021-02-02 10:58:39 +08:00
return nil
}
2021-02-04 15:31:02 +08:00
type LoadCollectionTask struct {
Condition
* milvuspb . LoadCollectionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
queryService types . QueryService
result * commonpb . Status
2021-02-04 15:31:02 +08:00
}
2021-03-25 14:41:46 +08:00
func ( lct * LoadCollectionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return lct . ctx
2021-02-04 15:31:02 +08:00
}
func ( lct * LoadCollectionTask ) ID ( ) UniqueID {
return lct . Base . MsgID
}
func ( lct * LoadCollectionTask ) SetID ( uid UniqueID ) {
lct . Base . MsgID = uid
}
2021-02-23 09:58:06 +08:00
func ( lct * LoadCollectionTask ) Name ( ) string {
return LoadCollectionTaskName
}
2021-02-04 15:31:02 +08:00
func ( lct * LoadCollectionTask ) Type ( ) commonpb . MsgType {
return lct . Base . MsgType
}
func ( lct * LoadCollectionTask ) BeginTs ( ) Timestamp {
return lct . Base . Timestamp
}
func ( lct * LoadCollectionTask ) EndTs ( ) Timestamp {
return lct . Base . Timestamp
}
func ( lct * LoadCollectionTask ) SetTs ( ts Timestamp ) {
lct . Base . Timestamp = ts
}
2021-02-23 09:58:06 +08:00
func ( lct * LoadCollectionTask ) OnEnqueue ( ) error {
lct . Base = & commonpb . MsgBase { }
return nil
}
func ( lct * LoadCollectionTask ) PreExecute ( ctx context . Context ) error {
2021-03-13 11:59:24 +08:00
log . Debug ( "LoadCollectionTask PreExecute" , zap . String ( "role" , Params . RoleName ) , zap . Int64 ( "msgID" , lct . Base . MsgID ) )
2021-03-10 14:45:35 +08:00
lct . Base . MsgType = commonpb . MsgType_LoadCollection
2021-02-04 15:31:02 +08:00
lct . Base . SourceID = Params . ProxyID
collName := lct . CollectionName
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( lct * LoadCollectionTask ) Execute ( ctx context . Context ) ( err error ) {
2021-03-13 11:59:24 +08:00
log . Debug ( "LoadCollectionTask Execute" , zap . String ( "role" , Params . RoleName ) , zap . Int64 ( "msgID" , lct . Base . MsgID ) )
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , lct . CollectionName )
2021-02-04 15:31:02 +08:00
if err != nil {
return err
}
2021-02-26 17:44:24 +08:00
collSchema , err := globalMetaCache . GetCollectionSchema ( ctx , lct . CollectionName )
2021-02-06 21:17:18 +08:00
if err != nil {
return err
}
2021-02-04 15:31:02 +08:00
request := & querypb . LoadCollectionRequest {
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_LoadCollection ,
2021-02-04 15:31:02 +08:00
MsgID : lct . Base . MsgID ,
Timestamp : lct . Base . Timestamp ,
SourceID : lct . Base . SourceID ,
} ,
DbID : 0 ,
CollectionID : collID ,
2021-02-06 21:17:18 +08:00
Schema : collSchema ,
2021-02-04 15:31:02 +08:00
}
2021-03-13 11:59:24 +08:00
log . Debug ( "send LoadCollectionRequest to query service" , zap . String ( "role" , Params . RoleName ) , zap . Int64 ( "msgID" , request . Base . MsgID ) , zap . Int64 ( "collectionID" , request . CollectionID ) ,
zap . Any ( "schema" , request . Schema ) )
2021-03-08 10:09:48 +08:00
lct . result , err = lct . queryService . LoadCollection ( ctx , request )
2021-03-13 11:59:24 +08:00
if err != nil {
return fmt . Errorf ( "call query service LoadCollection: %s" , err )
}
return nil
2021-02-04 15:31:02 +08:00
}
2021-02-23 09:58:06 +08:00
func ( lct * LoadCollectionTask ) PostExecute ( ctx context . Context ) error {
2021-03-13 11:59:24 +08:00
log . Debug ( "LoadCollectionTask PostExecute" , zap . String ( "role" , Params . RoleName ) , zap . Int64 ( "msgID" , lct . Base . MsgID ) )
2021-02-04 15:31:02 +08:00
return nil
}
type ReleaseCollectionTask struct {
Condition
* milvuspb . ReleaseCollectionRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
queryService types . QueryService
result * commonpb . Status
2021-02-04 15:31:02 +08:00
}
2021-03-25 14:41:46 +08:00
func ( rct * ReleaseCollectionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return rct . ctx
2021-02-04 15:31:02 +08:00
}
func ( rct * ReleaseCollectionTask ) ID ( ) UniqueID {
return rct . Base . MsgID
}
func ( rct * ReleaseCollectionTask ) SetID ( uid UniqueID ) {
rct . Base . MsgID = uid
}
2021-02-23 09:58:06 +08:00
func ( rct * ReleaseCollectionTask ) Name ( ) string {
return ReleaseCollectionTaskName
}
2021-02-04 15:31:02 +08:00
func ( rct * ReleaseCollectionTask ) Type ( ) commonpb . MsgType {
return rct . Base . MsgType
}
func ( rct * ReleaseCollectionTask ) BeginTs ( ) Timestamp {
return rct . Base . Timestamp
}
func ( rct * ReleaseCollectionTask ) EndTs ( ) Timestamp {
return rct . Base . Timestamp
}
func ( rct * ReleaseCollectionTask ) SetTs ( ts Timestamp ) {
rct . Base . Timestamp = ts
}
2021-02-23 09:58:06 +08:00
func ( rct * ReleaseCollectionTask ) OnEnqueue ( ) error {
rct . Base = & commonpb . MsgBase { }
return nil
}
func ( rct * ReleaseCollectionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
rct . Base . MsgType = commonpb . MsgType_ReleaseCollection
2021-02-04 15:31:02 +08:00
rct . Base . SourceID = Params . ProxyID
collName := rct . CollectionName
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( rct * ReleaseCollectionTask ) Execute ( ctx context . Context ) ( err error ) {
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , rct . CollectionName )
2021-02-04 15:31:02 +08:00
if err != nil {
return err
}
request := & querypb . ReleaseCollectionRequest {
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_ReleaseCollection ,
2021-02-04 15:31:02 +08:00
MsgID : rct . Base . MsgID ,
Timestamp : rct . Base . Timestamp ,
SourceID : rct . Base . SourceID ,
} ,
DbID : 0 ,
CollectionID : collID ,
}
2021-03-08 10:09:48 +08:00
rct . result , err = rct . queryService . ReleaseCollection ( ctx , request )
2021-02-04 15:31:02 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( rct * ReleaseCollectionTask ) PostExecute ( ctx context . Context ) error {
2021-02-04 15:31:02 +08:00
return nil
}
type LoadPartitionTask struct {
Condition
2021-03-12 14:22:09 +08:00
* milvuspb . LoadPartitionsRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
queryService types . QueryService
result * commonpb . Status
2021-02-04 15:31:02 +08:00
}
2021-03-25 14:41:46 +08:00
func ( lpt * LoadPartitionTask ) TraceCtx ( ) context . Context {
return lpt . ctx
}
2021-02-04 15:31:02 +08:00
func ( lpt * LoadPartitionTask ) ID ( ) UniqueID {
return lpt . Base . MsgID
}
func ( lpt * LoadPartitionTask ) SetID ( uid UniqueID ) {
lpt . Base . MsgID = uid
}
2021-02-23 09:58:06 +08:00
func ( lpt * LoadPartitionTask ) Name ( ) string {
return LoadPartitionTaskName
}
2021-02-04 15:31:02 +08:00
func ( lpt * LoadPartitionTask ) Type ( ) commonpb . MsgType {
return lpt . Base . MsgType
}
func ( lpt * LoadPartitionTask ) BeginTs ( ) Timestamp {
return lpt . Base . Timestamp
}
func ( lpt * LoadPartitionTask ) EndTs ( ) Timestamp {
return lpt . Base . Timestamp
}
func ( lpt * LoadPartitionTask ) SetTs ( ts Timestamp ) {
lpt . Base . Timestamp = ts
}
2021-02-23 09:58:06 +08:00
func ( lpt * LoadPartitionTask ) OnEnqueue ( ) error {
lpt . Base = & commonpb . MsgBase { }
return nil
}
func ( lpt * LoadPartitionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
lpt . Base . MsgType = commonpb . MsgType_LoadPartitions
2021-02-04 15:31:02 +08:00
lpt . Base . SourceID = Params . ProxyID
collName := lpt . CollectionName
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( lpt * LoadPartitionTask ) Execute ( ctx context . Context ) error {
2021-02-04 15:31:02 +08:00
var partitionIDs [ ] int64
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , lpt . CollectionName )
2021-02-04 15:31:02 +08:00
if err != nil {
return err
}
2021-02-26 17:44:24 +08:00
collSchema , err := globalMetaCache . GetCollectionSchema ( ctx , lpt . CollectionName )
2021-02-06 21:17:18 +08:00
if err != nil {
return err
}
2021-02-04 15:31:02 +08:00
for _ , partitionName := range lpt . PartitionNames {
2021-02-26 17:44:24 +08:00
partitionID , err := globalMetaCache . GetPartitionID ( ctx , lpt . CollectionName , partitionName )
2021-02-04 15:31:02 +08:00
if err != nil {
return err
}
partitionIDs = append ( partitionIDs , partitionID )
}
2021-03-12 14:22:09 +08:00
request := & querypb . LoadPartitionsRequest {
2021-02-04 15:31:02 +08:00
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_LoadPartitions ,
2021-02-04 15:31:02 +08:00
MsgID : lpt . Base . MsgID ,
Timestamp : lpt . Base . Timestamp ,
SourceID : lpt . Base . SourceID ,
} ,
DbID : 0 ,
CollectionID : collID ,
PartitionIDs : partitionIDs ,
2021-02-06 21:17:18 +08:00
Schema : collSchema ,
2021-02-04 15:31:02 +08:00
}
2021-03-08 10:09:48 +08:00
lpt . result , err = lpt . queryService . LoadPartitions ( ctx , request )
2021-02-04 15:31:02 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( lpt * LoadPartitionTask ) PostExecute ( ctx context . Context ) error {
2021-02-04 15:31:02 +08:00
return nil
}
type ReleasePartitionTask struct {
Condition
2021-03-12 14:22:09 +08:00
* milvuspb . ReleasePartitionsRequest
2021-03-08 10:09:48 +08:00
ctx context . Context
queryService types . QueryService
result * commonpb . Status
2021-02-04 15:31:02 +08:00
}
2021-03-25 14:41:46 +08:00
func ( rpt * ReleasePartitionTask ) TraceCtx ( ) context . Context {
2021-02-23 09:58:06 +08:00
return rpt . ctx
2021-02-04 15:31:02 +08:00
}
func ( rpt * ReleasePartitionTask ) ID ( ) UniqueID {
return rpt . Base . MsgID
}
func ( rpt * ReleasePartitionTask ) SetID ( uid UniqueID ) {
rpt . Base . MsgID = uid
}
func ( rpt * ReleasePartitionTask ) Type ( ) commonpb . MsgType {
return rpt . Base . MsgType
}
2021-02-23 09:58:06 +08:00
func ( rpt * ReleasePartitionTask ) Name ( ) string {
return ReleasePartitionTaskName
}
2021-02-04 15:31:02 +08:00
func ( rpt * ReleasePartitionTask ) BeginTs ( ) Timestamp {
return rpt . Base . Timestamp
}
func ( rpt * ReleasePartitionTask ) EndTs ( ) Timestamp {
return rpt . Base . Timestamp
}
func ( rpt * ReleasePartitionTask ) SetTs ( ts Timestamp ) {
rpt . Base . Timestamp = ts
}
2021-02-23 09:58:06 +08:00
func ( rpt * ReleasePartitionTask ) OnEnqueue ( ) error {
rpt . Base = & commonpb . MsgBase { }
return nil
}
func ( rpt * ReleasePartitionTask ) PreExecute ( ctx context . Context ) error {
2021-03-10 14:45:35 +08:00
rpt . Base . MsgType = commonpb . MsgType_ReleasePartitions
2021-02-04 15:31:02 +08:00
rpt . Base . SourceID = Params . ProxyID
collName := rpt . CollectionName
if err := ValidateCollectionName ( collName ) ; err != nil {
return err
}
return nil
}
2021-02-23 09:58:06 +08:00
func ( rpt * ReleasePartitionTask ) Execute ( ctx context . Context ) ( err error ) {
2021-02-04 15:31:02 +08:00
var partitionIDs [ ] int64
2021-02-26 17:44:24 +08:00
collID , err := globalMetaCache . GetCollectionID ( ctx , rpt . CollectionName )
2021-02-04 15:31:02 +08:00
if err != nil {
return err
}
for _ , partitionName := range rpt . PartitionNames {
2021-02-26 17:44:24 +08:00
partitionID , err := globalMetaCache . GetPartitionID ( ctx , rpt . CollectionName , partitionName )
2021-02-04 15:31:02 +08:00
if err != nil {
return err
}
partitionIDs = append ( partitionIDs , partitionID )
}
2021-03-12 14:22:09 +08:00
request := & querypb . ReleasePartitionsRequest {
2021-02-04 15:31:02 +08:00
Base : & commonpb . MsgBase {
2021-03-10 14:45:35 +08:00
MsgType : commonpb . MsgType_ReleasePartitions ,
2021-02-04 15:31:02 +08:00
MsgID : rpt . Base . MsgID ,
Timestamp : rpt . Base . Timestamp ,
SourceID : rpt . Base . SourceID ,
} ,
DbID : 0 ,
CollectionID : collID ,
PartitionIDs : partitionIDs ,
}
2021-03-08 10:09:48 +08:00
rpt . result , err = rpt . queryService . ReleasePartitions ( ctx , request )
2021-02-04 15:31:02 +08:00
return err
}
2021-02-23 09:58:06 +08:00
func ( rpt * ReleasePartitionTask ) PostExecute ( ctx context . Context ) error {
2021-02-04 15:31:02 +08:00
return nil
}