Fill MsgBase ALL (#19912)

Signed-off-by: lixinguo <xinguo.li@zilliz.com>

Signed-off-by: lixinguo <xinguo.li@zilliz.com>
Co-authored-by: lixinguo <xinguo.li@zilliz.com>
pull/19969/head
smellthemoon 2022-10-21 15:57:28 +08:00 committed by GitHub
parent 8d09bb8d8e
commit 581e1d8c35
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
38 changed files with 402 additions and 361 deletions

View File

@ -23,6 +23,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
@ -93,12 +94,12 @@ func (ia *IDAllocator) syncID() (bool, error) {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
req := &rootcoordpb.AllocIDRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_RequestID,
MsgID: 0,
Timestamp: 0,
SourceID: ia.PeerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestID),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(ia.PeerID),
),
Count: need,
}
resp, err := ia.remoteAllocator.AllocID(ctx, req)

View File

@ -22,6 +22,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
)
// allocator is the interface that allocating `UniqueID` or `Timestamp`
@ -49,12 +50,12 @@ func newRootCoordAllocator(rootCoordClient types.RootCoord) allocator {
// invoking RootCoord `AllocTimestamp`
func (alloc *rootCoordAllocator) allocTimestamp(ctx context.Context) (Timestamp, error) {
resp, err := alloc.AllocTimestamp(ctx, &rootcoordpb.AllocTimestampRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_RequestTSO,
MsgID: 0,
Timestamp: 0,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestTSO),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
Count: 1,
})
if err = VerifyResponse(resp, err); err != nil {
@ -66,12 +67,12 @@ func (alloc *rootCoordAllocator) allocTimestamp(ctx context.Context) (Timestamp,
// allocID allocates an `UniqueID` from RootCoord, invoking AllocID grpc
func (alloc *rootCoordAllocator) allocID(ctx context.Context) (UniqueID, error) {
resp, err := alloc.AllocID(ctx, &rootcoordpb.AllocIDRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_RequestID,
MsgID: 0,
Timestamp: 0,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestID),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
Count: 1,
})

View File

@ -23,6 +23,7 @@ import (
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"go.uber.org/zap"
)
@ -123,10 +124,10 @@ func (c *Cluster) Flush(ctx context.Context, segments []*datapb.SegmentInfo, mar
continue
}
req := &datapb.FlushSegmentsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Flush,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_Flush),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
CollectionID: collectionID,
SegmentIDs: segments,
MarkSegmentIDs: marks,

View File

@ -43,6 +43,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/logutil"
@ -881,10 +882,10 @@ func (s *Server) stopServerLoop() {
// collection information will be added to server meta info.
func (s *Server) loadCollectionFromRootCoord(ctx context.Context, collectionID int64) error {
resp, err := s.rootCoordClient.DescribeCollection(ctx, &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DescribeCollection),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
DbName: "",
CollectionID: collectionID,
})
@ -892,12 +893,12 @@ func (s *Server) loadCollectionFromRootCoord(ctx context.Context, collectionID i
return err
}
presp, err := s.rootCoordClient.ShowPartitions(ctx, &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowPartitions,
MsgID: 0,
Timestamp: 0,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_ShowPartitions),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
DbName: "",
CollectionName: resp.Schema.Name,
CollectionID: resp.CollectionID,

View File

@ -23,6 +23,7 @@ import (
"strconv"
"sync"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/errorutil"
"golang.org/x/sync/errgroup"
@ -594,10 +595,10 @@ func (s *Server) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInf
}
dresp, err := s.rootCoordClient.DescribeCollection(s.ctx, &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DescribeCollection),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
CollectionID: collectionID,
})
if err = VerifyResponse(dresp, err); err != nil {
@ -786,7 +787,7 @@ func (s *Server) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegment
return resp, nil
}
//ShowConfigurations returns the configurations of DataCoord matching req.Pattern
// ShowConfigurations returns the configurations of DataCoord matching req.Pattern
func (s *Server) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
log.Debug("DataCoord.ShowConfigurations", zap.String("pattern", req.Pattern))
if s.isClosed() {
@ -1275,10 +1276,10 @@ func (s *Server) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSe
}
resp, err := cli.AddImportSegment(ctx,
&datapb.AddImportSegmentRequest{
Base: &commonpb.MsgBase{
SourceID: Params.DataNodeCfg.GetNodeID(),
Timestamp: req.GetBase().GetTimestamp(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithTimeStamp(req.GetBase().GetTimestamp()),
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
SegmentId: req.GetSegmentId(),
ChannelName: req.GetChannelName(),
CollectionId: req.GetCollectionId(),

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"go.uber.org/zap"
)
@ -218,10 +219,10 @@ func (c *SessionManager) execReCollectSegmentStats(ctx context.Context, nodeID i
ctx, cancel := context.WithTimeout(ctx, reCollectTimeout)
defer cancel()
resp, err := cli.ResendSegmentStats(ctx, &datapb.ResendSegmentStatsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ResendSegmentStats,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_ResendSegmentStats),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
})
if err := VerifyResponse(resp, err); err != nil {
log.Error("re-collect segment stats call failed",
@ -253,10 +254,10 @@ func (c *SessionManager) GetCompactionState() map[int64]*datapb.CompactionStateR
ctx, cancel := context.WithTimeout(ctx, rpcCompactionTimeout)
defer cancel()
resp, err := cli.GetCompactionState(ctx, &datapb.CompactionStateRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_GetSystemConfigs,
SourceID: Params.DataCoordCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_GetSystemConfigs),
commonpbutil.WithSourceID(Params.DataCoordCfg.GetNodeID()),
),
})
if err != nil {
log.Info("Get State failed", zap.Error(err))

View File

@ -20,6 +20,7 @@ import (
"context"
"errors"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
@ -50,12 +51,12 @@ func newAllocator(s types.RootCoord) *allocator {
func (alloc *allocator) allocID() (UniqueID, error) {
ctx := context.TODO()
resp, err := alloc.rootCoord.AllocID(ctx, &rootcoordpb.AllocIDRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_RequestID,
MsgID: 1, // GOOSE TODO
Timestamp: 0, // GOOSE TODO
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestID),
commonpbutil.WithMsgID(1), // GOOSE TODO
commonpbutil.WithTimeStamp(0), // GOOSE TODO
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
Count: 1,
})
if err != nil {
@ -73,10 +74,10 @@ func (alloc *allocator) allocID() (UniqueID, error) {
func (alloc *allocator) allocIDBatch(count uint32) (UniqueID, uint32, error) {
ctx := context.Background()
resp, err := alloc.rootCoord.AllocID(ctx, &rootcoordpb.AllocIDRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_RequestID,
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestID),
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
Count: count,
})

View File

@ -50,6 +50,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/importutil"
"github.com/milvus-io/milvus/internal/util/logutil"
@ -476,12 +477,12 @@ func (node *DataNode) Start() error {
log.Debug("start id allocator done", zap.String("role", typeutil.DataNodeRole))
rep, err := node.rootCoord.AllocTimestamp(node.ctx, &rootcoordpb.AllocTimestampRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_RequestTSO,
MsgID: 0,
Timestamp: 0,
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestTSO),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
Count: 1,
})
if err != nil || rep.Status.ErrorCode != commonpb.ErrorCode_Success {
@ -998,12 +999,12 @@ func (node *DataNode) Import(ctx context.Context, req *datapb.ImportTaskRequest)
// get a timestamp for all the rows
// Ignore cancellation from parent context.
rep, err := node.rootCoord.AllocTimestamp(newCtx, &rootcoordpb.AllocTimestampRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_RequestTSO,
MsgID: 0,
Timestamp: 0,
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_RequestTSO),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
Count: 1,
})
@ -1232,23 +1233,22 @@ func importFlushReqFunc(node *DataNode, req *datapb.ImportTaskRequest, res *root
err = retry.Do(context.Background(), func() error {
// Ask DataCoord to save binlog path and add segment to the corresponding DataNode flow graph.
resp, err := node.dataCoord.SaveImportSegment(context.Background(), &datapb.SaveImportSegmentRequest{
Base: &commonpb.MsgBase{
SourceID: Params.DataNodeCfg.GetNodeID(),
// Pass current timestamp downstream.
Timestamp: ts,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithTimeStamp(ts), // Pass current timestamp downstream.
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
SegmentId: segmentID,
ChannelName: targetChName,
CollectionId: req.GetImportTask().GetCollectionId(),
PartitionId: req.GetImportTask().GetPartitionId(),
RowNum: int64(rowNum),
SaveBinlogPathReq: &datapb.SaveBinlogPathsRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: ts,
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
SegmentID: segmentID,
CollectionID: req.GetImportTask().GetCollectionId(),
Field2BinlogPaths: fieldInsert,

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/concurrency"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/util/funcutil"
@ -345,12 +346,12 @@ func (dsService *dataSyncService) initNodes(vchanInfo *datapb.VchannelInfo) erro
// getSegmentInfos return the SegmentInfo details according to the given ids through RPC to datacoord
func (dsService *dataSyncService) getSegmentInfos(segmentIDs []int64) ([]*datapb.SegmentInfo, error) {
infoResp, err := dsService.dataCoord.GetSegmentInfo(dsService.ctx, &datapb.GetSegmentInfoRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_SegmentInfo,
MsgID: 0,
Timestamp: 0,
SourceID: Params.ProxyCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_SegmentInfo),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(Params.ProxyCfg.GetNodeID()),
),
SegmentIDs: segmentIDs,
IncludeUnHealthy: true,
})

View File

@ -34,6 +34,7 @@ import (
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
@ -48,16 +49,19 @@ var _ flowgraph.Node = (*ddNode)(nil)
// ddNode filters messages from message streams.
//
// ddNode recives all the messages from message stream dml channels, including insert messages,
// delete messages and ddl messages like CreateCollectionMsg and DropCollectionMsg.
//
// delete messages and ddl messages like CreateCollectionMsg and DropCollectionMsg.
//
// ddNode filters insert messages according to the `sealedSegment`.
// ddNode will filter out the insert message for those who belong to `sealedSegment`
//
// When receiving a `DropCollection` message, ddNode will send a signal to DataNode `BackgroundGC`
// goroutinue, telling DataNode to release the resources of this particular flow graph.
//
// goroutinue, telling DataNode to release the resources of this particular flow graph.
//
// After the filtering process, ddNode passes all the valid insert messages and delete message
// to the following flow graph node, which in DataNode is `insertBufferNode`
//
// to the following flow graph node, which in DataNode is `insertBufferNode`
type ddNode struct {
BaseNode
@ -281,12 +285,12 @@ func (ddn *ddNode) sendDeltaTimeTick(ts Timestamp) error {
HashValues: []uint32{0},
}
timeTickResult := internalpb.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick,
MsgID: 0,
Timestamp: ts,
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_TimeTick),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
}
timeTickMsg := &msgstream.TimeTickMsg{
BaseMsg: baseMsg,

View File

@ -37,6 +37,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/trace"
@ -405,8 +406,9 @@ func (ibNode *insertBufferNode) Flush(fgMsg *flowGraphMsg, seg2Upload []UniqueID
}
// updateSegmentStates updates statistics in channel meta for the segments in insertMsgs.
// If the segment doesn't exist, a new segment will be created.
// The segment number of rows will be updated in mem, waiting to be uploaded to DataCoord.
//
// If the segment doesn't exist, a new segment will be created.
// The segment number of rows will be updated in mem, waiting to be uploaded to DataCoord.
func (ibNode *insertBufferNode) updateSegmentStates(insertMsgs []*msgstream.InsertMsg, startPos, endPos *internalpb.MsgPosition) (seg2Upload []UniqueID, err error) {
uniqueSeg := make(map[UniqueID]int64)
for _, msg := range insertMsgs {
@ -604,11 +606,11 @@ func newInsertBufferNode(ctx context.Context, collID UniqueID, flushCh <-chan fl
HashValues: []uint32{0},
},
DataNodeTtMsg: datapb.DataNodeTtMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DataNodeTt,
MsgID: 0,
Timestamp: ts,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DataNodeTt),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(ts),
),
ChannelName: config.vChannelName,
Timestamp: ts,
SegmentsStats: stats,

View File

@ -35,6 +35,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/timerecord"
@ -637,12 +638,12 @@ func getFieldBinlogs(fieldID UniqueID, binlogs []*datapb.FieldBinlog) *datapb.Fi
func dropVirtualChannelFunc(dsService *dataSyncService, opts ...retry.Option) flushAndDropFunc {
return func(packs []*segmentFlushPack) {
req := &datapb.DropVirtualChannelRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO msg type
MsgID: 0, //TODO msg id
Timestamp: 0, //TODO time stamp
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0), //TODO msg type
commonpbutil.WithMsgID(0), //TODO msg id
commonpbutil.WithTimeStamp(0), //TODO time stamp
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
ChannelName: dsService.vchannelName,
}
@ -790,12 +791,12 @@ func flushNotifyFunc(dsService *dataSyncService, opts ...retry.Option) notifyMet
)
req := &datapb.SaveBinlogPathsRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO msg type
MsgID: 0, //TODO msg id
Timestamp: 0, //TODO time stamp
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(0),
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
SegmentID: pack.segmentID,
CollectionID: dsService.collectionID,
Field2BinlogPaths: fieldInsert,

View File

@ -22,6 +22,7 @@ import (
"reflect"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
@ -58,12 +59,12 @@ func (mService *metaService) getCollectionSchema(ctx context.Context, collID Uni
// getCollectionInfo get collection info with provided collection id at specified timestamp.
func (mService *metaService) getCollectionInfo(ctx context.Context, collID UniqueID, timestamp Timestamp) (*milvuspb.DescribeCollectionResponse, error) {
req := &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
MsgID: 0, //GOOSE TODO
Timestamp: 0, // GOOSE TODO
SourceID: Params.DataNodeCfg.GetNodeID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DescribeCollection),
commonpbutil.WithMsgID(0), //GOOSE TODO
commonpbutil.WithTimeStamp(0), //GOOSE TODO
commonpbutil.WithSourceID(Params.DataNodeCfg.GetNodeID()),
),
DbName: "default", // GOOSE TODO
CollectionID: collID,
TimeStamp: timestamp,

View File

@ -46,6 +46,7 @@ var _ types.DataCoord = (*Client)(nil)
type Client struct {
grpcClient grpcclient.GrpcClient[datapb.DataCoordClient]
sess *sessionutil.Session
sourceID int64
}
// NewClient creates a new client instance
@ -163,7 +164,7 @@ func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResp
// Flush flushes a collection's data
func (c *Client) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -215,7 +216,7 @@ func (c *Client) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentI
// error is returned only when some communication issue occurs
func (c *Client) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -240,7 +241,7 @@ func (c *Client) GetSegmentStates(ctx context.Context, req *datapb.GetSegmentSta
// error is returned only when some communication issue occurs
func (c *Client) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -265,7 +266,7 @@ func (c *Client) GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsert
// error is returned only when some communication issue occurs
func (c *Client) GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -290,7 +291,7 @@ func (c *Client) GetCollectionStatistics(ctx context.Context, req *datapb.GetCol
// error is returned only when some communication issue occurs
func (c *Client) GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -327,7 +328,7 @@ func (c *Client) GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringRes
// error is returned only when some communication issue occurs
func (c *Client) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -357,7 +358,7 @@ func (c *Client) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoR
func (c *Client) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
// use Call here on purpose
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.Call(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -379,7 +380,7 @@ func (c *Client) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
// error is returned only when some communication issue occurs
func (c *Client) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -403,7 +404,7 @@ func (c *Client) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInf
// error is returned only when some communication issue occurs
func (c *Client) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -426,7 +427,7 @@ func (c *Client) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedS
// error is returned only when some communication issue occurs
func (c *Client) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegmentsByStatesRequest) (*datapb.GetSegmentsByStatesResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -442,7 +443,7 @@ func (c *Client) GetSegmentsByStates(ctx context.Context, req *datapb.GetSegment
// ShowConfigurations gets specified configurations para of DataCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -459,7 +460,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
// GetMetrics gets all metrics of datacoord
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -545,7 +546,7 @@ func (c *Client) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateR
// DropVirtualChannel drops virtual channel in datacoord.
func (c *Client) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -561,7 +562,7 @@ func (c *Client) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtual
// SetSegmentState sets the state of a given segment.
func (c *Client) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest) (*datapb.SetSegmentStateResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -577,7 +578,7 @@ func (c *Client) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStat
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -593,7 +594,7 @@ func (c *Client) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*da
// UpdateSegmentStatistics is the client side caller of UpdateSegmentStatistics.
func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -609,7 +610,7 @@ func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update
// AcquireSegmentLock acquire the reference lock of the segments.
func (c *Client) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -625,7 +626,7 @@ func (c *Client) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegm
// ReleaseSegmentLock release the reference lock of the segments.
func (c *Client) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -641,7 +642,7 @@ func (c *Client) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegm
// SaveImportSegment is the DataCoord client side code for SaveImportSegment call.
func (c *Client) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSegmentRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -656,7 +657,7 @@ func (c *Client) SaveImportSegment(ctx context.Context, req *datapb.SaveImportSe
func (c *Client) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsImportingStateRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -671,7 +672,7 @@ func (c *Client) UnsetIsImportingState(ctx context.Context, req *datapb.UnsetIsI
func (c *Client) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -687,7 +688,7 @@ func (c *Client) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmen
// BroadcastAlteredCollection is the DataCoord client side code for BroadcastAlteredCollection call.
func (c *Client) BroadcastAlteredCollection(ctx context.Context, req *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client datapb.DataCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()

View File

@ -246,7 +246,7 @@ func (c *Client) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (
// ShowConfigurations gets specified configurations para of IndexCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -263,7 +263,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
// GetMetrics gets the metrics info of IndexCoord.
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()

View File

@ -126,7 +126,7 @@ func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResp
// InvalidateCollectionMetaCache invalidate collection meta cache
func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, req *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.grpcClient.GetNodeID())
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client proxypb.ProxyClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -141,7 +141,7 @@ func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, req *proxypb
func (c *Client) InvalidateCredentialCache(ctx context.Context, req *proxypb.InvalidateCredCacheRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.grpcClient.GetNodeID())
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client proxypb.ProxyClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -156,7 +156,7 @@ func (c *Client) InvalidateCredentialCache(ctx context.Context, req *proxypb.Inv
func (c *Client) UpdateCredentialCache(ctx context.Context, req *proxypb.UpdateCredCacheRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.grpcClient.GetNodeID())
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client proxypb.ProxyClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -171,7 +171,7 @@ func (c *Client) UpdateCredentialCache(ctx context.Context, req *proxypb.UpdateC
func (c *Client) RefreshPolicyInfoCache(ctx context.Context, req *proxypb.RefreshPolicyInfoCacheRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.grpcClient.GetNodeID())
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client proxypb.ProxyClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -188,7 +188,7 @@ func (c *Client) RefreshPolicyInfoCache(ctx context.Context, req *proxypb.Refres
// because it only obtains the metrics of Proxy, not including the topological metrics of Query cluster and Data cluster.
func (c *Client) GetProxyMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.grpcClient.GetNodeID())
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client proxypb.ProxyClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -204,7 +204,7 @@ func (c *Client) GetProxyMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
// SetRates notifies Proxy to limit rates of requests.
func (c *Client) SetRates(ctx context.Context, req *proxypb.SetRatesRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.grpcClient.GetNodeID())
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client proxypb.ProxyClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()

View File

@ -159,7 +159,7 @@ func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResp
// ShowCollections shows the collections in the QueryCoord.
func (c *Client) ShowCollections(ctx context.Context, req *querypb.ShowCollectionsRequest) (*querypb.ShowCollectionsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -175,7 +175,7 @@ func (c *Client) ShowCollections(ctx context.Context, req *querypb.ShowCollectio
// LoadCollection loads the data of the specified collections in the QueryCoord.
func (c *Client) LoadCollection(ctx context.Context, req *querypb.LoadCollectionRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -191,7 +191,7 @@ func (c *Client) LoadCollection(ctx context.Context, req *querypb.LoadCollection
// ReleaseCollection release the data of the specified collections in the QueryCoord.
func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -207,7 +207,7 @@ func (c *Client) ReleaseCollection(ctx context.Context, req *querypb.ReleaseColl
// ShowPartitions shows the partitions in the QueryCoord.
func (c *Client) ShowPartitions(ctx context.Context, req *querypb.ShowPartitionsRequest) (*querypb.ShowPartitionsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -223,7 +223,7 @@ func (c *Client) ShowPartitions(ctx context.Context, req *querypb.ShowPartitions
// LoadPartitions loads the data of the specified partitions in the QueryCoord.
func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -239,7 +239,7 @@ func (c *Client) LoadPartitions(ctx context.Context, req *querypb.LoadPartitions
// ReleasePartitions release the data of the specified partitions in the QueryCoord.
func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -255,7 +255,7 @@ func (c *Client) ReleasePartitions(ctx context.Context, req *querypb.ReleasePart
// GetPartitionStates gets the states of the specified partition.
func (c *Client) GetPartitionStates(ctx context.Context, req *querypb.GetPartitionStatesRequest) (*querypb.GetPartitionStatesResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -271,7 +271,7 @@ func (c *Client) GetPartitionStates(ctx context.Context, req *querypb.GetPartiti
// GetSegmentInfo gets the information of the specified segment from QueryCoord.
func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfoRequest) (*querypb.GetSegmentInfoResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -287,7 +287,7 @@ func (c *Client) GetSegmentInfo(ctx context.Context, req *querypb.GetSegmentInfo
// LoadBalance migrate the sealed segments on the source node to the dst nodes.
func (c *Client) LoadBalance(ctx context.Context, req *querypb.LoadBalanceRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -303,7 +303,7 @@ func (c *Client) LoadBalance(ctx context.Context, req *querypb.LoadBalanceReques
// ShowConfigurations gets specified configurations para of QueryCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -320,7 +320,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
// GetMetrics gets the metrics information of QueryCoord.
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -336,7 +336,7 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
// GetReplicas gets the replicas of a certain collection.
func (c *Client) GetReplicas(ctx context.Context, req *milvuspb.GetReplicasRequest) (*milvuspb.GetReplicasResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -352,7 +352,7 @@ func (c *Client) GetReplicas(ctx context.Context, req *milvuspb.GetReplicasReque
// GetShardLeaders gets the shard leaders of a certain collection.
func (c *Client) GetShardLeaders(ctx context.Context, req *querypb.GetShardLeadersRequest) (*querypb.GetShardLeadersResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.sess.ServerID))
ret, err := c.grpcClient.ReCall(ctx, func(client querypb.QueryCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()

View File

@ -165,7 +165,7 @@ func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResp
// CreateCollection create collection
func (c *Client) CreateCollection(ctx context.Context, in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -181,7 +181,7 @@ func (c *Client) CreateCollection(ctx context.Context, in *milvuspb.CreateCollec
// DropCollection drop collection
func (c *Client) DropCollection(ctx context.Context, in *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -197,7 +197,7 @@ func (c *Client) DropCollection(ctx context.Context, in *milvuspb.DropCollection
// HasCollection check collection existence
func (c *Client) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -213,7 +213,7 @@ func (c *Client) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRe
// DescribeCollection return collection info
func (c *Client) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -229,7 +229,7 @@ func (c *Client) DescribeCollection(ctx context.Context, in *milvuspb.DescribeCo
// ShowCollections list all collection names
func (c *Client) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -244,7 +244,7 @@ func (c *Client) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectio
func (c *Client) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) {
request = typeutil.Clone(request)
commonpbutil.SetTargetID(request.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(request.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -260,7 +260,7 @@ func (c *Client) AlterCollection(ctx context.Context, request *milvuspb.AlterCol
// CreatePartition create partition
func (c *Client) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -276,7 +276,7 @@ func (c *Client) CreatePartition(ctx context.Context, in *milvuspb.CreatePartiti
// DropPartition drop partition
func (c *Client) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -292,7 +292,7 @@ func (c *Client) DropPartition(ctx context.Context, in *milvuspb.DropPartitionRe
// HasPartition check partition existence
func (c *Client) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -308,7 +308,7 @@ func (c *Client) HasPartition(ctx context.Context, in *milvuspb.HasPartitionRequ
// ShowPartitions list all partitions in collection
func (c *Client) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -324,7 +324,7 @@ func (c *Client) ShowPartitions(ctx context.Context, in *milvuspb.ShowPartitions
// AllocTimestamp global timestamp allocator
func (c *Client) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -340,7 +340,7 @@ func (c *Client) AllocTimestamp(ctx context.Context, in *rootcoordpb.AllocTimest
// AllocID global ID allocator
func (c *Client) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -356,7 +356,7 @@ func (c *Client) AllocID(ctx context.Context, in *rootcoordpb.AllocIDRequest) (*
// UpdateChannelTimeTick used to handle ChannelTimeTickMsg
func (c *Client) UpdateChannelTimeTick(ctx context.Context, in *internalpb.ChannelTimeTickMsg) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -372,7 +372,7 @@ func (c *Client) UpdateChannelTimeTick(ctx context.Context, in *internalpb.Chann
// ShowSegments list all segments
func (c *Client) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequest) (*milvuspb.ShowSegmentsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -388,7 +388,7 @@ func (c *Client) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentsRequ
// InvalidateCollectionMetaCache notifies RootCoord to release the collection cache in Proxies.
func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -404,7 +404,7 @@ func (c *Client) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.
// ShowConfigurations gets specified configurations para of RootCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -421,7 +421,7 @@ func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowCon
// GetMetrics get metrics
func (c *Client) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
in = typeutil.Clone(in)
commonpbutil.SetTargetID(in.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(in.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -437,7 +437,7 @@ func (c *Client) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest)
// CreateAlias create collection alias
func (c *Client) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -453,7 +453,7 @@ func (c *Client) CreateAlias(ctx context.Context, req *milvuspb.CreateAliasReque
// DropAlias drop collection alias
func (c *Client) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -469,7 +469,7 @@ func (c *Client) DropAlias(ctx context.Context, req *milvuspb.DropAliasRequest)
// AlterAlias alter collection alias
func (c *Client) AlterAlias(ctx context.Context, req *milvuspb.AlterAliasRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -553,7 +553,7 @@ func (c *Client) CreateCredential(ctx context.Context, req *internalpb.Credentia
func (c *Client) GetCredential(ctx context.Context, req *rootcoordpb.GetCredentialRequest) (*rootcoordpb.GetCredentialResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -581,7 +581,7 @@ func (c *Client) UpdateCredential(ctx context.Context, req *internalpb.Credentia
func (c *Client) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCredentialRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -596,7 +596,7 @@ func (c *Client) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCrede
func (c *Client) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -611,7 +611,7 @@ func (c *Client) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersR
func (c *Client) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -626,7 +626,7 @@ func (c *Client) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest
func (c *Client) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -641,7 +641,7 @@ func (c *Client) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest) (*
func (c *Client) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserRoleRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -656,7 +656,7 @@ func (c *Client) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserR
func (c *Client) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest) (*milvuspb.SelectRoleResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -671,7 +671,7 @@ func (c *Client) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest
func (c *Client) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest) (*milvuspb.SelectUserResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -686,7 +686,7 @@ func (c *Client) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest
func (c *Client) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePrivilegeRequest) (*commonpb.Status, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -701,7 +701,7 @@ func (c *Client) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePriv
func (c *Client) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
@ -716,7 +716,7 @@ func (c *Client) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantReque
func (c *Client) ListPolicy(ctx context.Context, req *internalpb.ListPolicyRequest) (*internalpb.ListPolicyResponse, error) {
req = typeutil.Clone(req)
commonpbutil.SetTargetID(req.GetBase(), c.sess.ServerID)
commonpbutil.UpdateMsgBase(req.GetBase(), commonpbutil.FillMsgBaseFromClient(c.grpcClient.GetNodeID()))
ret, err := c.grpcClient.ReCall(ctx, func(client rootcoordpb.RootCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
)
type task interface {
@ -155,12 +156,12 @@ func (cit *CreateIndexTask) Execute(ctx context.Context) error {
// Get flushed segments
flushedSegments, err := cit.dataCoordClient.GetFlushedSegments(cit.ctx, &datapb.GetFlushedSegmentsRequest{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: cit.indexID,
Timestamp: cit.req.Timestamp,
SourceID: cit.indexCoordClient.serverID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0),
commonpbutil.WithMsgID(cit.indexID),
commonpbutil.WithTimeStamp(cit.req.Timestamp),
commonpbutil.WithSourceID(cit.indexCoordClient.serverID),
),
CollectionID: cit.req.CollectionID,
PartitionID: -1,
})

View File

@ -23,6 +23,7 @@ import (
"time"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
@ -221,12 +222,12 @@ func (it *InsertMsg) CheckAligned() error {
func (it *InsertMsg) rowBasedIndexRequest(index int) internalpb.InsertRequest {
return internalpb.InsertRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Insert,
MsgID: it.Base.MsgID,
Timestamp: it.Timestamps[index],
SourceID: it.Base.SourceID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_Insert),
commonpbutil.WithMsgID(it.Base.MsgID),
commonpbutil.WithTimeStamp(it.Timestamps[index]),
commonpbutil.WithSourceID(it.Base.SourceID),
),
DbID: it.DbID,
CollectionID: it.CollectionID,
PartitionID: it.PartitionID,
@ -246,12 +247,12 @@ func (it *InsertMsg) columnBasedIndexRequest(index int) internalpb.InsertRequest
fieldsData := make([]*schemapb.FieldData, colNum)
typeutil.AppendFieldData(fieldsData, it.GetFieldsData(), int64(index))
return internalpb.InsertRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Insert,
MsgID: it.Base.MsgID,
Timestamp: it.Timestamps[index],
SourceID: it.Base.SourceID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_Insert),
commonpbutil.WithMsgID(it.Base.MsgID),
commonpbutil.WithTimeStamp(it.Timestamps[index]),
commonpbutil.WithSourceID(it.Base.SourceID),
),
DbID: it.DbID,
CollectionID: it.CollectionID,
PartitionID: it.PartitionID,

View File

@ -1783,7 +1783,7 @@ func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPar
func (node *Proxy) getCollectionProgress(ctx context.Context, request *milvuspb.GetLoadingProgressRequest, collectionID int64) (int64, error) {
resp, err := node.queryCoord.ShowCollections(ctx, &querypb.ShowCollectionsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
request.Base,
commonpbutil.WithMsgType(commonpb.MsgType_DescribeCollection),
),
@ -1815,7 +1815,7 @@ func (node *Proxy) getPartitionProgress(ctx context.Context, request *milvuspb.G
partitionIDs = append(partitionIDs, partitionID)
}
resp, err := node.queryCoord.ShowPartitions(ctx, &querypb.ShowPartitionsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
request.Base,
commonpbutil.WithMsgType(commonpb.MsgType_ShowPartitions),
),
@ -3873,10 +3873,10 @@ func (node *Proxy) GetReplicas(ctx context.Context, req *milvuspb.GetReplicasReq
return resp, nil
}
req.Base = &commonpb.MsgBase{
MsgType: commonpb.MsgType_GetReplicas,
SourceID: Params.ProxyCfg.GetNodeID(),
}
req.Base = commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_GetReplicas),
commonpbutil.WithSourceID(Params.ProxyCfg.GetNodeID()),
)
resp, err := node.queryCoord.GetReplicas(ctx, req)
if err != nil {

View File

@ -592,7 +592,7 @@ func (sct *showCollectionsTask) Execute(ctx context.Context) error {
}
resp, err := sct.queryCoord.ShowCollections(ctx, &querypb.ShowCollectionsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
sct.Base,
commonpbutil.WithMsgType(commonpb.MsgType_ShowCollections),
),
@ -1072,7 +1072,7 @@ func (spt *showPartitionsTask) Execute(ctx context.Context) error {
IDs2Names[partitionID] = partitionName
}
resp, err := spt.queryCoord.ShowPartitions(ctx, &querypb.ShowPartitionsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
spt.Base,
commonpbutil.WithMsgType(commonpb.MsgType_ShowCollections),
),
@ -1192,7 +1192,7 @@ func (ft *flushTask) Execute(ctx context.Context) error {
return err
}
flushReq := &datapb.FlushRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
ft.Base,
commonpbutil.WithMsgType(commonpb.MsgType_Flush),
),
@ -1334,7 +1334,7 @@ func (lct *loadCollectionTask) Execute(ctx context.Context) (err error) {
return errors.New(errMsg)
}
request := &querypb.LoadCollectionRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
lct.Base,
commonpbutil.WithMsgType(commonpb.MsgType_LoadCollection),
),
@ -1428,7 +1428,7 @@ func (rct *releaseCollectionTask) Execute(ctx context.Context) (err error) {
}
rct.collectionID = collID
request := &querypb.ReleaseCollectionRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
rct.Base,
commonpbutil.WithMsgType(commonpb.MsgType_ReleaseCollection),
),
@ -1555,7 +1555,7 @@ func (lpt *loadPartitionsTask) Execute(ctx context.Context) error {
partitionIDs = append(partitionIDs, partitionID)
}
request := &querypb.LoadPartitionsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
lpt.Base,
commonpbutil.WithMsgType(commonpb.MsgType_LoadPartitions),
),
@ -1649,7 +1649,7 @@ func (rpt *releasePartitionsTask) Execute(ctx context.Context) (err error) {
partitionIDs = append(partitionIDs, partitionID)
}
request := &querypb.ReleasePartitionsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
rpt.Base,
commonpbutil.WithMsgType(commonpb.MsgType_ReleasePartitions),
),

View File

@ -232,7 +232,7 @@ func (g *getStatisticsTask) getStatisticsFromDataCoord(ctx context.Context) erro
partIDs := g.unloadedPartitionIDs
req := &datapb.GetPartitionStatisticsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
g.Base,
commonpbutil.WithMsgType(commonpb.MsgType_GetPartitionStatistics),
),
@ -644,7 +644,7 @@ func (g *getCollectionStatisticsTask) Execute(ctx context.Context) error {
}
g.collectionID = collID
req := &datapb.GetCollectionStatisticsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
g.Base,
commonpbutil.WithMsgType(commonpb.MsgType_GetCollectionStatistics),
),
@ -736,7 +736,7 @@ func (g *getPartitionStatisticsTask) Execute(ctx context.Context) error {
return err
}
req := &datapb.GetPartitionStatisticsRequest{
Base: commonpbutil.NewMsgBaseCopy(
Base: commonpbutil.UpdateMsgBase(
g.Base,
commonpbutil.WithMsgType(commonpb.MsgType_GetCollectionStatistics),
),

View File

@ -16,6 +16,7 @@ import (
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/planpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil"
@ -72,7 +73,7 @@ func (dt *deleteTask) SetTs(ts Timestamp) {
}
func (dt *deleteTask) OnEnqueue() error {
dt.DeleteRequest.Base = &commonpb.MsgBase{}
dt.DeleteRequest.Base = commonpbutil.NewMsgBase()
return nil
}
@ -266,12 +267,12 @@ func (dt *deleteTask) Execute(ctx context.Context) (err error) {
_, ok := result[key]
if !ok {
sliceRequest := internalpb.DeleteRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Delete,
MsgID: dt.Base.MsgID,
Timestamp: ts,
SourceID: proxyID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_Delete),
commonpbutil.WithMsgID(dt.Base.MsgID),
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(proxyID),
),
CollectionID: collectionID,
PartitionID: partitionID,
CollectionName: collectionName,

View File

@ -31,6 +31,7 @@ import (
. "github.com/milvus-io/milvus/internal/querycoordv2/params"
"github.com/milvus-io/milvus/internal/querycoordv2/session"
"github.com/milvus-io/milvus/internal/querycoordv2/task"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"go.uber.org/zap"
)
@ -72,7 +73,9 @@ func (dh *distHandler) start(ctx context.Context) {
dh.mu.Lock()
cctx, cancel := context.WithTimeout(ctx, distReqTimeout)
resp, err := dh.client.GetDataDistribution(cctx, dh.nodeID, &querypb.GetDataDistributionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_GetDistribution},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_GetDistribution),
),
})
cancel()
@ -195,7 +198,9 @@ func (dh *distHandler) getDistribution(ctx context.Context) {
defer dh.mu.Unlock()
cctx, cancel := context.WithTimeout(ctx, distReqTimeout)
resp, err := dh.client.GetDataDistribution(cctx, dh.nodeID, &querypb.GetDataDistributionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_GetDistribution},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_GetDistribution),
),
})
cancel()

View File

@ -33,6 +33,7 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
. "github.com/milvus-io/milvus/internal/util/typeutil"
)
@ -70,9 +71,9 @@ func (broker *CoordinatorBroker) GetCollectionSchema(ctx context.Context, collec
defer cancel()
req := &milvuspb.DescribeCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DescribeCollection,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_GetDistribution),
),
CollectionID: collectionID,
}
resp, err := broker.rootCoord.DescribeCollection(ctx, req)
@ -83,9 +84,9 @@ func (broker *CoordinatorBroker) GetPartitions(ctx context.Context, collectionID
ctx, cancel := context.WithTimeout(ctx, brokerRPCTimeout)
defer cancel()
req := &milvuspb.ShowPartitionsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ShowPartitions,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_ShowPartitions),
),
CollectionID: collectionID,
}
resp, err := broker.rootCoord.ShowPartitions(ctx, req)
@ -108,9 +109,9 @@ func (broker *CoordinatorBroker) GetRecoveryInfo(ctx context.Context, collection
defer cancel()
getRecoveryInfoRequest := &datapb.GetRecoveryInfoRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_GetRecoveryInfo,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_GetRecoveryInfo),
),
CollectionID: collectionID,
PartitionID: partitionID,
}

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
"github.com/milvus-io/milvus/internal/querycoordv2/session"
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"go.uber.org/zap"
)
@ -149,9 +150,9 @@ func (o *LeaderObserver) sync(ctx context.Context, leaderView *meta.LeaderView,
zap.String("channel", leaderView.Channel),
)
req := &querypb.SyncDistributionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_SyncDistribution,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_SyncDistribution),
),
CollectionID: leaderView.CollectionID,
Channel: leaderView.Channel,
Actions: diffs,

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
@ -78,10 +79,10 @@ func packLoadSegmentRequest(
}
return &querypb.LoadSegmentsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_LoadSegments,
MsgID: task.ID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_LoadSegments),
commonpbutil.WithMsgID(task.ID()),
),
Infos: []*querypb.SegmentLoadInfo{loadInfo},
Schema: schema,
LoadMeta: loadMeta,
@ -96,10 +97,10 @@ func packLoadSegmentRequest(
func packReleaseSegmentRequest(task *SegmentTask, action *SegmentAction) *querypb.ReleaseSegmentsRequest {
return &querypb.ReleaseSegmentsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_ReleaseSegments,
MsgID: task.ID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_ReleaseSegments),
commonpbutil.WithMsgID(task.ID()),
),
NodeID: action.Node(),
CollectionID: task.CollectionID(),
@ -126,10 +127,10 @@ func packSubDmChannelRequest(
channel *meta.DmChannel,
) *querypb.WatchDmChannelsRequest {
return &querypb.WatchDmChannelsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_WatchDmChannels,
MsgID: task.ID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_WatchDmChannels),
commonpbutil.WithMsgID(task.ID()),
),
NodeID: action.Node(),
CollectionID: task.CollectionID(),
Infos: []*datapb.VchannelInfo{channel.VchannelInfo},
@ -168,10 +169,10 @@ func fillSubDmChannelRequest(
func packUnsubDmChannelRequest(task *ChannelTask, action Action) *querypb.UnsubDmChannelRequest {
return &querypb.UnsubDmChannelRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_UnsubDmChannel,
MsgID: task.ID(),
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_UnsubDmChannel),
commonpbutil.WithMsgID(task.ID()),
),
NodeID: action.Node(),
CollectionID: task.CollectionID(),
ChannelName: task.Channel(),

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
queryPb "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/samber/lo"
@ -214,10 +215,10 @@ func (w *watchDmChannelsTask) Execute(ctx context.Context) (err error) {
}
}
req := &queryPb.LoadSegmentsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_LoadSegments,
MsgID: w.req.Base.MsgID, // use parent task's msgID
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_LoadSegments),
commonpbutil.WithMsgID(w.req.Base.MsgID), // use parent task's msgID
),
Infos: unFlushedSegments,
CollectionID: collectionID,
Schema: w.req.GetSchema(),

View File

@ -8,6 +8,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/proto/datapb"
@ -60,7 +61,7 @@ func (b *ServerBroker) ReleaseCollection(ctx context.Context, collectionID Uniqu
log.Info("releasing collection", zap.Int64("collection", collectionID))
resp, err := b.s.queryCoord.ReleaseCollection(ctx, &querypb.ReleaseCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_ReleaseCollection},
Base: commonpbutil.NewMsgBase(commonpbutil.WithMsgType(commonpb.MsgType_ReleaseCollection)),
CollectionID: collectionID,
NodeID: b.s.session.ServerID,
})
@ -78,10 +79,10 @@ func (b *ServerBroker) ReleaseCollection(ctx context.Context, collectionID Uniqu
func (b *ServerBroker) GetQuerySegmentInfo(ctx context.Context, collectionID int64, segIDs []int64) (retResp *querypb.GetSegmentInfoResponse, retErr error) {
resp, err := b.s.queryCoord.GetSegmentInfo(ctx, &querypb.GetSegmentInfoRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_GetSegmentState,
SourceID: b.s.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_GetSegmentState),
commonpbutil.WithSourceID(b.s.session.ServerID),
),
CollectionID: collectionID,
SegmentIDs: segIDs,
})
@ -169,10 +170,10 @@ func (b *ServerBroker) ReleaseSegRefLock(ctx context.Context, taskID int64, segI
func (b *ServerBroker) Flush(ctx context.Context, cID int64, segIDs []int64) error {
resp, err := b.s.dataCoord.Flush(ctx, &datapb.FlushRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Flush,
SourceID: b.s.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_Flush),
commonpbutil.WithSourceID(b.s.session.ServerID),
),
DbID: 0,
SegmentIDs: segIDs,
CollectionID: cID,

View File

@ -8,6 +8,7 @@ import (
ms "github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
@ -191,7 +192,10 @@ func (t *createCollectionTask) genCreateCollectionMsg(ctx context.Context) *ms.M
msg := &ms.CreateCollectionMsg{
BaseMsg: baseMsg,
CreateCollectionRequest: internalpb.CreateCollectionRequest{
Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection, Timestamp: ts},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_CreateCollection),
commonpbutil.WithTimeStamp(ts),
),
CollectionID: collectionID,
PartitionID: partitionID,
Schema: marshaledSchema,

View File

@ -5,6 +5,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
@ -17,7 +18,7 @@ func (c expireCacheConfig) apply(req *proxypb.InvalidateCollMetaCacheRequest) {
return
}
if req.GetBase() == nil {
req.Base = &commonpb.MsgBase{}
req.Base = commonpbutil.NewMsgBase()
}
req.Base.MsgType = commonpb.MsgType_DropCollection
}
@ -39,10 +40,10 @@ func (c *Core) ExpireMetaCache(ctx context.Context, collNames []string, collecti
// if collectionID is specified, invalidate all the collection meta cache with the specified collectionID and return
if collectionID != InvalidCollectionID {
req := proxypb.InvalidateCollMetaCacheRequest{
Base: &commonpb.MsgBase{
Timestamp: ts,
SourceID: c.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(c.session.ServerID),
),
CollectionID: collectionID,
}
return c.proxyClientManager.InvalidateCollectionMetaCache(ctx, &req, opts...)
@ -51,12 +52,12 @@ func (c *Core) ExpireMetaCache(ctx context.Context, collNames []string, collecti
// if only collNames are specified, invalidate the collection meta cache with the specified collectionName
for _, collName := range collNames {
req := proxypb.InvalidateCollMetaCacheRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO, msg type
MsgID: 0, //TODO, msg id
Timestamp: ts,
SourceID: c.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0), //TODO, msg type
commonpbutil.WithMsgID(0), //TODO, msg id
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(c.session.ServerID),
),
CollectionName: collName,
}
err := c.proxyClientManager.InvalidateCollectionMetaCache(ctx, &req, opts...)

View File

@ -6,6 +6,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
ms "github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/metastore/model"
)
@ -139,11 +140,11 @@ func (c *bgGarbageCollector) notifyCollectionGc(ctx context.Context, coll *model
msg := &ms.DropCollectionMsg{
BaseMsg: baseMsg,
DropCollectionRequest: internalpb.DropCollectionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropCollection,
Timestamp: ts,
SourceID: c.s.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DropCollection),
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(c.s.session.ServerID),
),
CollectionName: coll.Name,
CollectionID: coll.CollectionID,
},
@ -172,11 +173,11 @@ func (c *bgGarbageCollector) notifyPartitionGc(ctx context.Context, pChannels []
msg := &ms.DropPartitionMsg{
BaseMsg: baseMsg,
DropPartitionRequest: internalpb.DropPartitionRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_DropPartition,
Timestamp: ts,
SourceID: c.s.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DropPartition),
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(c.s.session.ServerID),
),
PartitionName: partition.PartitionName,
CollectionID: partition.CollectionID,
PartitionID: partition.PartitionID,

View File

@ -33,6 +33,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/tso"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/ratelimitutil"
"github.com/milvus-io/milvus/internal/util/tsoutil"
@ -70,15 +71,17 @@ type Limit = ratelimitutil.Limit
// notifies Proxies to limit rate of requests from clients or reject
// all requests when the cluster met resources issues.
// Limitations:
// 1. DML throughput limitation;
// 2. DDL, DQL qps/rps limitation;
// 1. DML throughput limitation;
// 2. DDL, DQL qps/rps limitation;
//
// Protections:
// 1. TT protection -> dqlRate = maxDQLRate * (maxDelay - ttDelay) / maxDelay
// 2. Memory protection -> dmlRate = maxDMLRate * (highMem - curMem) / (highMem - lowMem)
// 3. Disk quota protection -> force deny writing if exceeded
// 4. DQL Queue length protection -> dqlRate = curDQLRate * CoolOffSpeed
// 5. DQL queue latency protection -> dqlRate = curDQLRate * CoolOffSpeed
// 6. Search result protection -> searchRate = curSearchRate * CoolOffSpeed
// 1. TT protection -> dqlRate = maxDQLRate * (maxDelay - ttDelay) / maxDelay
// 2. Memory protection -> dmlRate = maxDMLRate * (highMem - curMem) / (highMem - lowMem)
// 3. Disk quota protection -> force deny writing if exceeded
// 4. DQL Queue length protection -> dqlRate = curDQLRate * CoolOffSpeed
// 5. DQL queue latency protection -> dqlRate = curDQLRate * CoolOffSpeed
// 6. Search result protection -> searchRate = curSearchRate * CoolOffSpeed
//
// If necessary, user can also manually force to deny RW requests.
type QuotaCenter struct {
// clients
@ -151,7 +154,7 @@ func (q *QuotaCenter) stop() {
})
}
// clearMetrics removes all metrics stored in QuotaCenter.
// clearMetrics removes all metrics stored in QuotaCenter.
func (q *QuotaCenter) clearMetrics() {
q.dataNodeMetrics = make(map[UniqueID]*metricsinfo.DataNodeQuotaMetrics, 0)
q.queryNodeMetrics = make(map[UniqueID]*metricsinfo.QueryNodeQuotaMetrics, 0)
@ -610,11 +613,10 @@ func (q *QuotaCenter) setRates() error {
}
timestamp := tsoutil.ComposeTSByTime(time.Now(), 0)
req := &proxypb.SetRatesRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined,
MsgID: int64(timestamp),
Timestamp: timestamp,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgID(int64(timestamp)),
commonpbutil.WithTimeStamp(timestamp),
),
Rates: map2List(),
}
return q.proxies.SetRates(ctx, req)

View File

@ -52,6 +52,7 @@ import (
"github.com/milvus-io/milvus/internal/tso"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/crypto"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/errorutil"
@ -183,11 +184,11 @@ func (c *Core) sendTimeTick(t Timestamp, reason string) error {
pt[i] = t
}
ttMsg := internalpb.ChannelTimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick,
Timestamp: t,
SourceID: c.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_TimeTick),
commonpbutil.WithTimeStamp(t),
commonpbutil.WithSourceID(c.session.ServerID),
),
ChannelNames: pc,
Timestamps: pt,
DefaultTimestamp: t,
@ -1313,7 +1314,7 @@ func (c *Core) InvalidateCollectionMetaCache(ctx context.Context, in *proxypb.In
return succStatus(), nil
}
//ShowConfigurations returns the configurations of RootCoord matching req.Pattern
// ShowConfigurations returns the configurations of RootCoord matching req.Pattern
func (c *Core) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
if code, ok := c.checkHealthy(); !ok {
return &internalpb.ShowConfigurationsResponse{
@ -1681,11 +1682,11 @@ func (c *Core) ReportImport(ctx context.Context, ir *rootcoordpb.ImportResult) (
// ExpireCredCache will call invalidate credential cache
func (c *Core) ExpireCredCache(ctx context.Context, username string) error {
req := proxypb.InvalidateCredCacheRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO, msg type
MsgID: 0, //TODO, msg id
SourceID: c.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0), //TODO, msg type
commonpbutil.WithMsgID(0), //TODO, msg id
commonpbutil.WithSourceID(c.session.ServerID),
),
Username: username,
}
return c.proxyClientManager.InvalidateCredentialCache(ctx, &req)
@ -1694,11 +1695,11 @@ func (c *Core) ExpireCredCache(ctx context.Context, username string) error {
// UpdateCredCache will call update credential cache
func (c *Core) UpdateCredCache(ctx context.Context, credInfo *internalpb.CredentialInfo) error {
req := proxypb.UpdateCredCacheRequest{
Base: &commonpb.MsgBase{
MsgType: 0, //TODO, msg type
MsgID: 0, //TODO, msg id
SourceID: c.session.ServerID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0), //TODO, msg type
commonpbutil.WithMsgID(0), //TODO, msg id
commonpbutil.WithSourceID(c.session.ServerID),
),
Username: credInfo.Username,
Password: credInfo.Sha256Password,
}
@ -1706,9 +1707,9 @@ func (c *Core) UpdateCredCache(ctx context.Context, credInfo *internalpb.Credent
}
// CreateCredential create new user and password
// 1. decode ciphertext password to raw password
// 2. encrypt raw password
// 3. save in to etcd
// 1. decode ciphertext password to raw password
// 2. encrypt raw password
// 3. save in to etcd
func (c *Core) CreateCredential(ctx context.Context, credInfo *internalpb.CredentialInfo) (*commonpb.Status, error) {
method := "CreateCredential"
metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.TotalLabel).Inc()

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/mq/msgstream"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/timerecord"
"github.com/milvus-io/milvus/internal/util/tsoutil"
@ -301,12 +302,12 @@ func (t *timetickSync) sendTimeTickToChannel(chanNames []string, ts typeutil.Tim
HashValues: []uint32{0},
}
timeTickResult := internalpb.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick,
MsgID: 0,
Timestamp: ts,
SourceID: t.sourceID,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_TimeTick),
commonpbutil.WithMsgID(0),
commonpbutil.WithTimeStamp(ts),
commonpbutil.WithSourceID(t.sourceID),
),
}
timeTickMsg := &msgstream.TimeTickMsg{
BaseMsg: baseMsg,
@ -331,7 +332,7 @@ func (t *timetickSync) getSessionNum() int {
return len(t.sess2ChanTsMap)
}
///////////////////////////////////////////////////////////////////////////////
// /////////////////////////////////////////////////////////////////////////////
// getDmlChannelNames returns list of channel names.
func (t *timetickSync) getDmlChannelNames(count int) []string {
return t.dmlChannels.getChannelNames(count)

View File

@ -20,12 +20,9 @@ import (
"time"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/util/paramtable"
)
const MsgIDNeedFill int64 = -2
var Params paramtable.ComponentParam
const MsgIDNeedFill int64 = 0
type MsgBaseOptions func(*commonpb.MsgBase)
@ -57,32 +54,35 @@ func GetNowTimestamp() uint64 {
return uint64(time.Now().Unix())
}
func SetTargetID(msgBase *commonpb.MsgBase, targetID int64) *commonpb.MsgBase {
if msgBase == nil {
return nil
func FillMsgBaseFromClient(targetID int64) MsgBaseOptions {
return func(msgBase *commonpb.MsgBase) {
if msgBase.Timestamp == 0 {
msgBase.Timestamp = GetNowTimestamp()
}
msgBase.TargetID = targetID
}
msgBase.TargetID = targetID
return msgBase
}
func NewMsgBaseDefault() *commonpb.MsgBase {
func newMsgBaseDefault() *commonpb.MsgBase {
return &commonpb.MsgBase{
MsgType: commonpb.MsgType_Undefined,
MsgID: MsgIDNeedFill,
Timestamp: GetNowTimestamp(),
SourceID: Params.DataCoordCfg.GetNodeID(),
MsgType: commonpb.MsgType_Undefined,
MsgID: MsgIDNeedFill,
}
}
func NewMsgBase(options ...MsgBaseOptions) *commonpb.MsgBase {
msgBase := NewMsgBaseDefault()
msgBase := newMsgBaseDefault()
for _, op := range options {
op(msgBase)
}
return msgBase
}
func NewMsgBaseCopy(msgBase *commonpb.MsgBase, options ...MsgBaseOptions) *commonpb.MsgBase {
func UpdateMsgBase(msgBase *commonpb.MsgBase, options ...MsgBaseOptions) *commonpb.MsgBase {
if msgBase == nil {
return nil
}
msgBaseRt := msgBase
for _, op := range options {
op(msgBaseRt)

View File

@ -18,6 +18,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
)
const (
@ -52,9 +53,9 @@ func ConstructRequestByMetricType(metricType string) (*milvuspb.GetMetricsReques
}
//TODO:: switch metricType to different msgType and return err when metricType is not supported
return &milvuspb.GetMetricsRequest{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_SystemInfo,
},
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_SystemInfo),
),
Request: string(binary),
}, nil
}