mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-08 01:58:34 +08:00
Cherry-pick from master pr: #45018 #45030 Related to #44761 Refactor proxy shard client management by creating a new internal/proxy/shardclient package. This improves code organization and modularity by: - Moving load balancing logic (LookAsideBalancer, RoundRobinBalancer) to shardclient package - Extracting shard client manager and related interfaces into separate package - Relocating shard leader management and client lifecycle code - Adding package documentation (README.md, OWNERS) - Updating proxy code to use the new shardclient package interfaces This change makes the shard client functionality more maintainable and better encapsulated, reducing coupling in the proxy layer. Also consolidates the proxy package's mockery generation to use a centralized `.mockery.yaml` configuration file, aligning with the pattern used by other packages like querycoordv2. Changes - **Makefile**: Replace multiple individual mockery commands with a single config-based invocation for `generate-mockery-proxy` target - **internal/proxy/.mockery.yaml**: Add mockery configuration defining all mock interfaces for proxy and proxy/shardclient packages - **Mock files**: Regenerate mocks using the new configuration: - `mock_cache.go`: Clean up by removing unused interface methods (credential, shard cache, policy methods) - `shardclient/mock_lb_balancer.go`: Update type comments (nodeInfo → NodeInfo) - `shardclient/mock_lb_policy.go`: Update formatting - `shardclient/mock_shardclient_manager.go`: Fix parameter naming consistency (nodeInfo1 → nodeInfo) - **task_search_test.go**: Remove obsolete mock expectations for deprecated cache methods Benefits - Centralized mockery configuration for easier maintenance - Consistent with other packages (querycoordv2, etc.) - Cleaner mock interfaces by removing unused methods - Better type consistency in generated mocks --------- Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
6611 lines
204 KiB
Go
6611 lines
204 KiB
Go
// Licensed to the LF AI & Data foundation under one
|
|
// or more contributor license agreements. See the NOTICE file
|
|
// distributed with this work for additional information
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
// to you under the Apache License, Version 2.0 (the
|
|
// "License"); you may not use this file except in compliance
|
|
// with the License. You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package proxy
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"os"
|
|
"strconv"
|
|
"strings"
|
|
"sync"
|
|
"time"
|
|
|
|
"github.com/cockroachdb/errors"
|
|
"github.com/gin-gonic/gin"
|
|
"github.com/tidwall/gjson"
|
|
"go.opentelemetry.io/otel"
|
|
"go.opentelemetry.io/otel/trace"
|
|
"go.uber.org/zap"
|
|
"golang.org/x/sync/errgroup"
|
|
"google.golang.org/protobuf/encoding/protojson"
|
|
"google.golang.org/protobuf/proto"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/federpb"
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
|
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
|
"github.com/milvus-io/milvus/internal/http"
|
|
"github.com/milvus-io/milvus/internal/proxy/connection"
|
|
"github.com/milvus-io/milvus/internal/proxy/privilege"
|
|
"github.com/milvus-io/milvus/internal/proxy/replicate"
|
|
"github.com/milvus-io/milvus/internal/types"
|
|
"github.com/milvus-io/milvus/internal/util/analyzer"
|
|
"github.com/milvus-io/milvus/internal/util/hookutil"
|
|
"github.com/milvus-io/milvus/internal/util/segcore"
|
|
"github.com/milvus-io/milvus/pkg/v2/common"
|
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
|
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
|
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
|
|
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
|
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
|
"github.com/milvus-io/milvus/pkg/v2/proto/proxypb"
|
|
"github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
|
"github.com/milvus-io/milvus/pkg/v2/util"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/commonpbutil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/crypto"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/logutil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/ratelimitutil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/replicateutil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/requestutil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/retry"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
|
)
|
|
|
|
const moduleName = "Proxy"
|
|
|
|
// GetComponentStates gets the state of Proxy.
|
|
func (node *Proxy) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
|
|
stats := &milvuspb.ComponentStates{
|
|
Status: merr.Success(),
|
|
}
|
|
code := node.GetStateCode()
|
|
log.Ctx(ctx).Debug("Proxy current state", zap.String("StateCode", code.String()))
|
|
nodeID := common.NotRegisteredID
|
|
if node.session != nil && node.session.Registered() {
|
|
nodeID = node.session.ServerID
|
|
}
|
|
info := &milvuspb.ComponentInfo{
|
|
// NodeID: Params.ProxyID, // will race with Proxy.Register()
|
|
NodeID: nodeID,
|
|
Role: typeutil.ProxyRole,
|
|
StateCode: code,
|
|
}
|
|
stats.State = info
|
|
return stats, nil
|
|
}
|
|
|
|
// GetStatisticsChannel gets statistics channel of Proxy.
|
|
func (node *Proxy) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
|
|
return &milvuspb.StringResponse{
|
|
Status: merr.Success(),
|
|
Value: "",
|
|
}, nil
|
|
}
|
|
|
|
// InvalidateCollectionMetaCache invalidate the meta cache of specific collection.
|
|
func (node *Proxy) InvalidateCollectionMetaCache(ctx context.Context, request *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
ctx = logutil.WithModule(ctx, moduleName)
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-InvalidateCollectionMetaCache")
|
|
defer sp.End()
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collectionName", request.CollectionName),
|
|
zap.Int64("collectionID", request.CollectionID),
|
|
zap.String("msgType", request.GetBase().GetMsgType().String()),
|
|
zap.String("partitionName", request.GetPartitionName()),
|
|
)
|
|
|
|
log.Info("received request to invalidate collection meta cache")
|
|
|
|
dbName := request.DbName
|
|
collectionName := request.CollectionName
|
|
collectionID := request.CollectionID
|
|
msgType := request.GetBase().GetMsgType()
|
|
var aliasName []string
|
|
|
|
if globalMetaCache != nil {
|
|
switch msgType {
|
|
case commonpb.MsgType_DropCollection, commonpb.MsgType_RenameCollection, commonpb.MsgType_DropAlias, commonpb.MsgType_AlterAlias, commonpb.MsgType_CreateAlias:
|
|
if request.CollectionID != UniqueID(0) {
|
|
aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, request.GetBase().GetTimestamp(), msgType == commonpb.MsgType_DropCollection)
|
|
for _, name := range aliasName {
|
|
node.shardMgr.DeprecateShardCache(request.GetDbName(), name)
|
|
}
|
|
}
|
|
if collectionName != "" {
|
|
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName) // no need to return error, though collection may be not cached
|
|
node.shardMgr.DeprecateShardCache(request.GetDbName(), collectionName)
|
|
}
|
|
log.Info("complete to invalidate collection meta cache with collection name", zap.String("type", request.GetBase().GetMsgType().String()))
|
|
case commonpb.MsgType_LoadCollection, commonpb.MsgType_ReleaseCollection:
|
|
// All the request from query use collectionID
|
|
if request.CollectionID != UniqueID(0) {
|
|
aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, 0, false)
|
|
for _, name := range aliasName {
|
|
node.shardMgr.DeprecateShardCache(request.GetDbName(), name)
|
|
}
|
|
}
|
|
log.Info("complete to invalidate collection meta cache", zap.String("type", request.GetBase().GetMsgType().String()))
|
|
case commonpb.MsgType_CreatePartition, commonpb.MsgType_DropPartition:
|
|
if request.GetPartitionName() == "" {
|
|
log.Warn("invalidate collection meta cache failed. partitionName is empty")
|
|
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}, nil
|
|
}
|
|
// drop all the alias as well
|
|
if request.CollectionID != UniqueID(0) {
|
|
aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, request.GetBase().GetTimestamp(), false)
|
|
}
|
|
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName)
|
|
log.Info("complete to invalidate collection meta cache", zap.String("type", request.GetBase().GetMsgType().String()))
|
|
case commonpb.MsgType_DropDatabase:
|
|
node.shardMgr.RemoveDatabase(request.GetDbName())
|
|
fallthrough
|
|
case commonpb.MsgType_AlterDatabase:
|
|
globalMetaCache.RemoveDatabase(ctx, request.GetDbName())
|
|
case commonpb.MsgType_AlterCollection, commonpb.MsgType_AlterCollectionField:
|
|
if request.CollectionID != UniqueID(0) {
|
|
aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, 0, false)
|
|
for _, name := range aliasName {
|
|
node.shardMgr.DeprecateShardCache(request.GetDbName(), name)
|
|
}
|
|
}
|
|
if collectionName != "" {
|
|
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName)
|
|
}
|
|
log.Info("complete to invalidate collection meta cache", zap.String("type", request.GetBase().GetMsgType().String()))
|
|
default:
|
|
log.Warn("receive unexpected msgType of invalidate collection meta cache", zap.String("msgType", request.GetBase().GetMsgType().String()))
|
|
if request.CollectionID != UniqueID(0) {
|
|
aliasName = globalMetaCache.RemoveCollectionsByID(ctx, collectionID, request.GetBase().GetTimestamp(), false)
|
|
for _, name := range aliasName {
|
|
node.shardMgr.DeprecateShardCache(request.GetDbName(), name)
|
|
}
|
|
}
|
|
|
|
if collectionName != "" {
|
|
globalMetaCache.RemoveCollection(ctx, request.GetDbName(), collectionName) // no need to return error, though collection may be not cached
|
|
node.shardMgr.DeprecateShardCache(request.GetDbName(), collectionName)
|
|
}
|
|
}
|
|
}
|
|
|
|
if msgType == commonpb.MsgType_DropCollection {
|
|
// no need to handle error, since this Proxy may not create dml stream for the collection.
|
|
node.chMgr.removeDMLStream(request.GetCollectionID())
|
|
// clean up collection level metrics
|
|
metrics.CleanupProxyCollectionMetrics(paramtable.GetNodeID(), dbName, collectionName)
|
|
for _, alias := range aliasName {
|
|
metrics.CleanupProxyCollectionMetrics(paramtable.GetNodeID(), dbName, alias)
|
|
}
|
|
DeregisterSubLabel(ratelimitutil.GetCollectionSubLabel(request.GetDbName(), request.GetCollectionName()))
|
|
} else if msgType == commonpb.MsgType_DropDatabase {
|
|
metrics.CleanupProxyDBMetrics(paramtable.GetNodeID(), request.GetDbName())
|
|
DeregisterSubLabel(ratelimitutil.GetDBSubLabel(request.GetDbName()))
|
|
}
|
|
log.Info("complete to invalidate collection meta cache")
|
|
|
|
return merr.Success(), nil
|
|
}
|
|
|
|
// InvalidateCollectionMetaCache invalidate the meta cache of specific collection.
|
|
func (node *Proxy) InvalidateShardLeaderCache(ctx context.Context, request *proxypb.InvalidateShardLeaderCacheRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
ctx = logutil.WithModule(ctx, moduleName)
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-InvalidateShardLeaderCache")
|
|
defer sp.End()
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Info("received request to invalidate shard leader cache", zap.Int64s("collectionIDs", request.GetCollectionIDs()))
|
|
|
|
node.shardMgr.InvalidateShardLeaderCache(request.GetCollectionIDs())
|
|
|
|
log.Info("complete to invalidate shard leader cache", zap.Int64s("collectionIDs", request.GetCollectionIDs()))
|
|
|
|
return merr.Success(), nil
|
|
}
|
|
|
|
func (node *Proxy) CreateDatabase(ctx context.Context, request *milvuspb.CreateDatabaseRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateDatabase")
|
|
defer sp.End()
|
|
|
|
method := "CreateDatabase"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
cct := &createDatabaseTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
CreateDatabaseRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("dbName", request.DbName),
|
|
)
|
|
|
|
log.Info(rpcReceived(method))
|
|
if err := node.sched.ddQueue.Enqueue(cct); err != nil {
|
|
log.Warn(rpcFailedToEnqueue(method), zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(rpcEnqueued(method))
|
|
if err := cct.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method), zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(rpcDone(method))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
method,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
return cct.result, nil
|
|
}
|
|
|
|
func (node *Proxy) DropDatabase(ctx context.Context, request *milvuspb.DropDatabaseRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropDatabase")
|
|
defer sp.End()
|
|
|
|
method := "DropDatabase"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
dct := &dropDatabaseTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DropDatabaseRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("dbName", request.DbName),
|
|
)
|
|
|
|
log.Info(rpcReceived(method))
|
|
if err := node.sched.ddQueue.Enqueue(dct); err != nil {
|
|
log.Warn(rpcFailedToEnqueue(method), zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(rpcEnqueued(method))
|
|
if err := dct.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method), zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(rpcDone(method))
|
|
DeregisterSubLabel(ratelimitutil.GetDBSubLabel(request.GetDbName()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
method,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dct.result, nil
|
|
}
|
|
|
|
func (node *Proxy) ListDatabases(ctx context.Context, request *milvuspb.ListDatabasesRequest) (*milvuspb.ListDatabasesResponse, error) {
|
|
resp := &milvuspb.ListDatabasesResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ListDatabases")
|
|
defer sp.End()
|
|
|
|
method := "ListDatabases"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
dct := &listDatabaseTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
ListDatabasesRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dct); err != nil {
|
|
log.Warn(rpcFailedToEnqueue(method), zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Info(rpcEnqueued(method))
|
|
if err := dct.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method), zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Info(rpcDone(method), zap.Int("num of db", len(dct.result.DbNames)))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
method,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
return dct.result, nil
|
|
}
|
|
|
|
func (node *Proxy) AlterDatabase(ctx context.Context, request *milvuspb.AlterDatabaseRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-AlterDatabase")
|
|
defer sp.End()
|
|
method := "AlterDatabase"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
act := &alterDatabaseTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
AlterDatabaseRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(act); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()),
|
|
zap.Uint64("timestamp", request.Base.Timestamp))
|
|
|
|
if err := act.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(rpcDone(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return act.result, nil
|
|
}
|
|
|
|
func (node *Proxy) DescribeDatabase(ctx context.Context, request *milvuspb.DescribeDatabaseRequest) (*milvuspb.DescribeDatabaseResponse, error) {
|
|
resp := &milvuspb.DescribeDatabaseResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DescribeDatabase")
|
|
defer sp.End()
|
|
method := "DescribeDatabase"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
act := &describeDatabaseTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DescribeDatabaseRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(act); err != nil {
|
|
log.Warn(rpcFailedToEnqueue(method), zap.Error(err))
|
|
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Debug(rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()),
|
|
zap.Uint64("timestamp", request.Base.Timestamp))
|
|
|
|
if err := act.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Debug(rpcDone(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return act.result, nil
|
|
}
|
|
|
|
// CreateCollection create a collection by the schema.
|
|
// TODO(dragondriver): add more detailed ut for ConsistencyLevel, should we support multiple consistency level in Proxy?
|
|
func (node *Proxy) CreateCollection(ctx context.Context, request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateCollection")
|
|
defer sp.End()
|
|
method := "CreateCollection"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
cct := &createCollectionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
CreateCollectionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
// avoid data race
|
|
lenOfSchema := len(request.Schema)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Int("len(schema)", lenOfSchema),
|
|
zap.Int32("shards_num", request.ShardsNum),
|
|
zap.String("consistency_level", request.ConsistencyLevel.String()),
|
|
)
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(cct); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", cct.BeginTs()),
|
|
zap.Uint64("EndTs", cct.EndTs()),
|
|
zap.Uint64("timestamp", request.Base.Timestamp),
|
|
)
|
|
|
|
if err := cct.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", cct.BeginTs()),
|
|
zap.Uint64("EndTs", cct.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", cct.BeginTs()),
|
|
zap.Uint64("EndTs", cct.EndTs()),
|
|
)
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
method,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
return cct.result, nil
|
|
}
|
|
|
|
// DropCollection drop a collection.
|
|
func (node *Proxy) DropCollection(ctx context.Context, request *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropCollection")
|
|
defer sp.End()
|
|
method := "DropCollection"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
dct := &dropCollectionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DropCollectionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
chMgr: node.chMgr,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
)
|
|
|
|
log.Info("DropCollection received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dct); err != nil {
|
|
log.Warn("DropCollection failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
"DropCollection enqueued",
|
|
zap.Uint64("BeginTs", dct.BeginTs()),
|
|
zap.Uint64("EndTs", dct.EndTs()),
|
|
)
|
|
|
|
if err := dct.WaitToFinish(); err != nil {
|
|
log.Warn("DropCollection failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", dct.BeginTs()),
|
|
zap.Uint64("EndTs", dct.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
"DropCollection done",
|
|
zap.Uint64("BeginTs", dct.BeginTs()),
|
|
zap.Uint64("EndTs", dct.EndTs()),
|
|
)
|
|
DeregisterSubLabel(ratelimitutil.GetCollectionSubLabel(request.GetDbName(), request.GetCollectionName()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
method,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
return dct.result, nil
|
|
}
|
|
|
|
// HasCollection check if the specific collection exists in Milvus.
|
|
func (node *Proxy) HasCollection(ctx context.Context, request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.BoolResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-HasCollection")
|
|
defer sp.End()
|
|
method := "HasCollection"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
)
|
|
|
|
log.Debug("HasCollection received")
|
|
|
|
hct := &hasCollectionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
HasCollectionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
if err := node.sched.ddQueue.Enqueue(hct); err != nil {
|
|
log.Warn("HasCollection failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.BoolResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
"HasCollection enqueued",
|
|
zap.Uint64("BeginTS", hct.BeginTs()),
|
|
zap.Uint64("EndTS", hct.EndTs()),
|
|
)
|
|
|
|
if err := hct.WaitToFinish(); err != nil {
|
|
log.Warn("HasCollection failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", hct.BeginTs()),
|
|
zap.Uint64("EndTS", hct.EndTs()))
|
|
|
|
return &milvuspb.BoolResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
"HasCollection done",
|
|
zap.Uint64("BeginTS", hct.BeginTs()),
|
|
zap.Uint64("EndTS", hct.EndTs()),
|
|
)
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
method,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
return hct.result, nil
|
|
}
|
|
|
|
// LoadCollection load a collection into query nodes.
|
|
func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-LoadCollection")
|
|
defer sp.End()
|
|
method := "LoadCollection"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
lct := &loadCollectionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
LoadCollectionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Bool("refreshMode", request.Refresh),
|
|
)
|
|
|
|
log.Info("LoadCollection received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(lct); err != nil {
|
|
log.Warn("LoadCollection failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
"LoadCollection enqueued",
|
|
zap.Uint64("BeginTS", lct.BeginTs()),
|
|
zap.Uint64("EndTS", lct.EndTs()),
|
|
)
|
|
|
|
if err := lct.WaitToFinish(); err != nil {
|
|
log.Warn("LoadCollection failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", lct.BeginTs()),
|
|
zap.Uint64("EndTS", lct.EndTs()))
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
"LoadCollection done",
|
|
zap.Uint64("BeginTS", lct.BeginTs()),
|
|
zap.Uint64("EndTS", lct.EndTs()),
|
|
)
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
method,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
return lct.result, nil
|
|
}
|
|
|
|
// ReleaseCollection remove the loaded collection from query nodes.
|
|
func (node *Proxy) ReleaseCollection(ctx context.Context, request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ReleaseCollection")
|
|
defer sp.End()
|
|
method := "ReleaseCollection"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
rct := &releaseCollectionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
ReleaseCollectionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(rct); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", rct.BeginTs()),
|
|
zap.Uint64("EndTS", rct.EndTs()))
|
|
|
|
if err := rct.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", rct.BeginTs()),
|
|
zap.Uint64("EndTS", rct.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", rct.BeginTs()),
|
|
zap.Uint64("EndTS", rct.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return rct.result, nil
|
|
}
|
|
|
|
// DescribeCollection get the meta information of specific collection, such as schema, created timestamp and etc.
|
|
func (node *Proxy) DescribeCollection(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
|
|
interceptor, err := NewInterceptor[*milvuspb.DescribeCollectionRequest, *milvuspb.DescribeCollectionResponse](node, "DescribeCollection")
|
|
if err != nil {
|
|
return &milvuspb.DescribeCollectionResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return interceptor.Call(ctx, request)
|
|
}
|
|
|
|
func (node *Proxy) BatchDescribeCollection(ctx context.Context, request *milvuspb.BatchDescribeCollectionRequest) (*milvuspb.BatchDescribeCollectionResponse, error) {
|
|
collectionNames := request.GetCollectionName()
|
|
if len(collectionNames) == 0 {
|
|
return &milvuspb.BatchDescribeCollectionResponse{
|
|
Status: merr.Status(merr.WrapErrParameterInvalidMsg("collection names cannot be empty")),
|
|
}, nil
|
|
}
|
|
|
|
responses := make([]*milvuspb.DescribeCollectionResponse, 0, len(collectionNames))
|
|
|
|
for _, collectionName := range collectionNames {
|
|
describeCollectionRequest := &milvuspb.DescribeCollectionRequest{
|
|
DbName: request.GetDbName(),
|
|
CollectionName: collectionName,
|
|
}
|
|
|
|
describeCollectionResponse, err := node.DescribeCollection(ctx, describeCollectionRequest)
|
|
// If there's an error, create a response with error status
|
|
if err != nil {
|
|
describeCollectionResponse = &milvuspb.DescribeCollectionResponse{
|
|
Status: merr.Status(err),
|
|
CollectionName: collectionName,
|
|
}
|
|
}
|
|
|
|
responses = append(responses, describeCollectionResponse)
|
|
}
|
|
|
|
return &milvuspb.BatchDescribeCollectionResponse{
|
|
Status: merr.Success(),
|
|
Responses: responses,
|
|
}, nil
|
|
}
|
|
|
|
// AddCollectionField add a field to collection
|
|
func (node *Proxy) AddCollectionField(ctx context.Context, request *milvuspb.AddCollectionFieldRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-AddCollectionField")
|
|
defer sp.End()
|
|
|
|
dresp, err := node.DescribeCollection(ctx, &milvuspb.DescribeCollectionRequest{DbName: request.DbName, CollectionName: request.CollectionName})
|
|
|
|
if err := merr.CheckRPCCall(dresp, err); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
task := &addCollectionFieldTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
AddCollectionFieldRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
oldSchema: dresp.GetSchema(),
|
|
}
|
|
|
|
method := "AddCollectionField"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(task); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", task.BeginTs()),
|
|
zap.Uint64("EndTs", task.EndTs()))
|
|
|
|
if err := task.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", task.BeginTs()),
|
|
zap.Uint64("EndTs", task.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", task.BeginTs()),
|
|
zap.Uint64("EndTs", task.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return task.result, nil
|
|
}
|
|
|
|
// GetStatistics get the statistics, such as `num_rows`.
|
|
// WARNING: It is an experimental API
|
|
func (node *Proxy) GetStatistics(ctx context.Context, request *milvuspb.GetStatisticsRequest) (*milvuspb.GetStatisticsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetStatistics")
|
|
defer sp.End()
|
|
method := "GetStatistics"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
g := &getStatisticsTask{
|
|
request: request,
|
|
Condition: NewTaskCondition(ctx),
|
|
ctx: ctx,
|
|
tr: tr,
|
|
mixc: node.mixCoord,
|
|
lb: node.lbPolicy,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName))
|
|
|
|
log.Debug(
|
|
rpcReceived(method),
|
|
zap.Strings("partitions", request.PartitionNames))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(g); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err),
|
|
zap.Strings("partitions", request.PartitionNames))
|
|
|
|
return &milvuspb.GetStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()),
|
|
zap.Strings("partitions", request.PartitionNames))
|
|
|
|
if err := g.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()),
|
|
zap.Strings("partitions", request.PartitionNames))
|
|
|
|
return &milvuspb.GetStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return g.result, nil
|
|
}
|
|
|
|
// GetCollectionStatistics get the collection statistics, such as `num_rows`.
|
|
func (node *Proxy) GetCollectionStatistics(ctx context.Context, request *milvuspb.GetCollectionStatisticsRequest) (*milvuspb.GetCollectionStatisticsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetCollectionStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetCollectionStatistics")
|
|
defer sp.End()
|
|
method := "GetCollectionStatistics"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
g := &getCollectionStatisticsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
GetCollectionStatisticsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(g); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetCollectionStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()))
|
|
|
|
if err := g.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()))
|
|
|
|
return &milvuspb.GetCollectionStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return g.result, nil
|
|
}
|
|
|
|
// ShowCollections list all collections in Milvus.
|
|
func (node *Proxy) ShowCollections(ctx context.Context, request *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ShowCollectionsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ShowCollections")
|
|
defer sp.End()
|
|
method := "ShowCollections"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
sct := &showCollectionsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
ShowCollectionsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("DbName", request.DbName),
|
|
zap.Uint64("TimeStamp", request.TimeStamp),
|
|
zap.String("ShowType", request.Type.String()))
|
|
|
|
log.Debug("ShowCollections received",
|
|
zap.Any("CollectionNames", request.CollectionNames))
|
|
|
|
err := node.sched.ddQueue.Enqueue(sct)
|
|
if err != nil {
|
|
log.Warn("ShowCollections failed to enqueue",
|
|
zap.Error(err),
|
|
zap.Any("CollectionNames", request.CollectionNames))
|
|
|
|
return &milvuspb.ShowCollectionsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug("ShowCollections enqueued",
|
|
zap.Any("CollectionNames", request.CollectionNames))
|
|
|
|
err = sct.WaitToFinish()
|
|
if err != nil {
|
|
log.Warn("ShowCollections failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Any("CollectionNames", request.CollectionNames))
|
|
|
|
return &milvuspb.ShowCollectionsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug("ShowCollections Done",
|
|
zap.Int("len(CollectionNames)", len(request.CollectionNames)),
|
|
zap.Int("num_collections", len(sct.result.CollectionNames)))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return sct.result, nil
|
|
}
|
|
|
|
func (node *Proxy) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-AlterCollection")
|
|
defer sp.End()
|
|
method := "AlterCollection"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
act := &alterCollectionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
AlterCollectionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Any("props", request.Properties))
|
|
|
|
log.Info(
|
|
rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(act); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()),
|
|
zap.Uint64("timestamp", request.Base.Timestamp))
|
|
|
|
if err := act.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return act.result, nil
|
|
}
|
|
|
|
func (node *Proxy) AlterCollectionField(ctx context.Context, request *milvuspb.AlterCollectionFieldRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-AlterCollectionField")
|
|
defer sp.End()
|
|
method := "AlterCollectionField"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
act := &alterCollectionFieldTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
AlterCollectionFieldRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("fieldName", request.FieldName),
|
|
zap.Any("props", request.Properties))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(act); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()),
|
|
zap.Uint64("timestamp", request.Base.Timestamp))
|
|
|
|
if err := act.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", act.BeginTs()),
|
|
zap.Uint64("EndTs", act.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return act.result, nil
|
|
}
|
|
|
|
// CreatePartition create a partition in specific collection.
|
|
func (node *Proxy) CreatePartition(ctx context.Context, request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreatePartition")
|
|
defer sp.End()
|
|
method := "CreatePartition"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
cpt := &createPartitionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
CreatePartitionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
result: nil,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("partition", request.PartitionName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(cpt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", cpt.BeginTs()),
|
|
zap.Uint64("EndTS", cpt.EndTs()))
|
|
|
|
if err := cpt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", cpt.BeginTs()),
|
|
zap.Uint64("EndTS", cpt.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", cpt.BeginTs()),
|
|
zap.Uint64("EndTS", cpt.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return cpt.result, nil
|
|
}
|
|
|
|
// DropPartition drop a partition in specific collection.
|
|
func (node *Proxy) DropPartition(ctx context.Context, request *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropPartition")
|
|
defer sp.End()
|
|
method := "DropPartition"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
dpt := &dropPartitionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DropPartitionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
result: nil,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("partition", request.PartitionName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dpt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", dpt.BeginTs()),
|
|
zap.Uint64("EndTS", dpt.EndTs()))
|
|
|
|
if err := dpt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", dpt.BeginTs()),
|
|
zap.Uint64("EndTS", dpt.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", dpt.BeginTs()),
|
|
zap.Uint64("EndTS", dpt.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dpt.result, nil
|
|
}
|
|
|
|
// HasPartition check if partition exist.
|
|
func (node *Proxy) HasPartition(ctx context.Context, request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.BoolResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-HasPartition")
|
|
defer sp.End()
|
|
method := "HasPartition"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
hpt := &hasPartitionTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
HasPartitionRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
result: nil,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("partition", request.PartitionName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(hpt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.BoolResponse{
|
|
Status: merr.Status(err),
|
|
Value: false,
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", hpt.BeginTs()),
|
|
zap.Uint64("EndTS", hpt.EndTs()))
|
|
|
|
if err := hpt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", hpt.BeginTs()),
|
|
zap.Uint64("EndTS", hpt.EndTs()))
|
|
|
|
return &milvuspb.BoolResponse{
|
|
Status: merr.Status(err),
|
|
Value: false,
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", hpt.BeginTs()),
|
|
zap.Uint64("EndTS", hpt.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return hpt.result, nil
|
|
}
|
|
|
|
// LoadPartitions load specific partitions into query nodes.
|
|
func (node *Proxy) LoadPartitions(ctx context.Context, request *milvuspb.LoadPartitionsRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-LoadPartitions")
|
|
defer sp.End()
|
|
method := "LoadPartitions"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
lpt := &loadPartitionsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
LoadPartitionsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Strings("partitions", request.PartitionNames),
|
|
zap.Bool("refreshMode", request.Refresh))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(lpt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", lpt.BeginTs()),
|
|
zap.Uint64("EndTS", lpt.EndTs()))
|
|
|
|
if err := lpt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", lpt.BeginTs()),
|
|
zap.Uint64("EndTS", lpt.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", lpt.BeginTs()),
|
|
zap.Uint64("EndTS", lpt.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return lpt.result, nil
|
|
}
|
|
|
|
// ReleasePartitions release specific partitions from query nodes.
|
|
func (node *Proxy) ReleasePartitions(ctx context.Context, request *milvuspb.ReleasePartitionsRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ReleasePartitions")
|
|
defer sp.End()
|
|
|
|
rpt := &releasePartitionsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
ReleasePartitionsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "ReleasePartitions"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Any("partitions", request.PartitionNames))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(rpt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", rpt.BeginTs()),
|
|
zap.Uint64("EndTS", rpt.EndTs()))
|
|
|
|
if err := rpt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", rpt.BeginTs()),
|
|
zap.Uint64("EndTS", rpt.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", rpt.BeginTs()),
|
|
zap.Uint64("EndTS", rpt.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return rpt.result, nil
|
|
}
|
|
|
|
// GetPartitionStatistics get the statistics of partition, such as num_rows.
|
|
func (node *Proxy) GetPartitionStatistics(ctx context.Context, request *milvuspb.GetPartitionStatisticsRequest) (*milvuspb.GetPartitionStatisticsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetPartitionStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetPartitionStatistics")
|
|
defer sp.End()
|
|
method := "GetPartitionStatistics"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
g := &getPartitionStatisticsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
GetPartitionStatisticsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("partition", request.PartitionName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(g); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetPartitionStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()))
|
|
|
|
if err := g.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()))
|
|
|
|
return &milvuspb.GetPartitionStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", g.BeginTs()),
|
|
zap.Uint64("EndTS", g.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return g.result, nil
|
|
}
|
|
|
|
// ShowPartitions list all partitions in the specific collection.
|
|
func (node *Proxy) ShowPartitions(ctx context.Context, request *milvuspb.ShowPartitionsRequest) (*milvuspb.ShowPartitionsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ShowPartitionsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ShowPartitions")
|
|
defer sp.End()
|
|
|
|
spt := &showPartitionsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
ShowPartitionsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
result: nil,
|
|
}
|
|
|
|
method := "ShowPartitions"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(zap.String("role", typeutil.ProxyRole))
|
|
|
|
log.Debug(
|
|
rpcReceived(method),
|
|
zap.Any("request", request))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(spt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err),
|
|
zap.Any("request", request))
|
|
|
|
return &milvuspb.ShowPartitionsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTS", spt.BeginTs()),
|
|
zap.Uint64("EndTS", spt.EndTs()),
|
|
zap.String("db", spt.ShowPartitionsRequest.DbName),
|
|
zap.String("collection", spt.ShowPartitionsRequest.CollectionName),
|
|
zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames))
|
|
|
|
if err := spt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", spt.BeginTs()),
|
|
zap.Uint64("EndTS", spt.EndTs()),
|
|
zap.String("db", spt.ShowPartitionsRequest.DbName),
|
|
zap.String("collection", spt.ShowPartitionsRequest.CollectionName),
|
|
zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames))
|
|
|
|
return &milvuspb.ShowPartitionsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTS", spt.BeginTs()),
|
|
zap.Uint64("EndTS", spt.EndTs()),
|
|
zap.String("db", spt.ShowPartitionsRequest.DbName),
|
|
zap.String("collection", spt.ShowPartitionsRequest.CollectionName),
|
|
zap.Any("partitions", spt.ShowPartitionsRequest.PartitionNames))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return spt.result, nil
|
|
}
|
|
|
|
func (node *Proxy) GetLoadingProgress(ctx context.Context, request *milvuspb.GetLoadingProgressRequest) (*milvuspb.GetLoadingProgressResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetLoadingProgressResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
method := "GetLoadingProgress"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetLoadingProgress")
|
|
defer sp.End()
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug(
|
|
rpcReceived(method),
|
|
zap.Any("request", request))
|
|
|
|
getErrResponse := func(err error) *milvuspb.GetLoadingProgressResponse {
|
|
log.Warn("fail to get loading progress",
|
|
zap.String("collectionName", request.CollectionName),
|
|
zap.Strings("partitionName", request.PartitionNames),
|
|
zap.Error(err))
|
|
if errors.Is(err, merr.ErrServiceMemoryLimitExceeded) {
|
|
return &milvuspb.GetLoadingProgressResponse{
|
|
Status: merr.Status(err),
|
|
}
|
|
}
|
|
return &milvuspb.GetLoadingProgressResponse{
|
|
Status: merr.Status(err),
|
|
}
|
|
}
|
|
if err := validateCollectionName(request.CollectionName); err != nil {
|
|
return getErrResponse(err), nil
|
|
}
|
|
collectionID, err := globalMetaCache.GetCollectionID(ctx, request.GetDbName(), request.CollectionName)
|
|
if err != nil {
|
|
return getErrResponse(err), nil
|
|
}
|
|
|
|
msgBase := commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_SystemInfo),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
)
|
|
if request.Base == nil {
|
|
request.Base = msgBase
|
|
} else {
|
|
request.Base.MsgID = msgBase.MsgID
|
|
request.Base.Timestamp = msgBase.Timestamp
|
|
request.Base.SourceID = msgBase.SourceID
|
|
}
|
|
|
|
var (
|
|
loadProgress int64
|
|
refreshProgress int64
|
|
)
|
|
if len(request.GetPartitionNames()) == 0 {
|
|
if loadProgress, refreshProgress, err = getCollectionProgress(ctx, node.mixCoord, request.GetBase(), collectionID); err != nil {
|
|
return getErrResponse(err), nil
|
|
}
|
|
} else {
|
|
if loadProgress, refreshProgress, err = getPartitionProgress(ctx, node.mixCoord, request.GetBase(),
|
|
request.GetPartitionNames(), request.GetCollectionName(), collectionID, request.GetDbName()); err != nil {
|
|
return getErrResponse(err), nil
|
|
}
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Any("request", request),
|
|
zap.Int64("loadProgress", loadProgress),
|
|
zap.Int64("refreshProgress", refreshProgress))
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return &milvuspb.GetLoadingProgressResponse{
|
|
Status: merr.Success(),
|
|
Progress: loadProgress,
|
|
RefreshProgress: refreshProgress,
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) GetLoadState(ctx context.Context, request *milvuspb.GetLoadStateRequest) (resp *milvuspb.GetLoadStateResponse, err error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetLoadStateResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
method := "GetLoadState"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetLoadState")
|
|
defer sp.End()
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug(
|
|
rpcReceived(method),
|
|
zap.Any("request", request))
|
|
|
|
getErrResponse := func(err error) *milvuspb.GetLoadStateResponse {
|
|
log.Warn("fail to get load state",
|
|
zap.String("collection_name", request.CollectionName),
|
|
zap.Strings("partition_name", request.PartitionNames),
|
|
zap.Error(err))
|
|
return &milvuspb.GetLoadStateResponse{
|
|
Status: merr.Status(err),
|
|
}
|
|
}
|
|
|
|
if err := validateCollectionName(request.CollectionName); err != nil {
|
|
return getErrResponse(err), nil
|
|
}
|
|
|
|
successResponse := &milvuspb.GetLoadStateResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
defer func() {
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Any("request", request),
|
|
zap.Any("response", resp),
|
|
zap.Error(err),
|
|
)
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
}()
|
|
|
|
collectionID, err := globalMetaCache.GetCollectionID(ctx, request.GetDbName(), request.CollectionName)
|
|
if err != nil {
|
|
log.Warn("failed to get collection id",
|
|
zap.String("dbName", request.GetDbName()),
|
|
zap.String("collectionName", request.CollectionName),
|
|
zap.Error(err))
|
|
successResponse.State = commonpb.LoadState_LoadStateNotExist
|
|
return successResponse, nil
|
|
}
|
|
|
|
msgBase := commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_SystemInfo),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
)
|
|
if request.Base == nil {
|
|
request.Base = msgBase
|
|
} else {
|
|
request.Base.MsgID = msgBase.MsgID
|
|
request.Base.Timestamp = msgBase.Timestamp
|
|
request.Base.SourceID = msgBase.SourceID
|
|
}
|
|
|
|
var progress int64
|
|
if len(request.GetPartitionNames()) == 0 {
|
|
if progress, _, err = getCollectionProgress(ctx, node.mixCoord, request.GetBase(), collectionID); err != nil {
|
|
if errors.Is(err, merr.ErrCollectionNotLoaded) {
|
|
successResponse.State = commonpb.LoadState_LoadStateNotLoad
|
|
return successResponse, nil
|
|
}
|
|
return &milvuspb.GetLoadStateResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
} else {
|
|
if progress, _, err = getPartitionProgress(ctx, node.mixCoord, request.GetBase(),
|
|
request.GetPartitionNames(), request.GetCollectionName(), collectionID, request.GetDbName()); err != nil {
|
|
if errors.IsAny(err,
|
|
merr.ErrCollectionNotLoaded,
|
|
merr.ErrPartitionNotLoaded) {
|
|
successResponse.State = commonpb.LoadState_LoadStateNotLoad
|
|
return successResponse, nil
|
|
}
|
|
return &milvuspb.GetLoadStateResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
}
|
|
if progress >= 100 {
|
|
successResponse.State = commonpb.LoadState_LoadStateLoaded
|
|
} else {
|
|
successResponse.State = commonpb.LoadState_LoadStateLoading
|
|
}
|
|
return successResponse, nil
|
|
}
|
|
|
|
// CreateIndex create index for collection.
|
|
func (node *Proxy) CreateIndex(ctx context.Context, request *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateIndex")
|
|
defer sp.End()
|
|
|
|
cit := &createIndexTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
req: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "CreateIndex"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("field", request.FieldName),
|
|
zap.Any("extra_params", request.ExtraParams))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(cit); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", cit.BeginTs()),
|
|
zap.Uint64("EndTs", cit.EndTs()))
|
|
|
|
if err := cit.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", cit.BeginTs()),
|
|
zap.Uint64("EndTs", cit.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", cit.BeginTs()),
|
|
zap.Uint64("EndTs", cit.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return cit.result, nil
|
|
}
|
|
|
|
func (node *Proxy) AlterIndex(ctx context.Context, request *milvuspb.AlterIndexRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-AlterIndex")
|
|
defer sp.End()
|
|
|
|
task := &alterIndexTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
req: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "AlterIndex"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("indexName", request.GetIndexName()),
|
|
zap.Any("extraParams", request.ExtraParams))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(task); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", task.BeginTs()),
|
|
zap.Uint64("EndTs", task.EndTs()))
|
|
|
|
if err := task.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", task.BeginTs()),
|
|
zap.Uint64("EndTs", task.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", task.BeginTs()),
|
|
zap.Uint64("EndTs", task.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return task.result, nil
|
|
}
|
|
|
|
// DescribeIndex get the meta information of index, such as index state, index id and etc.
|
|
func (node *Proxy) DescribeIndex(ctx context.Context, request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.DescribeIndexResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DescribeIndex")
|
|
defer sp.End()
|
|
|
|
dit := &describeIndexTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DescribeIndexRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "DescribeIndex"
|
|
// avoid data race
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("field", request.FieldName),
|
|
zap.String("index name", request.IndexName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dit); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.DescribeIndexResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
if err := dit.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
return &milvuspb.DescribeIndexResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dit.result, nil
|
|
}
|
|
|
|
// GetIndexStatistics get the information of index.
|
|
func (node *Proxy) GetIndexStatistics(ctx context.Context, request *milvuspb.GetIndexStatisticsRequest) (*milvuspb.GetIndexStatisticsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetIndexStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetIndexStatistics")
|
|
defer sp.End()
|
|
|
|
dit := &getIndexStatisticsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
GetIndexStatisticsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "GetIndexStatistics"
|
|
// avoid data race
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("index name", request.IndexName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dit); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetIndexStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
if err := dit.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method), zap.Error(err), zap.Uint64("BeginTs", dit.BeginTs()), zap.Uint64("EndTs", dit.EndTs()))
|
|
return &milvuspb.GetIndexStatisticsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
return dit.result, nil
|
|
}
|
|
|
|
// DropIndex drop the index of collection.
|
|
func (node *Proxy) DropIndex(ctx context.Context, request *milvuspb.DropIndexRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropIndex")
|
|
defer sp.End()
|
|
|
|
dit := &dropIndexTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DropIndexRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "DropIndex"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("field", request.FieldName),
|
|
zap.String("index name", request.IndexName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dit); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
if err := dit.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", dit.BeginTs()),
|
|
zap.Uint64("EndTs", dit.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dit.result, nil
|
|
}
|
|
|
|
// GetIndexBuildProgress gets index build progress with field_name and index_name.
|
|
// IndexRows is the num of indexed rows. And TotalRows is the total number of segment rows.
|
|
// Deprecated: use DescribeIndex instead
|
|
func (node *Proxy) GetIndexBuildProgress(ctx context.Context, request *milvuspb.GetIndexBuildProgressRequest) (*milvuspb.GetIndexBuildProgressResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetIndexBuildProgressResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetIndexBuildProgress")
|
|
defer sp.End()
|
|
|
|
gibpt := &getIndexBuildProgressTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
GetIndexBuildProgressRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "GetIndexBuildProgress"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("field", request.FieldName),
|
|
zap.String("index name", request.IndexName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(gibpt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetIndexBuildProgressResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", gibpt.BeginTs()),
|
|
zap.Uint64("EndTs", gibpt.EndTs()))
|
|
|
|
if err := gibpt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", gibpt.BeginTs()),
|
|
zap.Uint64("EndTs", gibpt.EndTs()))
|
|
|
|
return &milvuspb.GetIndexBuildProgressResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", gibpt.BeginTs()),
|
|
zap.Uint64("EndTs", gibpt.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return gibpt.result, nil
|
|
}
|
|
|
|
// GetIndexState get the build-state of index.
|
|
// Deprecated: use DescribeIndex instead
|
|
func (node *Proxy) GetIndexState(ctx context.Context, request *milvuspb.GetIndexStateRequest) (*milvuspb.GetIndexStateResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.GetIndexStateResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetIndexState")
|
|
defer sp.End()
|
|
|
|
dipt := &getIndexStateTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
GetIndexStateRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "GetIndexState"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("field", request.FieldName),
|
|
zap.String("index name", request.IndexName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dipt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetIndexStateResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", dipt.BeginTs()),
|
|
zap.Uint64("EndTs", dipt.EndTs()))
|
|
|
|
if err := dipt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", dipt.BeginTs()),
|
|
zap.Uint64("EndTs", dipt.EndTs()))
|
|
|
|
return &milvuspb.GetIndexStateResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", dipt.BeginTs()),
|
|
zap.Uint64("EndTs", dipt.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dipt.result, nil
|
|
}
|
|
|
|
// Insert insert records into collection.
|
|
func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest) (*milvuspb.MutationResult, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Insert")
|
|
defer sp.End()
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("partition", request.PartitionName),
|
|
zap.Int("len(FieldsData)", len(request.FieldsData)),
|
|
zap.Int("len(HashKeys)", len(request.HashKeys)),
|
|
zap.Uint32("NumRows", request.NumRows),
|
|
)
|
|
method := "Insert"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
metrics.GetStats(ctx).
|
|
SetNodeID(paramtable.GetNodeID()).
|
|
SetInboundLabel(metrics.InsertLabel).
|
|
SetDatabaseName(request.GetDbName()).
|
|
SetCollectionName(request.GetCollectionName())
|
|
|
|
it := &insertTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
insertMsg: &msgstream.InsertMsg{
|
|
BaseMsg: msgstream.BaseMsg{
|
|
HashValues: request.HashKeys,
|
|
},
|
|
InsertRequest: &msgpb.InsertRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_Insert),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
),
|
|
DbName: request.GetDbName(),
|
|
CollectionName: request.CollectionName,
|
|
PartitionName: request.PartitionName,
|
|
FieldsData: request.FieldsData,
|
|
NumRows: uint64(request.NumRows),
|
|
Version: msgpb.InsertDataVersion_ColumnBased,
|
|
Namespace: request.Namespace,
|
|
},
|
|
},
|
|
idAllocator: node.rowIDAllocator,
|
|
chMgr: node.chMgr,
|
|
schemaTimestamp: request.SchemaTimestamp,
|
|
}
|
|
|
|
constructFailedResponse := func(err error) *milvuspb.MutationResult {
|
|
numRows := request.NumRows
|
|
errIndex := make([]uint32, numRows)
|
|
for i := uint32(0); i < numRows; i++ {
|
|
errIndex[i] = i
|
|
}
|
|
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
ErrIndex: errIndex,
|
|
}
|
|
}
|
|
|
|
log.Debug("Enqueue insert request in Proxy")
|
|
|
|
if err := node.sched.dmQueue.Enqueue(it); err != nil {
|
|
log.Warn("Failed to enqueue insert task: " + err.Error())
|
|
return constructFailedResponse(merr.WrapErrAsInputErrorWhen(err, merr.ErrCollectionNotFound, merr.ErrDatabaseNotFound)), nil
|
|
}
|
|
|
|
log.Debug("Detail of insert request in Proxy")
|
|
|
|
if err := it.WaitToFinish(); err != nil {
|
|
log.Warn("Failed to execute insert task in task scheduler: " + err.Error())
|
|
return constructFailedResponse(err), nil
|
|
}
|
|
|
|
if it.result.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
|
setErrorIndex := func() {
|
|
numRows := request.NumRows
|
|
errIndex := make([]uint32, numRows)
|
|
for i := uint32(0); i < numRows; i++ {
|
|
errIndex[i] = i
|
|
}
|
|
it.result.ErrIndex = errIndex
|
|
}
|
|
|
|
setErrorIndex()
|
|
log.Warn("fail to insert data", zap.Uint32s("err_index", it.result.ErrIndex))
|
|
}
|
|
|
|
// InsertCnt always equals to the number of entities in the request
|
|
it.result.InsertCnt = int64(request.NumRows)
|
|
|
|
rateCol.Add(internalpb.RateType_DMLInsert.String(), float64(it.insertMsg.Size()))
|
|
|
|
successCnt := it.result.InsertCnt - int64(len(it.result.ErrIndex))
|
|
username := GetCurUserFromContextOrDefault(ctx)
|
|
nodeID := paramtable.GetStringNodeID()
|
|
dbName := request.DbName
|
|
collectionName := request.CollectionName
|
|
|
|
v := hookutil.GetExtension().Report(map[string]any{
|
|
hookutil.OpTypeKey: hookutil.OpTypeInsert,
|
|
hookutil.DatabaseKey: dbName,
|
|
hookutil.UsernameKey: username,
|
|
hookutil.RequestDataSizeKey: proto.Size(request),
|
|
hookutil.SuccessCntKey: successCnt,
|
|
hookutil.FailCntKey: len(it.result.ErrIndex),
|
|
})
|
|
SetReportValue(it.result.GetStatus(), v)
|
|
if merr.Ok(it.result.GetStatus()) {
|
|
metrics.ProxyReportValue.WithLabelValues(nodeID, hookutil.OpTypeInsert, request.DbName, username).Add(float64(v))
|
|
}
|
|
metrics.ProxyInsertVectors.
|
|
WithLabelValues(nodeID, dbName, collectionName).
|
|
Add(float64(successCnt))
|
|
metrics.ProxyMutationLatency.
|
|
WithLabelValues(nodeID, metrics.InsertLabel, dbName, collectionName).
|
|
Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
metrics.ProxyCollectionMutationLatency.
|
|
WithLabelValues(nodeID, metrics.InsertLabel, dbName, collectionName).
|
|
Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return it.result, nil
|
|
}
|
|
|
|
// Delete delete records from collection, then these records cannot be searched.
|
|
func (node *Proxy) Delete(ctx context.Context, request *milvuspb.DeleteRequest) (*milvuspb.MutationResult, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Delete")
|
|
defer sp.End()
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.String("partition", request.PartitionName),
|
|
zap.String("expr", request.Expr),
|
|
)
|
|
log.Debug("Start processing delete request in Proxy")
|
|
defer log.Debug("Finish processing delete request in Proxy")
|
|
method := "Delete"
|
|
|
|
metrics.GetStats(ctx).
|
|
SetNodeID(paramtable.GetNodeID()).
|
|
SetInboundLabel(metrics.DeleteLabel).
|
|
SetDatabaseName(request.GetDbName()).
|
|
SetCollectionName(request.GetCollectionName())
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
var limiter types.Limiter
|
|
if node.enableComplexDeleteLimit {
|
|
limiter, _ = node.GetRateLimiter()
|
|
}
|
|
|
|
dr := &deleteRunner{
|
|
req: request,
|
|
idAllocator: node.rowIDAllocator,
|
|
tsoAllocatorIns: node.tsoAllocator,
|
|
chMgr: node.chMgr,
|
|
queue: node.sched.dmQueue,
|
|
lb: node.lbPolicy,
|
|
limiter: limiter,
|
|
}
|
|
|
|
log.Debug("init delete runner in Proxy")
|
|
if err := dr.Init(ctx); err != nil {
|
|
log.Error("Failed to enqueue delete task: " + err.Error())
|
|
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug("Run delete in Proxy")
|
|
|
|
if err := dr.Run(ctx); err != nil {
|
|
log.Error("Failed to run delete task: " + err.Error())
|
|
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
receiveSize := proto.Size(dr.req)
|
|
rateCol.Add(internalpb.RateType_DMLDelete.String(), float64(receiveSize))
|
|
|
|
successCnt := dr.result.GetDeleteCnt()
|
|
|
|
dbName := request.DbName
|
|
nodeID := paramtable.GetStringNodeID()
|
|
|
|
username := GetCurUserFromContextOrDefault(ctx)
|
|
collectionName := request.CollectionName
|
|
v := hookutil.GetExtension().Report(map[string]any{
|
|
hookutil.OpTypeKey: hookutil.OpTypeDelete,
|
|
hookutil.DatabaseKey: dbName,
|
|
hookutil.UsernameKey: username,
|
|
hookutil.SuccessCntKey: successCnt,
|
|
hookutil.RelatedCntKey: dr.allQueryCnt.Load(),
|
|
})
|
|
SetReportValue(dr.result.GetStatus(), v)
|
|
|
|
if Params.QueryNodeCfg.StorageUsageTrackingEnabled.GetAsBool() {
|
|
metrics.ProxyScannedRemoteMB.WithLabelValues(nodeID, metrics.DeleteLabel, dbName, collectionName).Add(float64(dr.scannedRemoteBytes.Load()) / 1024 / 1024)
|
|
metrics.ProxyScannedTotalMB.WithLabelValues(nodeID, metrics.DeleteLabel, dbName, collectionName).Add(float64(dr.scannedTotalBytes.Load()) / 1024 / 1024)
|
|
}
|
|
|
|
SetStorageCost(dr.result.GetStatus(), segcore.StorageCost{
|
|
ScannedRemoteBytes: dr.scannedRemoteBytes.Load(),
|
|
ScannedTotalBytes: dr.scannedTotalBytes.Load(),
|
|
})
|
|
|
|
if merr.Ok(dr.result.GetStatus()) {
|
|
metrics.ProxyReportValue.WithLabelValues(nodeID, hookutil.OpTypeDelete, dbName, username).Add(float64(v))
|
|
}
|
|
|
|
metrics.ProxyMutationLatency.
|
|
WithLabelValues(nodeID, metrics.DeleteLabel, dbName, collectionName).
|
|
Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
metrics.ProxyCollectionMutationLatency.WithLabelValues(nodeID, metrics.DeleteLabel, dbName, collectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dr.result, nil
|
|
}
|
|
|
|
// Upsert upsert records into collection.
|
|
func (node *Proxy) Upsert(ctx context.Context, request *milvuspb.UpsertRequest) (*milvuspb.MutationResult, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Upsert")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.GetDbName()),
|
|
zap.String("collection", request.GetCollectionName()),
|
|
zap.String("partition", request.GetPartitionName()),
|
|
zap.Uint32("NumRows", request.GetNumRows()),
|
|
zap.Bool("partialUpdate", request.GetPartialUpdate()),
|
|
)
|
|
log.Debug("Start processing upsert request in Proxy")
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
method := "Upsert"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
metrics.GetStats(ctx).
|
|
SetNodeID(paramtable.GetNodeID()).
|
|
SetInboundLabel(metrics.UpsertLabel).
|
|
SetDatabaseName(request.GetDbName()).
|
|
SetCollectionName(request.GetCollectionName())
|
|
|
|
request.Base = commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_Upsert),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
)
|
|
|
|
it := &upsertTask{
|
|
baseMsg: msgstream.BaseMsg{
|
|
HashValues: request.HashKeys,
|
|
},
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
req: request,
|
|
result: &milvuspb.MutationResult{
|
|
Status: merr.Success(),
|
|
IDs: &schemapb.IDs{
|
|
IdField: nil,
|
|
},
|
|
},
|
|
|
|
idAllocator: node.rowIDAllocator,
|
|
chMgr: node.chMgr,
|
|
schemaTimestamp: request.SchemaTimestamp,
|
|
node: node,
|
|
}
|
|
|
|
log.Debug("Enqueue upsert request in Proxy",
|
|
zap.Int("len(FieldsData)", len(request.FieldsData)),
|
|
zap.Int("len(HashKeys)", len(request.HashKeys)))
|
|
|
|
if err := node.sched.dmQueue.Enqueue(it); err != nil {
|
|
log.Info("Failed to enqueue upsert task",
|
|
zap.Error(err))
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug("Detail of upsert request in Proxy",
|
|
zap.Uint64("BeginTS", it.BeginTs()),
|
|
zap.Uint64("EndTS", it.EndTs()))
|
|
|
|
if err := it.WaitToFinish(); err != nil {
|
|
log.Info("Failed to execute insert task in task scheduler",
|
|
zap.Error(err))
|
|
// Not every error case changes the status internally
|
|
// change status there to handle it
|
|
if it.result.GetStatus().GetErrorCode() == commonpb.ErrorCode_Success {
|
|
it.result.Status = merr.Status(err)
|
|
}
|
|
|
|
numRows := request.NumRows
|
|
errIndex := make([]uint32, numRows)
|
|
for i := uint32(0); i < numRows; i++ {
|
|
errIndex[i] = i
|
|
}
|
|
|
|
return &milvuspb.MutationResult{
|
|
Status: merr.Status(err),
|
|
ErrIndex: errIndex,
|
|
}, nil
|
|
}
|
|
|
|
if it.result.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
|
setErrorIndex := func() {
|
|
numRows := request.NumRows
|
|
errIndex := make([]uint32, numRows)
|
|
for i := uint32(0); i < numRows; i++ {
|
|
errIndex[i] = i
|
|
}
|
|
it.result.ErrIndex = errIndex
|
|
}
|
|
setErrorIndex()
|
|
}
|
|
|
|
// UpsertCnt always equals to the number of entities in the request
|
|
it.result.UpsertCnt = int64(request.NumRows)
|
|
|
|
username := GetCurUserFromContextOrDefault(ctx)
|
|
nodeID := paramtable.GetStringNodeID()
|
|
dbName := request.DbName
|
|
collectionName := request.CollectionName
|
|
v := hookutil.GetExtension().Report(map[string]any{
|
|
hookutil.OpTypeKey: hookutil.OpTypeUpsert,
|
|
hookutil.DatabaseKey: request.DbName,
|
|
hookutil.UsernameKey: username,
|
|
hookutil.RequestDataSizeKey: proto.Size(it.req),
|
|
hookutil.SuccessCntKey: it.result.UpsertCnt,
|
|
hookutil.FailCntKey: len(it.result.ErrIndex),
|
|
})
|
|
SetReportValue(it.result.GetStatus(), v)
|
|
SetStorageCost(it.result.GetStatus(), it.storageCost)
|
|
if Params.QueryNodeCfg.StorageUsageTrackingEnabled.GetAsBool() {
|
|
metrics.ProxyScannedRemoteMB.WithLabelValues(nodeID, metrics.UpsertLabel, dbName, collectionName).Add(float64(it.storageCost.ScannedRemoteBytes) / 1024 / 1024)
|
|
metrics.ProxyScannedTotalMB.WithLabelValues(nodeID, metrics.UpsertLabel, dbName, collectionName).Add(float64(it.storageCost.ScannedTotalBytes) / 1024 / 1024)
|
|
}
|
|
if merr.Ok(it.result.GetStatus()) {
|
|
metrics.ProxyReportValue.WithLabelValues(nodeID, hookutil.OpTypeUpsert, dbName, username).Add(float64(v))
|
|
}
|
|
|
|
rateCol.Add(internalpb.RateType_DMLUpsert.String(), float64(it.upsertMsg.InsertMsg.Size()+it.upsertMsg.DeleteMsg.Size()))
|
|
if merr.Ok(it.result.GetStatus()) {
|
|
metrics.ProxyReportValue.WithLabelValues(nodeID, hookutil.OpTypeUpsert, dbName, username).Add(float64(v))
|
|
}
|
|
successCnt := it.result.UpsertCnt - int64(len(it.result.ErrIndex))
|
|
metrics.ProxyUpsertVectors.
|
|
WithLabelValues(nodeID, dbName, collectionName).
|
|
Add(float64(successCnt))
|
|
metrics.ProxyMutationLatency.
|
|
WithLabelValues(nodeID, metrics.UpsertLabel, dbName, collectionName).
|
|
Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
metrics.ProxyCollectionMutationLatency.WithLabelValues(nodeID, metrics.UpsertLabel, dbName, collectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
log.Debug("Finish processing upsert request in Proxy")
|
|
return it.result, nil
|
|
}
|
|
|
|
func GetCollectionRateSubLabel(req any) string {
|
|
dbName, _ := requestutil.GetDbNameFromRequest(req)
|
|
if dbName == "" {
|
|
return ""
|
|
}
|
|
collectionName, _ := requestutil.GetCollectionNameFromRequest(req)
|
|
if collectionName == "" {
|
|
return ""
|
|
}
|
|
return ratelimitutil.GetCollectionSubLabel(dbName.(string), collectionName.(string))
|
|
}
|
|
|
|
// Search searches the most similar records of requests.
|
|
func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error) {
|
|
var err error
|
|
rsp := &milvuspb.SearchResults{
|
|
Status: merr.Success(),
|
|
}
|
|
|
|
optimizedSearch := true
|
|
resultSizeInsufficient := false
|
|
isTopkReduce := false
|
|
isRecallEvaluation := false
|
|
err2 := retry.Handle(ctx, func() (bool, error) {
|
|
rsp, resultSizeInsufficient, isTopkReduce, isRecallEvaluation, err = node.search(ctx, request, optimizedSearch, false)
|
|
if merr.Ok(rsp.GetStatus()) && optimizedSearch && resultSizeInsufficient && isTopkReduce && paramtable.Get().AutoIndexConfig.EnableResultLimitCheck.GetAsBool() {
|
|
// without optimize search
|
|
optimizedSearch = false
|
|
rsp, resultSizeInsufficient, isTopkReduce, isRecallEvaluation, err = node.search(ctx, request, optimizedSearch, false)
|
|
metrics.ProxyRetrySearchCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.SearchLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Inc()
|
|
// result size still insufficient
|
|
if resultSizeInsufficient {
|
|
metrics.ProxyRetrySearchResultInsufficientCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.SearchLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Inc()
|
|
}
|
|
}
|
|
if errors.Is(merr.Error(rsp.GetStatus()), merr.ErrInconsistentRequery) {
|
|
return true, merr.Error(rsp.GetStatus())
|
|
}
|
|
// search for ground truth and compute recall
|
|
if isRecallEvaluation && merr.Ok(rsp.GetStatus()) {
|
|
var rspGT *milvuspb.SearchResults
|
|
rspGT, _, _, _, err = node.search(ctx, request, false, true)
|
|
metrics.ProxyRecallSearchCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.SearchLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Inc()
|
|
if merr.Ok(rspGT.GetStatus()) {
|
|
return false, computeRecall(rsp.GetResults(), rspGT.GetResults())
|
|
}
|
|
if errors.Is(merr.Error(rspGT.GetStatus()), merr.ErrInconsistentRequery) {
|
|
return true, merr.Error(rspGT.GetStatus())
|
|
}
|
|
return false, merr.Error(rspGT.GetStatus())
|
|
}
|
|
return false, nil
|
|
})
|
|
if err2 != nil {
|
|
rsp.Status = merr.Status(err2)
|
|
} else if err != nil {
|
|
rsp.Status = merr.Status(err)
|
|
}
|
|
if err != nil {
|
|
rsp.Status = merr.Status(err)
|
|
}
|
|
return rsp, nil
|
|
}
|
|
|
|
func (node *Proxy) search(ctx context.Context, request *milvuspb.SearchRequest, optimizedSearch bool, isRecallEvaluation bool) (*milvuspb.SearchResults, bool, bool, bool, error) {
|
|
metrics.GetStats(ctx).
|
|
SetNodeID(paramtable.GetNodeID()).
|
|
SetInboundLabel(metrics.SearchLabel).
|
|
SetDatabaseName(request.GetDbName()).
|
|
SetCollectionName(request.GetCollectionName())
|
|
|
|
metrics.ProxyReceivedNQ.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.SearchLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Add(float64(request.GetNq()))
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.SearchResults{
|
|
Status: merr.Status(err),
|
|
}, false, false, false, nil
|
|
}
|
|
|
|
method := "Search"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Search")
|
|
defer sp.End()
|
|
|
|
if request.SearchByPrimaryKeys {
|
|
placeholderGroupBytes, err := node.getVectorPlaceholderGroupForSearchByPks(ctx, request)
|
|
if err != nil {
|
|
return &milvuspb.SearchResults{
|
|
Status: merr.Status(err),
|
|
}, false, false, false, nil
|
|
}
|
|
|
|
request.PlaceholderGroup = placeholderGroupBytes
|
|
}
|
|
|
|
qt := &searchTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
SearchRequest: &internalpb.SearchRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_Search),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
),
|
|
ReqID: paramtable.GetNodeID(),
|
|
IsTopkReduce: optimizedSearch,
|
|
IsRecallEvaluation: isRecallEvaluation,
|
|
},
|
|
request: request,
|
|
tr: timerecord.NewTimeRecorder("search"),
|
|
mixCoord: node.mixCoord,
|
|
node: node,
|
|
lb: node.lbPolicy,
|
|
shardClientMgr: node.shardMgr,
|
|
enableMaterializedView: node.enableMaterializedView,
|
|
mustUsePartitionKey: Params.ProxyCfg.MustUsePartitionKey.GetAsBool(),
|
|
}
|
|
|
|
log := log.Ctx(ctx).With( // TODO: it might cause some cpu consumption
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Strings("partitions", request.PartitionNames),
|
|
zap.String("dsl", request.Dsl),
|
|
zap.Int("len(PlaceholderGroup)", len(request.PlaceholderGroup)),
|
|
zap.Strings("OutputFields", request.OutputFields),
|
|
zap.Any("search_params", request.SearchParams),
|
|
zap.String("ConsistencyLevel", request.GetConsistencyLevel().String()),
|
|
zap.Bool("useDefaultConsistency", request.GetUseDefaultConsistency()),
|
|
)
|
|
|
|
defer func() {
|
|
span := tr.ElapseSpan()
|
|
spanPerNq := span
|
|
if qt.SearchRequest.GetNq() > 0 {
|
|
spanPerNq = span / time.Duration(qt.SearchRequest.GetNq())
|
|
}
|
|
if spanPerNq >= paramtable.Get().ProxyCfg.SlowLogSpanInSeconds.GetAsDuration(time.Second) {
|
|
log.Info(rpcSlow(method), zap.Uint64("guarantee_timestamp", qt.GetGuaranteeTimestamp()),
|
|
zap.Int64("nq", qt.SearchRequest.GetNq()), zap.Duration("duration", span), zap.Duration("durationPerNq", spanPerNq))
|
|
// WebUI slow query shall use slow log as well.
|
|
user, _ := GetCurUserFromContext(ctx)
|
|
traceID := ""
|
|
if sp != nil {
|
|
traceID = sp.SpanContext().TraceID().String()
|
|
}
|
|
if node.slowQueries != nil {
|
|
node.slowQueries.Add(qt.BeginTs(), metricsinfo.NewSlowQueryWithSearchRequest(request, user, span, traceID))
|
|
}
|
|
}
|
|
if span >= paramtable.Get().ProxyCfg.SlowQuerySpanInSeconds.GetAsDuration(time.Second) {
|
|
metrics.ProxySlowQueryCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.SearchLabel,
|
|
).Inc()
|
|
}
|
|
}()
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.dqQueue.Enqueue(qt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err),
|
|
)
|
|
|
|
return &milvuspb.SearchResults{
|
|
Status: merr.Status(err),
|
|
}, false, false, false, nil
|
|
}
|
|
tr.CtxRecord(ctx, "search request enqueue")
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("timestamp", qt.Base.Timestamp),
|
|
)
|
|
|
|
if err := qt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Int64("nq", qt.SearchRequest.GetNq()),
|
|
zap.Error(err),
|
|
)
|
|
|
|
return &milvuspb.SearchResults{
|
|
Status: merr.Status(err),
|
|
}, false, false, false, nil
|
|
}
|
|
|
|
span := tr.CtxRecord(ctx, "wait search result")
|
|
nodeID := paramtable.GetStringNodeID()
|
|
dbName := request.DbName
|
|
collectionName := request.CollectionName
|
|
metrics.ProxyWaitForSearchResultLatency.WithLabelValues(
|
|
nodeID,
|
|
metrics.SearchLabel,
|
|
).Observe(float64(span.Milliseconds()))
|
|
|
|
tr.CtxRecord(ctx, "wait search result")
|
|
log.Debug(rpcDone(method))
|
|
|
|
metrics.ProxySearchVectors.
|
|
WithLabelValues(nodeID, dbName, collectionName).
|
|
Add(float64(qt.result.GetResults().GetNumQueries()))
|
|
|
|
searchDur := tr.ElapseSpan().Milliseconds()
|
|
metrics.ProxySQLatency.WithLabelValues(
|
|
nodeID,
|
|
metrics.SearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Observe(float64(searchDur))
|
|
|
|
metrics.ProxyCollectionSQLatency.WithLabelValues(
|
|
nodeID,
|
|
metrics.SearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Observe(float64(searchDur))
|
|
|
|
if Params.QueryNodeCfg.StorageUsageTrackingEnabled.GetAsBool() {
|
|
metrics.ProxyScannedRemoteMB.WithLabelValues(
|
|
nodeID,
|
|
metrics.SearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Add(float64(qt.storageCost.ScannedRemoteBytes) / 1024 / 1024)
|
|
|
|
metrics.ProxyScannedTotalMB.WithLabelValues(
|
|
nodeID,
|
|
metrics.SearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Add(float64(qt.storageCost.ScannedTotalBytes) / 1024 / 1024)
|
|
}
|
|
|
|
if qt.result != nil {
|
|
username := GetCurUserFromContextOrDefault(ctx)
|
|
sentSize := proto.Size(qt.result)
|
|
v := hookutil.GetExtension().Report(map[string]any{
|
|
hookutil.OpTypeKey: hookutil.OpTypeSearch,
|
|
hookutil.DatabaseKey: dbName,
|
|
hookutil.UsernameKey: username,
|
|
hookutil.ResultDataSizeKey: sentSize,
|
|
hookutil.RelatedDataSizeKey: qt.relatedDataSize,
|
|
hookutil.RelatedCntKey: qt.result.GetResults().GetAllSearchCount(),
|
|
})
|
|
SetReportValue(qt.result.GetStatus(), v)
|
|
SetStorageCost(qt.result.GetStatus(), qt.storageCost)
|
|
if merr.Ok(qt.result.GetStatus()) {
|
|
metrics.ProxyReportValue.WithLabelValues(nodeID, hookutil.OpTypeSearch, dbName, username).Add(float64(v))
|
|
}
|
|
}
|
|
return qt.result, qt.resultSizeInsufficient, qt.isTopkReduce, qt.isRecallEvaluation, nil
|
|
}
|
|
|
|
func (node *Proxy) HybridSearch(ctx context.Context, request *milvuspb.HybridSearchRequest) (*milvuspb.SearchResults, error) {
|
|
var err error
|
|
rsp := &milvuspb.SearchResults{
|
|
Status: merr.Success(),
|
|
}
|
|
optimizedSearch := true
|
|
resultSizeInsufficient := false
|
|
isTopkReduce := false
|
|
err2 := retry.Handle(ctx, func() (bool, error) {
|
|
rsp, resultSizeInsufficient, isTopkReduce, err = node.hybridSearch(ctx, request, optimizedSearch)
|
|
if merr.Ok(rsp.GetStatus()) && optimizedSearch && resultSizeInsufficient && isTopkReduce && paramtable.Get().AutoIndexConfig.EnableResultLimitCheck.GetAsBool() {
|
|
// without optimize search
|
|
optimizedSearch = false
|
|
rsp, resultSizeInsufficient, isTopkReduce, err = node.hybridSearch(ctx, request, optimizedSearch)
|
|
metrics.ProxyRetrySearchCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.HybridSearchLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Inc()
|
|
// result size still insufficient
|
|
if resultSizeInsufficient {
|
|
metrics.ProxyRetrySearchResultInsufficientCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.HybridSearchLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Inc()
|
|
}
|
|
}
|
|
if errors.Is(merr.Error(rsp.GetStatus()), merr.ErrInconsistentRequery) {
|
|
return true, merr.Error(rsp.GetStatus())
|
|
}
|
|
return false, nil
|
|
})
|
|
if err2 != nil {
|
|
rsp.Status = merr.Status(err2)
|
|
}
|
|
return rsp, err
|
|
}
|
|
|
|
type hybridSearchRequestExprLogger struct {
|
|
*milvuspb.HybridSearchRequest
|
|
}
|
|
|
|
// String implements Stringer interface for lazy logging.
|
|
func (l *hybridSearchRequestExprLogger) String() string {
|
|
builder := &strings.Builder{}
|
|
|
|
for idx, subReq := range l.Requests {
|
|
builder.WriteString(fmt.Sprintf("[No.%d req, expr: %s]", idx, subReq.GetDsl()))
|
|
}
|
|
|
|
return builder.String()
|
|
}
|
|
|
|
func (node *Proxy) hybridSearch(ctx context.Context, request *milvuspb.HybridSearchRequest, optimizedSearch bool) (*milvuspb.SearchResults, bool, bool, error) {
|
|
metrics.GetStats(ctx).
|
|
SetNodeID(paramtable.GetNodeID()).
|
|
SetInboundLabel(metrics.HybridSearchLabel).
|
|
SetDatabaseName(request.GetDbName()).
|
|
SetCollectionName(request.GetCollectionName())
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.SearchResults{
|
|
Status: merr.Status(err),
|
|
}, false, false, nil
|
|
}
|
|
|
|
method := "HybridSearch"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-HybridSearch")
|
|
defer sp.End()
|
|
newSearchReq := convertHybridSearchToSearch(request)
|
|
qt := &searchTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
SearchRequest: &internalpb.SearchRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_Search),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
),
|
|
ReqID: paramtable.GetNodeID(),
|
|
IsTopkReduce: optimizedSearch,
|
|
},
|
|
request: newSearchReq,
|
|
tr: timerecord.NewTimeRecorder(method),
|
|
mixCoord: node.mixCoord,
|
|
node: node,
|
|
lb: node.lbPolicy,
|
|
shardClientMgr: node.shardMgr,
|
|
mustUsePartitionKey: Params.ProxyCfg.MustUsePartitionKey.GetAsBool(),
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Any("partitions", request.PartitionNames),
|
|
zap.Any("OutputFields", request.OutputFields),
|
|
zap.String("ConsistencyLevel", request.GetConsistencyLevel().String()),
|
|
zap.Bool("useDefaultConsistency", request.GetUseDefaultConsistency()),
|
|
zap.Stringer("dsls", &hybridSearchRequestExprLogger{HybridSearchRequest: request}),
|
|
)
|
|
|
|
defer func() {
|
|
span := tr.ElapseSpan()
|
|
if span >= paramtable.Get().ProxyCfg.SlowLogSpanInSeconds.GetAsDuration(time.Second) {
|
|
log.Info(rpcSlow(method), zap.Uint64("guarantee_timestamp", qt.GetGuaranteeTimestamp()), zap.Duration("duration", span))
|
|
// WebUI slow query shall use slow log as well.
|
|
user, _ := GetCurUserFromContext(ctx)
|
|
traceID := ""
|
|
if sp != nil {
|
|
traceID = sp.SpanContext().TraceID().String()
|
|
}
|
|
if node.slowQueries != nil {
|
|
node.slowQueries.Add(qt.BeginTs(), metricsinfo.NewSlowQueryWithSearchRequest(newSearchReq, user, span, traceID))
|
|
}
|
|
}
|
|
if span >= paramtable.Get().ProxyCfg.SlowQuerySpanInSeconds.GetAsDuration(time.Second) {
|
|
metrics.ProxySlowQueryCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.HybridSearchLabel,
|
|
).Inc()
|
|
}
|
|
}()
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.dqQueue.Enqueue(qt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err),
|
|
)
|
|
|
|
return &milvuspb.SearchResults{
|
|
Status: merr.Status(err),
|
|
}, false, false, nil
|
|
}
|
|
tr.CtxRecord(ctx, "hybrid search request enqueue")
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("timestamp", qt.Base.Timestamp),
|
|
)
|
|
|
|
if err := qt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
)
|
|
|
|
return &milvuspb.SearchResults{
|
|
Status: merr.Status(err),
|
|
}, false, false, nil
|
|
}
|
|
|
|
span := tr.CtxRecord(ctx, "wait hybrid search result")
|
|
nodeID := paramtable.GetStringNodeID()
|
|
dbName := request.DbName
|
|
collectionName := request.CollectionName
|
|
metrics.ProxyWaitForSearchResultLatency.WithLabelValues(
|
|
nodeID,
|
|
metrics.HybridSearchLabel,
|
|
).Observe(float64(span.Milliseconds()))
|
|
|
|
tr.CtxRecord(ctx, "wait hybrid search result")
|
|
log.Debug(rpcDone(method))
|
|
|
|
metrics.ProxySearchVectors.
|
|
WithLabelValues(nodeID, dbName, collectionName).
|
|
Add(float64(len(request.GetRequests()) * int(qt.SearchRequest.GetNq())))
|
|
|
|
searchDur := tr.ElapseSpan().Milliseconds()
|
|
metrics.ProxySQLatency.WithLabelValues(
|
|
nodeID,
|
|
metrics.HybridSearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Observe(float64(searchDur))
|
|
|
|
metrics.ProxyCollectionSQLatency.WithLabelValues(
|
|
nodeID,
|
|
metrics.HybridSearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Observe(float64(searchDur))
|
|
|
|
if Params.QueryNodeCfg.StorageUsageTrackingEnabled.GetAsBool() {
|
|
metrics.ProxyScannedRemoteMB.WithLabelValues(
|
|
nodeID,
|
|
metrics.HybridSearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Add(float64(qt.storageCost.ScannedRemoteBytes) / 1024 / 1024)
|
|
|
|
metrics.ProxyScannedTotalMB.WithLabelValues(
|
|
nodeID,
|
|
metrics.HybridSearchLabel,
|
|
dbName,
|
|
collectionName,
|
|
).Add(float64(qt.storageCost.ScannedTotalBytes) / 1024 / 1024)
|
|
}
|
|
|
|
if qt.result != nil {
|
|
sentSize := proto.Size(qt.result)
|
|
username := GetCurUserFromContextOrDefault(ctx)
|
|
v := hookutil.GetExtension().Report(map[string]any{
|
|
hookutil.OpTypeKey: hookutil.OpTypeHybridSearch,
|
|
hookutil.DatabaseKey: dbName,
|
|
hookutil.UsernameKey: username,
|
|
hookutil.ResultDataSizeKey: sentSize,
|
|
hookutil.RelatedDataSizeKey: qt.relatedDataSize,
|
|
hookutil.RelatedCntKey: qt.result.GetResults().GetAllSearchCount(),
|
|
})
|
|
SetReportValue(qt.result.GetStatus(), v)
|
|
SetStorageCost(qt.result.GetStatus(), qt.storageCost)
|
|
if merr.Ok(qt.result.GetStatus()) {
|
|
metrics.ProxyReportValue.WithLabelValues(nodeID, hookutil.OpTypeHybridSearch, dbName, username).Add(float64(v))
|
|
}
|
|
}
|
|
return qt.result, qt.resultSizeInsufficient, qt.isTopkReduce, nil
|
|
}
|
|
|
|
func (node *Proxy) getVectorPlaceholderGroupForSearchByPks(ctx context.Context, request *milvuspb.SearchRequest) ([]byte, error) {
|
|
placeholderGroup := &commonpb.PlaceholderGroup{}
|
|
err := proto.Unmarshal(request.PlaceholderGroup, placeholderGroup)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if len(placeholderGroup.Placeholders) != 1 || len(placeholderGroup.Placeholders[0].Values) != 1 {
|
|
return nil, merr.WrapErrParameterInvalidMsg("please provide primary key")
|
|
}
|
|
queryExpr := string(placeholderGroup.Placeholders[0].Values[0])
|
|
|
|
annsField, err := funcutil.GetAttrByKeyFromRepeatedKV(AnnsFieldKey, request.SearchParams)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
queryRequest := &milvuspb.QueryRequest{
|
|
Base: request.Base,
|
|
DbName: request.DbName,
|
|
CollectionName: request.CollectionName,
|
|
Expr: queryExpr,
|
|
OutputFields: []string{annsField},
|
|
PartitionNames: request.PartitionNames,
|
|
TravelTimestamp: request.TravelTimestamp,
|
|
GuaranteeTimestamp: request.GuaranteeTimestamp,
|
|
QueryParams: nil,
|
|
NotReturnAllMeta: request.NotReturnAllMeta,
|
|
ConsistencyLevel: request.ConsistencyLevel,
|
|
UseDefaultConsistency: request.UseDefaultConsistency,
|
|
}
|
|
|
|
queryResults, _ := node.Query(ctx, queryRequest)
|
|
|
|
err = merr.Error(queryResults.GetStatus())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
var vectorFieldsData *schemapb.FieldData
|
|
for _, fieldsData := range queryResults.GetFieldsData() {
|
|
if fieldsData.GetFieldName() == annsField {
|
|
vectorFieldsData = fieldsData
|
|
break
|
|
}
|
|
}
|
|
|
|
placeholderGroupBytes, err := funcutil.FieldDataToPlaceholderGroupBytes(vectorFieldsData)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return placeholderGroupBytes, nil
|
|
}
|
|
|
|
// Flush notify data nodes to persist the data of collection.
|
|
func (node *Proxy) Flush(ctx context.Context, request *milvuspb.FlushRequest) (*milvuspb.FlushResponse, error) {
|
|
resp := &milvuspb.FlushResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Flush")
|
|
defer sp.End()
|
|
|
|
ft := &flushTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
FlushRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
chMgr: node.chMgr,
|
|
}
|
|
|
|
method := "Flush"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.Any("collections", request.CollectionNames))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
if err := node.sched.dcQueue.Enqueue(ft); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", ft.BeginTs()),
|
|
zap.Uint64("EndTs", ft.EndTs()))
|
|
|
|
if err := ft.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", ft.BeginTs()),
|
|
zap.Uint64("EndTs", ft.EndTs()))
|
|
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", ft.BeginTs()),
|
|
zap.Uint64("EndTs", ft.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return ft.result, nil
|
|
}
|
|
|
|
// Query get the records by primary keys.
|
|
func (node *Proxy) query(ctx context.Context, qt *queryTask, sp trace.Span) (*milvuspb.QueryResults, segcore.StorageCost, error) {
|
|
request := qt.request
|
|
method := "Query"
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.QueryResults{
|
|
Status: merr.Status(err),
|
|
}, segcore.StorageCost{}, nil
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("collection", request.CollectionName),
|
|
zap.Strings("partitions", request.PartitionNames),
|
|
zap.String("ConsistencyLevel", request.GetConsistencyLevel().String()),
|
|
zap.Bool("useDefaultConsistency", request.GetUseDefaultConsistency()),
|
|
)
|
|
|
|
log.Debug(
|
|
rpcReceived(method),
|
|
zap.String("expr", request.Expr),
|
|
zap.Strings("OutputFields", request.OutputFields),
|
|
zap.Uint64("travel_timestamp", request.TravelTimestamp),
|
|
)
|
|
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
defer func() {
|
|
span := tr.ElapseSpan()
|
|
if span >= paramtable.Get().ProxyCfg.SlowLogSpanInSeconds.GetAsDuration(time.Second) {
|
|
log.Info(
|
|
rpcSlow(method),
|
|
zap.String("expr", request.Expr),
|
|
zap.Strings("OutputFields", request.OutputFields),
|
|
zap.Uint64("travel_timestamp", request.TravelTimestamp),
|
|
zap.Uint64("guarantee_timestamp", qt.GetGuaranteeTimestamp()),
|
|
zap.Duration("duration", span))
|
|
// WebUI slow query shall use slow log as well.
|
|
user, _ := GetCurUserFromContext(ctx)
|
|
traceID := ""
|
|
if sp != nil {
|
|
traceID = sp.SpanContext().TraceID().String()
|
|
}
|
|
|
|
if node.slowQueries != nil {
|
|
node.slowQueries.Add(qt.BeginTs(), metricsinfo.NewSlowQueryWithQueryRequest(request, user, span, traceID))
|
|
}
|
|
}
|
|
if span >= paramtable.Get().ProxyCfg.SlowQuerySpanInSeconds.GetAsDuration(time.Second) {
|
|
metrics.ProxySlowQueryCount.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.QueryLabel,
|
|
).Inc()
|
|
}
|
|
}()
|
|
|
|
if err := node.sched.dqQueue.Enqueue(qt); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err),
|
|
)
|
|
|
|
return &milvuspb.QueryResults{
|
|
Status: merr.Status(err),
|
|
}, segcore.StorageCost{}, nil
|
|
}
|
|
tr.CtxRecord(ctx, "query request enqueue")
|
|
|
|
log.Debug(rpcEnqueued(method))
|
|
|
|
if err := qt.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.QueryResults{
|
|
Status: merr.Status(err),
|
|
}, segcore.StorageCost{}, nil
|
|
}
|
|
|
|
if !qt.reQuery {
|
|
span := tr.CtxRecord(ctx, "wait query result")
|
|
metrics.ProxyWaitForSearchResultLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.QueryLabel,
|
|
).Observe(float64(span.Milliseconds()))
|
|
|
|
metrics.ProxySQLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.QueryLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
metrics.ProxyCollectionSQLatency.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.QueryLabel,
|
|
request.DbName,
|
|
request.CollectionName,
|
|
).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
}
|
|
|
|
return qt.result, qt.storageCost, nil
|
|
}
|
|
|
|
// Query get the records by primary keys.
|
|
func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (*milvuspb.QueryResults, error) {
|
|
qt := &queryTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
RetrieveRequest: &internalpb.RetrieveRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_Retrieve),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
),
|
|
ReqID: paramtable.GetNodeID(),
|
|
ConsistencyLevel: request.ConsistencyLevel,
|
|
},
|
|
request: request,
|
|
mixCoord: node.mixCoord,
|
|
lb: node.lbPolicy,
|
|
shardclientMgr: node.shardMgr,
|
|
mustUsePartitionKey: Params.ProxyCfg.MustUsePartitionKey.GetAsBool(),
|
|
}
|
|
|
|
subLabel := GetCollectionRateSubLabel(request)
|
|
metrics.GetStats(ctx).
|
|
SetNodeID(paramtable.GetNodeID()).
|
|
SetInboundLabel(metrics.QueryLabel).
|
|
SetDatabaseName(request.GetDbName()).
|
|
SetCollectionName(request.GetCollectionName())
|
|
metrics.ProxyReceivedNQ.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.QueryLabel,
|
|
request.GetDbName(),
|
|
request.GetCollectionName(),
|
|
).Add(float64(1))
|
|
|
|
rateCol.Add(internalpb.RateType_DQLQuery.String(), 1, subLabel)
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.QueryResults{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Query")
|
|
defer sp.End()
|
|
method := "Query"
|
|
|
|
res, storageCost, err := node.query(ctx, qt, sp)
|
|
|
|
if Params.QueryNodeCfg.StorageUsageTrackingEnabled.GetAsBool() {
|
|
metrics.ProxyScannedRemoteMB.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.QueryLabel,
|
|
request.DbName,
|
|
request.CollectionName,
|
|
).Add(float64(qt.storageCost.ScannedRemoteBytes) / 1024 / 1024)
|
|
|
|
metrics.ProxyScannedTotalMB.WithLabelValues(
|
|
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
|
metrics.QueryLabel,
|
|
request.DbName,
|
|
request.CollectionName,
|
|
).Add(float64(qt.storageCost.ScannedTotalBytes) / 1024 / 1024)
|
|
}
|
|
|
|
if err != nil || !merr.Ok(res.Status) {
|
|
return res, err
|
|
}
|
|
|
|
log.Ctx(ctx).Debug(rpcDone(method))
|
|
|
|
username := GetCurUserFromContextOrDefault(ctx)
|
|
nodeID := paramtable.GetStringNodeID()
|
|
v := hookutil.GetExtension().Report(map[string]any{
|
|
hookutil.OpTypeKey: hookutil.OpTypeQuery,
|
|
hookutil.DatabaseKey: request.DbName,
|
|
hookutil.UsernameKey: username,
|
|
hookutil.ResultDataSizeKey: proto.Size(res),
|
|
hookutil.RelatedDataSizeKey: qt.totalRelatedDataSize,
|
|
hookutil.RelatedCntKey: qt.allQueryCnt,
|
|
})
|
|
SetReportValue(res.Status, v)
|
|
SetStorageCost(res.Status, storageCost)
|
|
metrics.ProxyReportValue.WithLabelValues(nodeID, hookutil.OpTypeQuery, request.DbName, username).Add(float64(v))
|
|
|
|
if log.Ctx(ctx).Core().Enabled(zap.DebugLevel) && matchCountRule(request.GetOutputFields()) {
|
|
r, _ := protojson.Marshal(res)
|
|
log.Ctx(ctx).Debug("Count result", zap.String("result", string(r)))
|
|
}
|
|
return res, nil
|
|
}
|
|
|
|
// CreateAlias create alias for collection, then you can search the collection with alias.
|
|
func (node *Proxy) CreateAlias(ctx context.Context, request *milvuspb.CreateAliasRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateAlias")
|
|
defer sp.End()
|
|
|
|
cat := &CreateAliasTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
CreateAliasRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "CreateAlias"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("alias", request.Alias),
|
|
zap.String("collection", request.CollectionName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(cat); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", cat.BeginTs()),
|
|
zap.Uint64("EndTs", cat.EndTs()))
|
|
|
|
if err := cat.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", cat.BeginTs()),
|
|
zap.Uint64("EndTs", cat.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", cat.BeginTs()),
|
|
zap.Uint64("EndTs", cat.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return cat.result, nil
|
|
}
|
|
|
|
// DescribeAlias describe alias of collection.
|
|
func (node *Proxy) DescribeAlias(ctx context.Context, request *milvuspb.DescribeAliasRequest) (*milvuspb.DescribeAliasResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.DescribeAliasResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DescribeAlias")
|
|
defer sp.End()
|
|
|
|
dat := &DescribeAliasTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
nodeID: node.session.ServerID,
|
|
DescribeAliasRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "DescribeAlias"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("alias", request.Alias))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dat); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.DescribeAliasResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", dat.BeginTs()),
|
|
zap.Uint64("EndTs", dat.EndTs()))
|
|
|
|
if err := dat.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method), zap.Uint64("BeginTs", dat.BeginTs()), zap.Uint64("EndTs", dat.EndTs()), zap.Error(err))
|
|
return &milvuspb.DescribeAliasResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", dat.BeginTs()),
|
|
zap.Uint64("EndTs", dat.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dat.result, nil
|
|
}
|
|
|
|
// ListAliases show all aliases of db.
|
|
func (node *Proxy) ListAliases(ctx context.Context, request *milvuspb.ListAliasesRequest) (*milvuspb.ListAliasesResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ListAliasesResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ListAliases")
|
|
defer sp.End()
|
|
|
|
lat := &ListAliasesTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
nodeID: node.session.ServerID,
|
|
ListAliasesRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "ListAliases"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(lat); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.ListAliasesResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", lat.BeginTs()),
|
|
zap.Uint64("EndTs", lat.EndTs()))
|
|
|
|
if err := lat.WaitToFinish(); err != nil {
|
|
log.Warn(rpcFailedToWaitToFinish(method), zap.Uint64("BeginTs", lat.BeginTs()), zap.Uint64("EndTs", lat.EndTs()), zap.Error(err))
|
|
return &milvuspb.ListAliasesResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", lat.BeginTs()),
|
|
zap.Uint64("EndTs", lat.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(node.session.ServerID, 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return lat.result, nil
|
|
}
|
|
|
|
// DropAlias alter the alias of collection.
|
|
func (node *Proxy) DropAlias(ctx context.Context, request *milvuspb.DropAliasRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropAlias")
|
|
defer sp.End()
|
|
|
|
dat := &DropAliasTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DropAliasRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "DropAlias"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("alias", request.Alias))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(dat); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", dat.BeginTs()),
|
|
zap.Uint64("EndTs", dat.EndTs()))
|
|
|
|
if err := dat.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", dat.BeginTs()),
|
|
zap.Uint64("EndTs", dat.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", dat.BeginTs()),
|
|
zap.Uint64("EndTs", dat.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return dat.result, nil
|
|
}
|
|
|
|
// AlterAlias alter alias of collection.
|
|
func (node *Proxy) AlterAlias(ctx context.Context, request *milvuspb.AlterAliasRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-AlterAlias")
|
|
defer sp.End()
|
|
|
|
aat := &AlterAliasTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
AlterAliasRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
method := "AlterAlias"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName),
|
|
zap.String("alias", request.Alias),
|
|
zap.String("collection", request.CollectionName))
|
|
|
|
log.Info(rpcReceived(method))
|
|
|
|
if err := node.sched.ddQueue.Enqueue(aat); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", aat.BeginTs()),
|
|
zap.Uint64("EndTs", aat.EndTs()))
|
|
|
|
if err := aat.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", aat.BeginTs()),
|
|
zap.Uint64("EndTs", aat.EndTs()))
|
|
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcDone(method),
|
|
zap.Uint64("BeginTs", aat.BeginTs()),
|
|
zap.Uint64("EndTs", aat.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return aat.result, nil
|
|
}
|
|
|
|
// CalcDistance calculates the distances between vectors.
|
|
func (node *Proxy) CalcDistance(ctx context.Context, request *milvuspb.CalcDistanceRequest) (*milvuspb.CalcDistanceResults, error) {
|
|
return &milvuspb.CalcDistanceResults{
|
|
Status: merr.Status(merr.WrapErrServiceUnavailable("CalcDistance deprecated")),
|
|
}, nil
|
|
}
|
|
|
|
// Flush notify data nodes to persist the data of collection.
|
|
func (node *Proxy) FlushAll(ctx context.Context, request *milvuspb.FlushAllRequest) (*milvuspb.FlushAllResponse, error) {
|
|
resp := &milvuspb.FlushAllResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-FlushAll")
|
|
defer sp.End()
|
|
|
|
ft := &flushAllTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
FlushAllRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
chMgr: node.chMgr,
|
|
}
|
|
|
|
method := "FlushAll"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", request.DbName))
|
|
|
|
log.Debug(rpcReceived(method))
|
|
|
|
if err := node.sched.dcQueue.Enqueue(ft); err != nil {
|
|
log.Warn(rpcFailedToEnqueue(method), zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Debug(rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", ft.BeginTs()),
|
|
zap.Uint64("EndTs", ft.EndTs()))
|
|
|
|
if err := ft.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", ft.BeginTs()),
|
|
zap.Uint64("EndTs", ft.EndTs()))
|
|
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Debug(
|
|
rpcDone(method),
|
|
zap.Uint64("FlushAllTs", ft.result.GetFlushAllTs()),
|
|
zap.Uint64("BeginTs", ft.BeginTs()),
|
|
zap.Uint64("EndTs", ft.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return ft.result, nil
|
|
}
|
|
|
|
// GetDdChannel returns the used channel for dd operations.
|
|
func (node *Proxy) GetDdChannel(ctx context.Context, request *internalpb.GetDdChannelRequest) (*milvuspb.StringResponse, error) {
|
|
return &milvuspb.StringResponse{
|
|
Status: merr.Status(merr.WrapErrServiceUnavailable("unimp")),
|
|
}, nil
|
|
}
|
|
|
|
// GetPersistentSegmentInfo get the information of sealed segment.
|
|
func (node *Proxy) GetPersistentSegmentInfo(ctx context.Context, req *milvuspb.GetPersistentSegmentInfoRequest) (*milvuspb.GetPersistentSegmentInfoResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetPersistentSegmentInfo")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("GetPersistentSegmentInfo",
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", req.DbName),
|
|
zap.Any("collection", req.CollectionName))
|
|
|
|
resp := &milvuspb.GetPersistentSegmentInfoResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
method := "GetPersistentSegmentInfo"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
// list segments
|
|
collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
getSegmentsByStatesResponse, err := node.mixCoord.GetSegmentsByStates(ctx, &datapb.GetSegmentsByStatesRequest{
|
|
CollectionID: collectionID,
|
|
// -1 means list all partition segemnts
|
|
PartitionID: -1,
|
|
States: []commonpb.SegmentState{commonpb.SegmentState_Flushing, commonpb.SegmentState_Flushed, commonpb.SegmentState_Sealed},
|
|
})
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
// get Segment info
|
|
infoResp, err := node.mixCoord.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_SegmentInfo),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
),
|
|
SegmentIDs: getSegmentsByStatesResponse.Segments,
|
|
})
|
|
if err != nil {
|
|
log.Warn("GetPersistentSegmentInfo fail",
|
|
zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
err = merr.Error(infoResp.GetStatus())
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
log.Debug("GetPersistentSegmentInfo",
|
|
zap.Int("len(infos)", len(infoResp.Infos)),
|
|
zap.Any("status", infoResp.Status))
|
|
persistentInfos := make([]*milvuspb.PersistentSegmentInfo, len(infoResp.Infos))
|
|
for i, info := range infoResp.Infos {
|
|
persistentInfos[i] = &milvuspb.PersistentSegmentInfo{
|
|
SegmentID: info.ID,
|
|
CollectionID: info.CollectionID,
|
|
PartitionID: info.PartitionID,
|
|
NumRows: info.NumOfRows,
|
|
State: info.State,
|
|
Level: commonpb.SegmentLevel(info.Level),
|
|
IsSorted: info.GetIsSorted(),
|
|
StorageVersion: info.GetStorageVersion(),
|
|
}
|
|
}
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
resp.Infos = persistentInfos
|
|
return resp, nil
|
|
}
|
|
|
|
func (node *Proxy) GetSegmentsInfo(ctx context.Context, req *internalpb.GetSegmentsInfoRequest) (*internalpb.GetSegmentsInfoResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetSegmentsInfo")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
log.Debug("GetSegmentsInfo",
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", req.DbName),
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
|
zap.Int64s("segmentIDs", req.GetSegmentIDs()))
|
|
|
|
resp := &internalpb.GetSegmentsInfoResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
method := "GetSegmentsInfo"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
defer func() {
|
|
if resp.GetStatus().GetCode() != 0 {
|
|
log.Warn("GetSegmentsInfo failed", zap.String("err", resp.GetStatus().GetReason()))
|
|
}
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
}()
|
|
|
|
infoResp, err := node.mixCoord.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{
|
|
SegmentIDs: req.GetSegmentIDs(),
|
|
IncludeUnHealthy: true,
|
|
})
|
|
if err != nil {
|
|
log.Warn("GetSegmentInfo fail",
|
|
zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
err = merr.Error(infoResp.GetStatus())
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
log.Debug("GetPersistentSegmentInfo",
|
|
zap.Int("len(infos)", len(infoResp.Infos)),
|
|
zap.Any("status", infoResp.Status))
|
|
getLogIDs := func(binlogs []*datapb.FieldBinlog) []*internalpb.FieldBinlog {
|
|
logIDs := make([]*internalpb.FieldBinlog, 0, len(binlogs))
|
|
for _, fb := range binlogs {
|
|
fieldLogIDs := make([]int64, 0, len(fb.GetBinlogs()))
|
|
for _, b := range fb.GetBinlogs() {
|
|
fieldLogIDs = append(fieldLogIDs, b.GetLogID())
|
|
}
|
|
logIDs = append(logIDs, &internalpb.FieldBinlog{
|
|
FieldID: fb.GetFieldID(),
|
|
LogIDs: fieldLogIDs,
|
|
})
|
|
}
|
|
return logIDs
|
|
}
|
|
segmentInfos := make([]*internalpb.SegmentInfo, 0, len(req.GetSegmentIDs()))
|
|
for _, info := range infoResp.GetInfos() {
|
|
segmentInfos = append(segmentInfos, &internalpb.SegmentInfo{
|
|
SegmentID: info.GetID(),
|
|
CollectionID: info.GetCollectionID(),
|
|
PartitionID: info.GetPartitionID(),
|
|
VChannel: info.GetInsertChannel(),
|
|
NumRows: info.GetNumOfRows(),
|
|
State: info.GetState(),
|
|
Level: commonpb.SegmentLevel(info.GetLevel()),
|
|
IsSorted: info.GetIsSorted(),
|
|
InsertLogs: getLogIDs(info.GetBinlogs()),
|
|
DeltaLogs: getLogIDs(info.GetDeltalogs()),
|
|
StatsLogs: getLogIDs(info.GetStatslogs()),
|
|
})
|
|
}
|
|
|
|
resp.SegmentInfos = segmentInfos
|
|
return resp, nil
|
|
}
|
|
|
|
// GetQuerySegmentInfo gets segment information from QueryCoord.
|
|
func (node *Proxy) GetQuerySegmentInfo(ctx context.Context, req *milvuspb.GetQuerySegmentInfoRequest) (*milvuspb.GetQuerySegmentInfoResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetQuerySegmentInfo")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("GetQuerySegmentInfo",
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("db", req.DbName),
|
|
zap.Any("collection", req.CollectionName))
|
|
|
|
resp := &milvuspb.GetQuerySegmentInfoResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
method := "GetQuerySegmentInfo"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
|
|
collID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.CollectionName)
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
infoResp, err := node.mixCoord.GetLoadSegmentInfo(ctx, &querypb.GetSegmentInfoRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_SegmentInfo),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
),
|
|
CollectionID: collID,
|
|
})
|
|
if err == nil {
|
|
err = merr.Error(infoResp.GetStatus())
|
|
}
|
|
if err != nil {
|
|
log.Error("Failed to get segment info from QueryCoord",
|
|
zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
log.Debug("GetQuerySegmentInfo",
|
|
zap.Any("infos", infoResp.Infos),
|
|
zap.Any("status", infoResp.Status))
|
|
queryInfos := make([]*milvuspb.QuerySegmentInfo, len(infoResp.Infos))
|
|
for i, info := range infoResp.Infos {
|
|
queryInfos[i] = &milvuspb.QuerySegmentInfo{
|
|
SegmentID: info.SegmentID,
|
|
CollectionID: info.CollectionID,
|
|
PartitionID: info.PartitionID,
|
|
NumRows: info.NumRows,
|
|
MemSize: info.MemSize,
|
|
IndexName: info.IndexName,
|
|
IndexID: info.IndexID,
|
|
State: info.SegmentState,
|
|
NodeIds: info.NodeIds,
|
|
Level: commonpb.SegmentLevel(info.Level),
|
|
IsSorted: info.GetIsSorted(),
|
|
StorageVersion: info.GetStorageVersion(),
|
|
}
|
|
}
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
resp.Infos = queryInfos
|
|
return resp, nil
|
|
}
|
|
|
|
// Dummy handles dummy request
|
|
func (node *Proxy) Dummy(ctx context.Context, req *milvuspb.DummyRequest) (*milvuspb.DummyResponse, error) {
|
|
failedResponse := &milvuspb.DummyResponse{
|
|
Response: `{"status": "fail"}`,
|
|
}
|
|
|
|
// TODO(wxyu): change name RequestType to Request
|
|
drt, err := parseDummyRequestType(req.RequestType)
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Dummy")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
if err != nil {
|
|
log.Warn("Failed to parse dummy request type",
|
|
zap.Error(err))
|
|
return failedResponse, nil
|
|
}
|
|
|
|
if drt.RequestType == "query" {
|
|
drr, err := parseDummyQueryRequest(req.RequestType)
|
|
if err != nil {
|
|
log.Warn("Failed to parse dummy query request",
|
|
zap.Error(err))
|
|
return failedResponse, nil
|
|
}
|
|
|
|
request := &milvuspb.QueryRequest{
|
|
DbName: drr.DbName,
|
|
CollectionName: drr.CollectionName,
|
|
PartitionNames: drr.PartitionNames,
|
|
OutputFields: drr.OutputFields,
|
|
}
|
|
|
|
_, err = node.Query(ctx, request)
|
|
if err != nil {
|
|
log.Warn("Failed to execute dummy query",
|
|
zap.Error(err))
|
|
return failedResponse, err
|
|
}
|
|
|
|
return &milvuspb.DummyResponse{
|
|
Response: `{"status": "success"}`,
|
|
}, nil
|
|
}
|
|
|
|
log.Debug("cannot find specify dummy request type")
|
|
return failedResponse, nil
|
|
}
|
|
|
|
// RegisterLink registers a link
|
|
func (node *Proxy) RegisterLink(ctx context.Context, req *milvuspb.RegisterLinkRequest) (*milvuspb.RegisterLinkResponse, error) {
|
|
code := node.GetStateCode()
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-RegisterLink")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("state", code.String()))
|
|
|
|
log.Debug("RegisterLink")
|
|
|
|
if err := merr.CheckHealthy(code); err != nil {
|
|
return &milvuspb.RegisterLinkResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
// metrics.ProxyLinkedSDKs.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Inc()
|
|
return &milvuspb.RegisterLinkResponse{
|
|
Status: merr.Success(os.Getenv(metricsinfo.DeployModeEnvKey)),
|
|
}, nil
|
|
}
|
|
|
|
// GetMetrics gets the metrics of proxy
|
|
// TODO(dragondriver): cache the Metrics and set a retention to the cache
|
|
func (node *Proxy) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetMetrics")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.RatedDebug(60, "Proxy.GetMetrics",
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
|
zap.String("req", req.Request))
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
log.Warn("Proxy.GetMetrics failed",
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
|
zap.String("req", req.Request),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ret := gjson.Parse(req.GetRequest())
|
|
metricType, err := metricsinfo.ParseMetricRequestType(ret)
|
|
if err != nil {
|
|
log.Warn("Proxy.GetMetrics failed to parse metric type",
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
|
zap.String("req", req.Request),
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
req.Base = commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_SystemInfo),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
)
|
|
if metricType == metricsinfo.SystemInfoMetrics {
|
|
metrics, err := node.metricsCacheManager.GetSystemInfoMetrics()
|
|
if err != nil {
|
|
metrics, err = getSystemInfoMetrics(ctx, req, node)
|
|
}
|
|
|
|
log.RatedDebug(60, "Proxy.GetMetrics",
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
|
zap.String("req", req.Request),
|
|
zap.String("metricType", metricType),
|
|
zap.Any("metrics", metrics), // TODO(dragondriver): necessary? may be very large
|
|
zap.Error(err))
|
|
|
|
node.metricsCacheManager.UpdateSystemInfoMetrics(metrics)
|
|
|
|
return metrics, nil
|
|
}
|
|
|
|
log.RatedWarn(60, "Proxy.GetMetrics failed, request metric type is not implemented yet",
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
|
zap.String("req", req.Request),
|
|
zap.String("metricType", metricType))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
|
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
|
|
}, nil
|
|
}
|
|
|
|
// GetProxyMetrics gets the metrics of proxy, it's an internal interface which is different from GetMetrics interface,
|
|
// because it only obtains the metrics of Proxy, not including the topological metrics of Query cluster and Data cluster.
|
|
func (node *Proxy) GetProxyMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetProxyMetrics")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.Int64("nodeID", paramtable.GetNodeID()),
|
|
zap.String("req", req.Request))
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
log.Warn("Proxy.GetProxyMetrics failed",
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ret := gjson.Parse(req.GetRequest())
|
|
metricType, err := metricsinfo.ParseMetricRequestType(ret)
|
|
if err != nil {
|
|
log.Warn("Proxy.GetProxyMetrics failed to parse metric type",
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
req.Base = commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_SystemInfo),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
)
|
|
|
|
if metricType == metricsinfo.SystemInfoMetrics {
|
|
proxyMetrics, err := getProxyMetrics(ctx, req, node)
|
|
if err != nil {
|
|
log.Warn("Proxy.GetProxyMetrics failed to getProxyMetrics",
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
// log.Debug("Proxy.GetProxyMetrics",
|
|
// zap.String("metricType", metricType))
|
|
|
|
return proxyMetrics, nil
|
|
}
|
|
|
|
log.Warn("Proxy.GetProxyMetrics failed, request metric type is not implemented yet",
|
|
zap.String("metricType", metricType))
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
|
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
|
|
}, nil
|
|
}
|
|
|
|
// LoadBalance would do a load balancing operation between query nodes
|
|
func (node *Proxy) LoadBalance(ctx context.Context, req *milvuspb.LoadBalanceRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-LoadBalance")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("Proxy.LoadBalance",
|
|
zap.Int64("proxy_id", paramtable.GetNodeID()),
|
|
zap.Any("req", req))
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
status := merr.Success()
|
|
|
|
collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
|
|
if err != nil {
|
|
log.Warn("failed to get collection id",
|
|
zap.String("collectionName", req.GetCollectionName()),
|
|
zap.Error(err))
|
|
status = merr.Status(err)
|
|
return status, nil
|
|
}
|
|
infoResp, err := node.mixCoord.LoadBalance(ctx, &querypb.LoadBalanceRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_LoadBalanceSegments),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
),
|
|
SourceNodeIDs: []int64{req.SrcNodeID},
|
|
DstNodeIDs: req.DstNodeIDs,
|
|
BalanceReason: querypb.TriggerCondition_GrpcRequest,
|
|
SealedSegmentIDs: req.SealedSegmentIDs,
|
|
CollectionID: collectionID,
|
|
})
|
|
if err != nil {
|
|
log.Warn("Failed to LoadBalance from Query Coordinator",
|
|
zap.Any("req", req),
|
|
zap.Error(err))
|
|
status = merr.Status(err)
|
|
return status, nil
|
|
}
|
|
if infoResp.ErrorCode != commonpb.ErrorCode_Success {
|
|
log.Warn("Failed to LoadBalance from Query Coordinator",
|
|
zap.String("errMsg", infoResp.Reason))
|
|
status = infoResp
|
|
return status, nil
|
|
}
|
|
log.Debug("LoadBalance Done",
|
|
zap.Any("req", req),
|
|
zap.Any("status", infoResp))
|
|
return status, nil
|
|
}
|
|
|
|
// GetReplicas gets replica info
|
|
func (node *Proxy) GetReplicas(ctx context.Context, req *milvuspb.GetReplicasRequest) (*milvuspb.GetReplicasResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetReplicas")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("received get replicas request",
|
|
zap.Int64("collection", req.GetCollectionID()),
|
|
zap.Bool("with shard nodes", req.GetWithShardNodes()))
|
|
resp := &milvuspb.GetReplicasResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
req.Base = commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_GetReplicas),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
)
|
|
|
|
if req.GetCollectionName() != "" {
|
|
var err error
|
|
req.CollectionID, err = globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
}
|
|
|
|
r, err := node.mixCoord.GetReplicas(ctx, req)
|
|
if err != nil {
|
|
log.Warn("Failed to get replicas from Query Coordinator",
|
|
zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
log.Debug("received get replicas response", zap.String("resp", r.String()))
|
|
return r, nil
|
|
}
|
|
|
|
// GetCompactionState gets the compaction state of multiple segments
|
|
func (node *Proxy) GetCompactionState(ctx context.Context, req *milvuspb.GetCompactionStateRequest) (*milvuspb.GetCompactionStateResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetCompactionState")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.Int64("compactionID", req.GetCompactionID()))
|
|
|
|
log.Debug("received GetCompactionState request")
|
|
resp := &milvuspb.GetCompactionStateResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
resp, err := node.mixCoord.GetCompactionState(ctx, req)
|
|
log.Debug("received GetCompactionState response",
|
|
zap.Any("resp", resp),
|
|
zap.Error(err))
|
|
return resp, err
|
|
}
|
|
|
|
// ManualCompaction invokes compaction on specified collection
|
|
func (node *Proxy) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompactionRequest) (*milvuspb.ManualCompactionResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ManualCompaction")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("collectionName", req.GetCollectionName()),
|
|
zap.Int64("collectionID", req.GetCollectionID()))
|
|
|
|
log.Info("received ManualCompaction request")
|
|
resp := &milvuspb.ManualCompactionResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
// before v2.4.18, manual compact request only pass collectionID, should correct sdk's behavior to pass collectionName
|
|
if req.GetCollectionName() != "" {
|
|
var err error
|
|
req.CollectionID, err = globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
}
|
|
|
|
resp, err := node.mixCoord.ManualCompaction(ctx, req)
|
|
log.Info("received ManualCompaction response",
|
|
zap.Any("resp", resp),
|
|
zap.Error(err))
|
|
return resp, err
|
|
}
|
|
|
|
// GetCompactionStateWithPlans returns the compactions states with the given plan ID
|
|
func (node *Proxy) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest) (*milvuspb.GetCompactionPlansResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetCompactionStateWithPlans")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.Int64("compactionID", req.GetCompactionID()))
|
|
|
|
log.Debug("received GetCompactionStateWithPlans request")
|
|
resp := &milvuspb.GetCompactionPlansResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
resp, err := node.mixCoord.GetCompactionStateWithPlans(ctx, req)
|
|
log.Debug("received GetCompactionStateWithPlans response",
|
|
zap.Any("resp", resp),
|
|
zap.Error(err))
|
|
return resp, err
|
|
}
|
|
|
|
// GetFlushState gets the flush state of the collection based on the provided flush ts and segment IDs.
|
|
func (node *Proxy) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest) (*milvuspb.GetFlushStateResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetFlushState")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("received get flush state request",
|
|
zap.Any("request", req))
|
|
var err error
|
|
failResp := &milvuspb.GetFlushStateResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
failResp.Status = merr.Status(err)
|
|
log.Warn("unable to get flush state because of closed server")
|
|
return failResp, nil
|
|
}
|
|
|
|
stateReq := &datapb.GetFlushStateRequest{
|
|
SegmentIDs: req.GetSegmentIDs(),
|
|
FlushTs: req.GetFlushTs(),
|
|
}
|
|
|
|
if len(req.GetCollectionName()) > 0 { // For compatibility with old client
|
|
if err = validateCollectionName(req.GetCollectionName()); err != nil {
|
|
failResp.Status = merr.Status(err)
|
|
return failResp, nil
|
|
}
|
|
collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
|
|
if err != nil {
|
|
failResp.Status = merr.Status(err)
|
|
return failResp, nil
|
|
}
|
|
stateReq.CollectionID = collectionID
|
|
}
|
|
|
|
resp, err := node.mixCoord.GetFlushState(ctx, stateReq)
|
|
if err != nil {
|
|
log.Warn("failed to get flush state response",
|
|
zap.Error(err))
|
|
failResp.Status = merr.Status(err)
|
|
return failResp, nil
|
|
}
|
|
log.Debug("received get flush state response",
|
|
zap.Any("response", resp))
|
|
return resp, err
|
|
}
|
|
|
|
// GetFlushAllState checks if all DML messages before `FlushAllTs` have been flushed.
|
|
func (node *Proxy) GetFlushAllState(ctx context.Context, req *milvuspb.GetFlushAllStateRequest) (*milvuspb.GetFlushAllStateResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetFlushAllState")
|
|
defer sp.End()
|
|
log := log.Ctx(ctx).With(zap.Uint64("FlushAllTs", req.GetFlushAllTs()),
|
|
zap.Time("FlushAllTime", tsoutil.PhysicalTime(req.GetFlushAllTs())),
|
|
zap.String("db", req.GetDbName()))
|
|
log.Debug("receive GetFlushAllState request")
|
|
|
|
var err error
|
|
resp := &milvuspb.GetFlushAllStateResponse{}
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp.Status = merr.Status(err)
|
|
log.Warn("GetFlushAllState failed, closed server")
|
|
return resp, nil
|
|
}
|
|
|
|
resp, err = node.mixCoord.GetFlushAllState(ctx, req)
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
log.Warn("GetFlushAllState failed", zap.Error(err))
|
|
return resp, nil
|
|
}
|
|
log.Debug("GetFlushAllState done", zap.Bool("flushed", resp.GetFlushed()))
|
|
return resp, err
|
|
}
|
|
|
|
// checkHealthy checks proxy state is Healthy
|
|
func (node *Proxy) checkHealthy() bool {
|
|
code := node.GetStateCode()
|
|
return code == commonpb.StateCode_Healthy
|
|
}
|
|
|
|
func convertToV2ImportRequest(req *milvuspb.ImportRequest) *internalpb.ImportRequest {
|
|
return &internalpb.ImportRequest{
|
|
DbName: req.GetDbName(),
|
|
CollectionName: req.GetCollectionName(),
|
|
PartitionName: req.GetPartitionName(),
|
|
Files: []*internalpb.ImportFile{{
|
|
Paths: req.GetFiles(),
|
|
}},
|
|
Options: req.GetOptions(),
|
|
}
|
|
}
|
|
|
|
func convertToV1ImportResponse(rsp *internalpb.ImportResponse) *milvuspb.ImportResponse {
|
|
if rsp.GetStatus().GetCode() != 0 {
|
|
return &milvuspb.ImportResponse{
|
|
Status: rsp.GetStatus(),
|
|
}
|
|
}
|
|
jobID, err := strconv.ParseInt(rsp.GetJobID(), 10, 64)
|
|
if err != nil {
|
|
return &milvuspb.ImportResponse{
|
|
Status: merr.Status(merr.WrapErrImportFailed(err.Error())),
|
|
}
|
|
}
|
|
return &milvuspb.ImportResponse{
|
|
Status: rsp.GetStatus(),
|
|
Tasks: []int64{jobID},
|
|
}
|
|
}
|
|
|
|
func convertToV2GetImportRequest(req *milvuspb.GetImportStateRequest) *internalpb.GetImportProgressRequest {
|
|
return &internalpb.GetImportProgressRequest{
|
|
JobID: strconv.FormatInt(req.GetTask(), 10),
|
|
}
|
|
}
|
|
|
|
func convertToV1GetImportResponse(rsp *internalpb.GetImportProgressResponse) *milvuspb.GetImportStateResponse {
|
|
const (
|
|
failedReason = "failed_reason"
|
|
progressPercent = "progress_percent"
|
|
)
|
|
if rsp.GetStatus().GetCode() != 0 {
|
|
return &milvuspb.GetImportStateResponse{
|
|
Status: rsp.GetStatus(),
|
|
}
|
|
}
|
|
convertState := func(state internalpb.ImportJobState) commonpb.ImportState {
|
|
switch state {
|
|
case internalpb.ImportJobState_Pending:
|
|
return commonpb.ImportState_ImportPending
|
|
case internalpb.ImportJobState_Importing:
|
|
return commonpb.ImportState_ImportStarted
|
|
case internalpb.ImportJobState_Completed:
|
|
return commonpb.ImportState_ImportCompleted
|
|
case internalpb.ImportJobState_Failed:
|
|
return commonpb.ImportState_ImportFailed
|
|
}
|
|
return commonpb.ImportState_ImportFailed
|
|
}
|
|
infos := make([]*commonpb.KeyValuePair, 0)
|
|
infos = append(infos, &commonpb.KeyValuePair{
|
|
Key: failedReason,
|
|
Value: rsp.GetReason(),
|
|
})
|
|
infos = append(infos, &commonpb.KeyValuePair{
|
|
Key: progressPercent,
|
|
Value: strconv.FormatInt(rsp.GetProgress(), 10),
|
|
})
|
|
var createTs int64
|
|
createTime, err := time.Parse("2006-01-02T15:04:05Z07:00", rsp.GetStartTime())
|
|
if err == nil {
|
|
createTs = createTime.Unix()
|
|
}
|
|
return &milvuspb.GetImportStateResponse{
|
|
Status: rsp.GetStatus(),
|
|
State: convertState(rsp.GetState()),
|
|
RowCount: rsp.GetImportedRows(),
|
|
IdList: nil,
|
|
Infos: infos,
|
|
Id: 0,
|
|
CollectionId: 0,
|
|
SegmentIds: nil,
|
|
CreateTs: createTs,
|
|
}
|
|
}
|
|
|
|
func convertToV2ListImportRequest(req *milvuspb.ListImportTasksRequest) *internalpb.ListImportsRequest {
|
|
return &internalpb.ListImportsRequest{
|
|
DbName: req.GetDbName(),
|
|
CollectionName: req.GetCollectionName(),
|
|
}
|
|
}
|
|
|
|
func convertToV1ListImportResponse(rsp *internalpb.ListImportsResponse) *milvuspb.ListImportTasksResponse {
|
|
if rsp.GetStatus().GetCode() != 0 {
|
|
return &milvuspb.ListImportTasksResponse{
|
|
Status: rsp.GetStatus(),
|
|
}
|
|
}
|
|
responses := make([]*milvuspb.GetImportStateResponse, 0, len(rsp.GetStates()))
|
|
for i := 0; i < len(rsp.GetStates()); i++ {
|
|
responses = append(responses, convertToV1GetImportResponse(&internalpb.GetImportProgressResponse{
|
|
Status: rsp.GetStatus(),
|
|
State: rsp.GetStates()[i],
|
|
Reason: rsp.GetReasons()[i],
|
|
Progress: rsp.GetProgresses()[i],
|
|
}))
|
|
}
|
|
return &milvuspb.ListImportTasksResponse{
|
|
Status: rsp.GetStatus(),
|
|
Tasks: responses,
|
|
}
|
|
}
|
|
|
|
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
|
func (node *Proxy) Import(ctx context.Context, req *milvuspb.ImportRequest) (*milvuspb.ImportResponse, error) {
|
|
rsp, err := node.ImportV2(ctx, convertToV2ImportRequest(req))
|
|
if err != nil {
|
|
return &milvuspb.ImportResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return convertToV1ImportResponse(rsp), err
|
|
}
|
|
|
|
// GetImportState checks import task state from RootCoord.
|
|
func (node *Proxy) GetImportState(ctx context.Context, req *milvuspb.GetImportStateRequest) (*milvuspb.GetImportStateResponse, error) {
|
|
rsp, err := node.GetImportProgress(ctx, convertToV2GetImportRequest(req))
|
|
if err != nil {
|
|
return &milvuspb.GetImportStateResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return convertToV1GetImportResponse(rsp), err
|
|
}
|
|
|
|
// ListImportTasks get id array of all import tasks from rootcoord
|
|
func (node *Proxy) ListImportTasks(ctx context.Context, req *milvuspb.ListImportTasksRequest) (*milvuspb.ListImportTasksResponse, error) {
|
|
rsp, err := node.ListImports(ctx, convertToV2ListImportRequest(req))
|
|
if err != nil {
|
|
return &milvuspb.ListImportTasksResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return convertToV1ListImportResponse(rsp), err
|
|
}
|
|
|
|
// InvalidateCredentialCache invalidate the credential cache of specified username.
|
|
func (node *Proxy) InvalidateCredentialCache(ctx context.Context, request *proxypb.InvalidateCredCacheRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-InvalidateCredentialCache")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("username", request.Username))
|
|
|
|
log.Debug("received request to invalidate credential cache")
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
username := request.Username
|
|
priCache := privilege.GetPrivilegeCache()
|
|
if priCache != nil {
|
|
priCache.RemoveCredential(username) // no need to return error, though credential may be not cached
|
|
}
|
|
log.Debug("complete to invalidate credential cache")
|
|
|
|
return merr.Success(), nil
|
|
}
|
|
|
|
// UpdateCredentialCache update the credential cache of specified username.
|
|
func (node *Proxy) UpdateCredentialCache(ctx context.Context, request *proxypb.UpdateCredCacheRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-UpdateCredentialCache")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("username", request.Username))
|
|
|
|
log.Debug("received request to update credential cache")
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
credInfo := &internalpb.CredentialInfo{
|
|
Username: request.Username,
|
|
Sha256Password: request.Password,
|
|
}
|
|
priCache := privilege.GetPrivilegeCache()
|
|
if priCache != nil {
|
|
priCache.UpdateCredential(credInfo) // no need to return error, though credential may be not cached
|
|
}
|
|
log.Debug("complete to update credential cache")
|
|
|
|
return merr.Success(), nil
|
|
}
|
|
|
|
func (node *Proxy) CreateCredential(ctx context.Context, req *milvuspb.CreateCredentialRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateCredential")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("username", req.Username))
|
|
|
|
log.Info("CreateCredential",
|
|
zap.String("role", typeutil.ProxyRole))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
// validate params
|
|
username := req.Username
|
|
if err := ValidateUsername(username); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
rawPassword, err := crypto.Base64Decode(req.Password)
|
|
if err != nil {
|
|
log.Error("decode password fail",
|
|
zap.Error(err))
|
|
err = errors.Wrap(err, "decode password fail")
|
|
return merr.Status(err), nil
|
|
}
|
|
if err = ValidatePassword(rawPassword); err != nil {
|
|
log.Error("illegal password",
|
|
zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
encryptedPassword, err := crypto.PasswordEncrypt(rawPassword)
|
|
if err != nil {
|
|
log.Error("encrypt password fail",
|
|
zap.Error(err))
|
|
err = errors.Wrap(err, "encrypt password failed")
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_CreateCredential
|
|
|
|
credInfo := &internalpb.CredentialInfo{
|
|
Username: req.Username,
|
|
EncryptedPassword: encryptedPassword,
|
|
Sha256Password: crypto.SHA256(rawPassword, req.Username),
|
|
}
|
|
result, err := node.mixCoord.CreateCredential(ctx, credInfo)
|
|
if err != nil { // for error like conntext timeout etc.
|
|
log.Error("create credential fail",
|
|
zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, err
|
|
}
|
|
|
|
func (node *Proxy) UpdateCredential(ctx context.Context, req *milvuspb.UpdateCredentialRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-UpdateCredential")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("username", req.Username))
|
|
|
|
log.Info("UpdateCredential",
|
|
zap.String("role", typeutil.ProxyRole))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
rawOldPassword, err := crypto.Base64Decode(req.OldPassword)
|
|
if err != nil {
|
|
log.Error("decode old password fail",
|
|
zap.Error(err))
|
|
err = errors.Wrap(err, "decode old password failed")
|
|
return merr.Status(err), nil
|
|
}
|
|
rawNewPassword, err := crypto.Base64Decode(req.NewPassword)
|
|
if err != nil {
|
|
log.Error("decode password fail",
|
|
zap.Error(err))
|
|
err = errors.Wrap(err, "decode password failed")
|
|
return merr.Status(err), nil
|
|
}
|
|
// valid new password
|
|
if err = ValidatePassword(rawNewPassword); err != nil {
|
|
log.Error("illegal password",
|
|
zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
skipPasswordVerify := false
|
|
if currentUser, _ := GetCurUserFromContext(ctx); currentUser != "" {
|
|
for _, s := range Params.CommonCfg.SuperUsers.GetAsStrings() {
|
|
if s == currentUser {
|
|
skipPasswordVerify = true
|
|
}
|
|
}
|
|
}
|
|
|
|
if !skipPasswordVerify && !passwordVerify(ctx, req.Username, rawOldPassword, privilege.GetPrivilegeCache()) {
|
|
err := merr.WrapErrPrivilegeNotAuthenticated("old password not correct for %s", req.GetUsername())
|
|
return merr.Status(err), nil
|
|
}
|
|
// update meta data
|
|
encryptedPassword, err := crypto.PasswordEncrypt(rawNewPassword)
|
|
if err != nil {
|
|
log.Error("encrypt password fail",
|
|
zap.Error(err))
|
|
err = errors.Wrap(err, "encrypt password failed")
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_UpdateCredential
|
|
updateCredReq := &internalpb.CredentialInfo{
|
|
Username: req.Username,
|
|
Sha256Password: crypto.SHA256(rawNewPassword, req.Username),
|
|
EncryptedPassword: encryptedPassword,
|
|
}
|
|
result, err := node.mixCoord.UpdateCredential(ctx, updateCredReq)
|
|
if err != nil { // for error like conntext timeout etc.
|
|
log.Error("update credential fail",
|
|
zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, err
|
|
}
|
|
|
|
func (node *Proxy) DeleteCredential(ctx context.Context, req *milvuspb.DeleteCredentialRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DeleteCredential")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("username", req.Username))
|
|
|
|
log.Info("DeleteCredential",
|
|
zap.String("role", typeutil.ProxyRole))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
if req.Username == util.UserRoot {
|
|
err := merr.WrapErrPrivilegeNotPermitted("root user cannot be deleted")
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_DeleteCredential
|
|
result, err := node.mixCoord.DeleteCredential(ctx, req)
|
|
if err != nil { // for error like conntext timeout etc.
|
|
log.Error("delete credential fail",
|
|
zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, err
|
|
}
|
|
|
|
func (node *Proxy) ListCredUsers(ctx context.Context, req *milvuspb.ListCredUsersRequest) (*milvuspb.ListCredUsersResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ListCredUsers")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole))
|
|
|
|
log.Debug("ListCredUsers")
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ListCredUsersResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_ListCredUsernames
|
|
rootCoordReq := &milvuspb.ListCredUsersRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_ListCredUsernames),
|
|
),
|
|
}
|
|
resp, err := node.mixCoord.ListCredUsers(ctx, rootCoordReq)
|
|
if err != nil {
|
|
return &milvuspb.ListCredUsersResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return &milvuspb.ListCredUsersResponse{
|
|
Status: merr.Success(),
|
|
Usernames: resp.Usernames,
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) CreateRole(ctx context.Context, req *milvuspb.CreateRoleRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateRole")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("CreateRole", zap.Stringer("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
var roleName string
|
|
if req.Entity != nil {
|
|
roleName = req.Entity.Name
|
|
}
|
|
if err := ValidateRoleName(roleName); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_CreateRole
|
|
|
|
result, err := node.mixCoord.CreateRole(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to create role", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) DropRole(ctx context.Context, req *milvuspb.DropRoleRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropRole")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("DropRole",
|
|
zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := ValidateRoleName(req.RoleName); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_DropRole
|
|
if IsDefaultRole(req.RoleName) {
|
|
err := merr.WrapErrPrivilegeNotPermitted("the role[%s] is a default role, which can't be dropped", req.GetRoleName())
|
|
return merr.Status(err), nil
|
|
}
|
|
result, err := node.mixCoord.DropRole(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to drop role",
|
|
zap.String("role_name", req.RoleName),
|
|
zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) OperateUserRole(ctx context.Context, req *milvuspb.OperateUserRoleRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-OperateUserRole")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("OperateUserRole", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := ValidateUsername(req.Username); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := ValidateRoleName(req.RoleName); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_OperateUserRole
|
|
|
|
result, err := node.mixCoord.OperateUserRole(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to operate user role", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) SelectRole(ctx context.Context, req *milvuspb.SelectRoleRequest) (*milvuspb.SelectRoleResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-SelectRole")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("SelectRole", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.SelectRoleResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
|
|
if req.Role != nil {
|
|
if err := ValidateRoleName(req.Role.Name); err != nil {
|
|
return &milvuspb.SelectRoleResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_SelectRole
|
|
|
|
result, err := node.mixCoord.SelectRole(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to select role", zap.Error(err))
|
|
return &milvuspb.SelectRoleResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) SelectUser(ctx context.Context, req *milvuspb.SelectUserRequest) (*milvuspb.SelectUserResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-SelectUser")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("SelectUser", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.SelectUserResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
|
|
if req.User != nil {
|
|
if err := ValidateUsername(req.User.Name); err != nil {
|
|
log.Warn("invalid username", zap.Error(err))
|
|
return &milvuspb.SelectUserResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_SelectUser
|
|
|
|
result, err := node.mixCoord.SelectUser(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to select user", zap.Error(err))
|
|
return &milvuspb.SelectUserResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) validPrivilegeParams(req *milvuspb.OperatePrivilegeRequest) error {
|
|
if req.Entity == nil {
|
|
return errors.New("the entity in the request is nil")
|
|
}
|
|
if req.Entity.Grantor == nil {
|
|
return errors.New("the grantor entity in the grant entity is nil")
|
|
}
|
|
if req.Entity.Grantor.Privilege == nil {
|
|
return errors.New("the privilege entity in the grantor entity is nil")
|
|
}
|
|
if err := ValidatePrivilege(req.Entity.Grantor.Privilege.Name); err != nil {
|
|
return err
|
|
}
|
|
if req.Entity.Object == nil {
|
|
return errors.New("the resource entity in the grant entity is nil")
|
|
}
|
|
if err := ValidateObjectType(req.Entity.Object.Name); err != nil {
|
|
return err
|
|
}
|
|
if err := ValidateObjectName(req.Entity.ObjectName); err != nil {
|
|
return err
|
|
}
|
|
if req.Entity.Role == nil {
|
|
return errors.New("the object entity in the grant entity is nil")
|
|
}
|
|
if err := ValidateRoleName(req.Entity.Role.Name); err != nil {
|
|
return err
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (node *Proxy) validateOperatePrivilegeV2Params(req *milvuspb.OperatePrivilegeV2Request) error {
|
|
if req.Role == nil {
|
|
return merr.WrapErrParameterInvalidMsg("the role in the request is nil")
|
|
}
|
|
if err := ValidateRoleName(req.Role.Name); err != nil {
|
|
return err
|
|
}
|
|
if err := ValidatePrivilege(req.Grantor.Privilege.Name); err != nil {
|
|
return err
|
|
}
|
|
if req.Type != milvuspb.OperatePrivilegeType_Grant && req.Type != milvuspb.OperatePrivilegeType_Revoke {
|
|
return merr.WrapErrParameterInvalidMsg("the type in the request not grant or revoke")
|
|
}
|
|
if req.DbName != "" && !util.IsAnyWord(req.DbName) {
|
|
if err := ValidateDatabaseName(req.DbName); err != nil {
|
|
return err
|
|
}
|
|
}
|
|
if err := ValidateCollectionName(req.CollectionName); err != nil {
|
|
return err
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (node *Proxy) OperatePrivilegeV2(ctx context.Context, req *milvuspb.OperatePrivilegeV2Request) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-OperatePrivilegeV2")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("OperatePrivilegeV2",
|
|
zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := node.validateOperatePrivilegeV2Params(req); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
curUser, err := GetCurUserFromContext(ctx)
|
|
if err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_OperatePrivilegeV2
|
|
req.Grantor.User = &milvuspb.UserEntity{Name: curUser}
|
|
request := &milvuspb.OperatePrivilegeRequest{
|
|
Entity: &milvuspb.GrantEntity{
|
|
Role: req.Role,
|
|
Object: &milvuspb.ObjectEntity{Name: commonpb.ObjectType_Global.String()},
|
|
ObjectName: req.CollectionName,
|
|
DbName: req.DbName,
|
|
Grantor: req.Grantor,
|
|
},
|
|
Type: req.Type,
|
|
Version: "v2",
|
|
}
|
|
request.Base = req.Base
|
|
result, err := node.mixCoord.OperatePrivilege(ctx, request)
|
|
if err != nil {
|
|
log.Warn("fail to operate privilege", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
relatedPrivileges := util.RelatedPrivileges[util.PrivilegeNameForMetastore(req.Grantor.Privilege.Name)]
|
|
if len(relatedPrivileges) != 0 {
|
|
for _, relatedPrivilege := range relatedPrivileges {
|
|
relatedReq := proto.Clone(request).(*milvuspb.OperatePrivilegeRequest)
|
|
relatedReq.Entity.Grantor.Privilege.Name = util.PrivilegeNameForAPI(relatedPrivilege)
|
|
result, err = node.mixCoord.OperatePrivilege(ctx, relatedReq)
|
|
if err != nil {
|
|
log.Warn("fail to operate related privilege", zap.String("related_privilege", relatedPrivilege), zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
if !merr.Ok(result) {
|
|
log.Warn("fail to operate related privilege", zap.String("related_privilege", relatedPrivilege), zap.Any("result", result))
|
|
return result, nil
|
|
}
|
|
}
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) OperatePrivilege(ctx context.Context, req *milvuspb.OperatePrivilegeRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-OperatePrivilege")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("OperatePrivilege",
|
|
zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := node.validPrivilegeParams(req); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_OperatePrivilege
|
|
curUser, err := GetCurUserFromContext(ctx)
|
|
if err != nil {
|
|
log.Warn("fail to get current user", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
req.Entity.Grantor.User = &milvuspb.UserEntity{Name: curUser}
|
|
result, err := node.mixCoord.OperatePrivilege(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to operate privilege", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
relatedPrivileges := util.RelatedPrivileges[util.PrivilegeNameForMetastore(req.Entity.Grantor.Privilege.Name)]
|
|
if len(relatedPrivileges) != 0 {
|
|
for _, relatedPrivilege := range relatedPrivileges {
|
|
relatedReq := proto.Clone(req).(*milvuspb.OperatePrivilegeRequest)
|
|
relatedReq.Entity.Grantor.Privilege.Name = util.PrivilegeNameForAPI(relatedPrivilege)
|
|
result, err = node.mixCoord.OperatePrivilege(ctx, relatedReq)
|
|
if err != nil {
|
|
log.Warn("fail to operate related privilege", zap.String("related_privilege", relatedPrivilege), zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
if !merr.Ok(result) {
|
|
log.Warn("fail to operate related privilege", zap.String("related_privilege", relatedPrivilege), zap.Any("result", result))
|
|
return result, nil
|
|
}
|
|
}
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) validGrantParams(req *milvuspb.SelectGrantRequest) error {
|
|
if req.Entity == nil {
|
|
return merr.WrapErrParameterInvalidMsg("the grant entity in the request is nil")
|
|
}
|
|
|
|
if req.Entity.Object != nil {
|
|
if err := ValidateObjectType(req.Entity.Object.Name); err != nil {
|
|
return err
|
|
}
|
|
|
|
if err := ValidateObjectName(req.Entity.ObjectName); err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
if req.Entity.Role == nil {
|
|
return merr.WrapErrParameterInvalidMsg("the role entity in the grant entity is nil")
|
|
}
|
|
|
|
if err := ValidateRoleName(req.Entity.Role.Name); err != nil {
|
|
return err
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (node *Proxy) SelectGrant(ctx context.Context, req *milvuspb.SelectGrantRequest) (*milvuspb.SelectGrantResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-SelectGrant")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("SelectGrant",
|
|
zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.SelectGrantResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
|
|
if err := node.validGrantParams(req); err != nil {
|
|
return &milvuspb.SelectGrantResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_SelectGrant
|
|
|
|
result, err := node.mixCoord.SelectGrant(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to select grant", zap.Error(err))
|
|
return &milvuspb.SelectGrantResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) BackupRBAC(ctx context.Context, req *milvuspb.BackupRBACMetaRequest) (*milvuspb.BackupRBACMetaResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-BackupRBAC")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("BackupRBAC", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.BackupRBACMetaResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
|
|
result, err := node.mixCoord.BackupRBAC(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to backup rbac", zap.Error(err))
|
|
return &milvuspb.BackupRBACMetaResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) RestoreRBAC(ctx context.Context, req *milvuspb.RestoreRBACMetaRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-RestoreRBAC")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("RestoreRBAC", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if req.RBACMeta == nil {
|
|
return merr.Success(), nil
|
|
}
|
|
|
|
result, err := node.mixCoord.RestoreRBAC(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to restore rbac", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) RefreshPolicyInfoCache(ctx context.Context, req *proxypb.RefreshPolicyInfoCacheRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-RefreshPolicyInfoCache")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Debug("RefreshPrivilegeInfoCache",
|
|
zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
priCache := privilege.GetPrivilegeCache()
|
|
if priCache != nil {
|
|
err := priCache.RefreshPolicyInfo(typeutil.CacheOp{
|
|
OpType: typeutil.CacheOpType(req.OpType),
|
|
OpKey: req.OpKey,
|
|
})
|
|
if err != nil {
|
|
log.Warn("fail to refresh policy info",
|
|
zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
}
|
|
log.Debug("RefreshPrivilegeInfoCache success")
|
|
|
|
return merr.Success(), nil
|
|
}
|
|
|
|
// SetRates limits the rates of requests.
|
|
func (node *Proxy) SetRates(ctx context.Context, request *proxypb.SetRatesRequest) (*commonpb.Status, error) {
|
|
resp := merr.Success()
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
resp = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
err := node.simpleLimiter.SetRates(request.GetRootLimiter())
|
|
// TODO: set multiple rate limiter rates
|
|
if err != nil {
|
|
resp = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
return resp, nil
|
|
}
|
|
|
|
func (node *Proxy) CheckHealth(ctx context.Context, request *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.CheckHealthResponse{
|
|
Status: merr.Status(err),
|
|
IsHealthy: false,
|
|
Reasons: []string{err.Error()},
|
|
}, nil
|
|
}
|
|
|
|
group, ctx := errgroup.WithContext(ctx)
|
|
errReasons := make([]string, 0)
|
|
|
|
mu := &sync.Mutex{}
|
|
fn := func(role string, resp *milvuspb.CheckHealthResponse, err error) error {
|
|
mu.Lock()
|
|
defer mu.Unlock()
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-RefreshPolicyInfoCache")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(zap.String("role", role))
|
|
|
|
if err != nil {
|
|
log.Warn("check health fail",
|
|
zap.Error(err))
|
|
errReasons = append(errReasons, fmt.Sprintf("check health fail for %s", role))
|
|
return err
|
|
}
|
|
|
|
if !resp.IsHealthy {
|
|
log.Warn("check health fail")
|
|
errReasons = append(errReasons, resp.Reasons...)
|
|
}
|
|
return nil
|
|
}
|
|
|
|
group.Go(func() error {
|
|
resp, err := node.mixCoord.CheckHealth(ctx, request)
|
|
return fn("mixcoord", resp, err)
|
|
})
|
|
|
|
err := group.Wait()
|
|
if err != nil || len(errReasons) != 0 {
|
|
return &milvuspb.CheckHealthResponse{
|
|
Status: merr.Success(),
|
|
IsHealthy: false,
|
|
Reasons: errReasons,
|
|
}, nil
|
|
}
|
|
|
|
return &milvuspb.CheckHealthResponse{
|
|
Status: merr.Success(),
|
|
IsHealthy: true,
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) RenameCollection(ctx context.Context, req *milvuspb.RenameCollectionRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-RenameCollection")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("oldName", req.GetOldName()),
|
|
zap.String("newName", req.GetNewName()))
|
|
|
|
log.Info("received rename collection request")
|
|
var err error
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
if err := validateCollectionName(req.GetNewName()); err != nil {
|
|
log.Warn("validate new collection name fail", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
req.Base = commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_RenameCollection),
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
)
|
|
resp, err := node.mixCoord.RenameCollection(ctx, req)
|
|
if err != nil {
|
|
log.Warn("failed to rename collection", zap.Error(err))
|
|
return merr.Status(err), err
|
|
}
|
|
return resp, nil
|
|
}
|
|
|
|
func (node *Proxy) CreateResourceGroup(ctx context.Context, request *milvuspb.CreateResourceGroupRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
method := "CreateResourceGroup"
|
|
if err := ValidateResourceGroupName(request.GetResourceGroup()); err != nil {
|
|
log.Warn("CreateResourceGroup failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateResourceGroup")
|
|
defer sp.End()
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
t := &CreateResourceGroupTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
CreateResourceGroupRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Info("CreateResourceGroup received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(t); err != nil {
|
|
log.Warn("CreateResourceGroup failed to enqueue",
|
|
zap.Error(err))
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Debug("CreateResourceGroup enqueued",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
log.Warn("CreateResourceGroup failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Info("CreateResourceGroup done",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return t.result, nil
|
|
}
|
|
|
|
func (node *Proxy) UpdateResourceGroups(ctx context.Context, request *milvuspb.UpdateResourceGroupsRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
method := "UpdateResourceGroups"
|
|
for name := range request.GetResourceGroups() {
|
|
if err := ValidateResourceGroupName(name); err != nil {
|
|
log.Warn("UpdateResourceGroups failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-UpdateResourceGroups")
|
|
defer sp.End()
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
t := &UpdateResourceGroupsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
UpdateResourceGroupsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Info("UpdateResourceGroups received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(t); err != nil {
|
|
log.Warn("UpdateResourceGroups failed to enqueue",
|
|
zap.Error(err))
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Debug("UpdateResourceGroups enqueued",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
log.Warn("UpdateResourceGroups failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Info("UpdateResourceGroups done",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return t.result, nil
|
|
}
|
|
|
|
func getErrResponse(err error, method string, dbName string, collectionName string) *commonpb.Status {
|
|
return merr.Status(err)
|
|
}
|
|
|
|
func (node *Proxy) DropResourceGroup(ctx context.Context, request *milvuspb.DropResourceGroupRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
method := "DropResourceGroup"
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropResourceGroup")
|
|
defer sp.End()
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
t := &DropResourceGroupTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DropResourceGroupRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Info("DropResourceGroup received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(t); err != nil {
|
|
log.Warn("DropResourceGroup failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Debug("DropResourceGroup enqueued",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
log.Warn("DropResourceGroup failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Info("DropResourceGroup done",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return t.result, nil
|
|
}
|
|
|
|
func (node *Proxy) TransferNode(ctx context.Context, request *milvuspb.TransferNodeRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
method := "TransferNode"
|
|
if err := ValidateResourceGroupName(request.GetSourceResourceGroup()); err != nil {
|
|
log.Ctx(ctx).Warn("TransferNode failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
if err := ValidateResourceGroupName(request.GetTargetResourceGroup()); err != nil {
|
|
log.Ctx(ctx).Warn("TransferNode failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-TransferNode")
|
|
defer sp.End()
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
t := &TransferNodeTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
TransferNodeRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Info("TransferNode received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(t); err != nil {
|
|
log.Warn("TransferNode failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Debug("TransferNode enqueued",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
log.Warn("TransferNode failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
return getErrResponse(err, method, "", ""), nil
|
|
}
|
|
|
|
log.Info("TransferNode done",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return t.result, nil
|
|
}
|
|
|
|
func (node *Proxy) TransferReplica(ctx context.Context, request *milvuspb.TransferReplicaRequest) (*commonpb.Status, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
method := "TransferReplica"
|
|
if err := ValidateResourceGroupName(request.GetSourceResourceGroup()); err != nil {
|
|
log.Ctx(ctx).Warn("TransferReplica failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, method, request.GetDbName(), request.GetCollectionName()), nil
|
|
}
|
|
|
|
if err := ValidateResourceGroupName(request.GetTargetResourceGroup()); err != nil {
|
|
log.Ctx(ctx).Warn("TransferReplica failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, method, request.GetDbName(), request.GetCollectionName()), nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-TransferReplica")
|
|
defer sp.End()
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
t := &TransferReplicaTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
TransferReplicaRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Info("TransferReplica received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(t); err != nil {
|
|
log.Warn("TransferReplica failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return getErrResponse(err, method, request.GetDbName(), request.GetCollectionName()), nil
|
|
}
|
|
|
|
log.Debug("TransferReplica enqueued",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
log.Warn("TransferReplica failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
return getErrResponse(err, method, request.GetDbName(), request.GetCollectionName()), nil
|
|
}
|
|
|
|
log.Info("TransferReplica done",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return t.result, nil
|
|
}
|
|
|
|
func (node *Proxy) ListResourceGroups(ctx context.Context, request *milvuspb.ListResourceGroupsRequest) (*milvuspb.ListResourceGroupsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ListResourceGroupsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ListResourceGroups")
|
|
defer sp.End()
|
|
method := "ListResourceGroups"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
t := &ListResourceGroupsTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
ListResourceGroupsRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Debug("ListResourceGroups received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(t); err != nil {
|
|
log.Warn("ListResourceGroups failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return &milvuspb.ListResourceGroupsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug("ListResourceGroups enqueued",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
log.Warn("ListResourceGroups failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
return &milvuspb.ListResourceGroupsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Debug("ListResourceGroups done",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return t.result, nil
|
|
}
|
|
|
|
func (node *Proxy) DescribeResourceGroup(ctx context.Context, request *milvuspb.DescribeResourceGroupRequest) (*milvuspb.DescribeResourceGroupResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.DescribeResourceGroupResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
method := "DescribeResourceGroup"
|
|
GetErrResponse := func(err error) *milvuspb.DescribeResourceGroupResponse {
|
|
return &milvuspb.DescribeResourceGroupResponse{
|
|
Status: merr.Status(err),
|
|
}
|
|
}
|
|
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DescribeResourceGroup")
|
|
defer sp.End()
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
t := &DescribeResourceGroupTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
DescribeResourceGroupRequest: request,
|
|
mixCoord: node.mixCoord,
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
|
|
log.Debug("DescribeResourceGroup received")
|
|
|
|
if err := node.sched.ddQueue.Enqueue(t); err != nil {
|
|
log.Warn("DescribeResourceGroup failed to enqueue",
|
|
zap.Error(err))
|
|
|
|
return GetErrResponse(err), nil
|
|
}
|
|
|
|
log.Debug("DescribeResourceGroup enqueued",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
if err := t.WaitToFinish(); err != nil {
|
|
log.Warn("DescribeResourceGroup failed to WaitToFinish",
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
return GetErrResponse(err), nil
|
|
}
|
|
|
|
log.Debug("DescribeResourceGroup done",
|
|
zap.Uint64("BeginTS", t.BeginTs()),
|
|
zap.Uint64("EndTS", t.EndTs()))
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return t.result, nil
|
|
}
|
|
|
|
func (node *Proxy) ListIndexedSegment(ctx context.Context, request *federpb.ListIndexedSegmentRequest) (*federpb.ListIndexedSegmentResponse, error) {
|
|
return &federpb.ListIndexedSegmentResponse{
|
|
Status: merr.Status(merr.WrapErrServiceUnavailable("unimp")),
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) DescribeSegmentIndexData(ctx context.Context, request *federpb.DescribeSegmentIndexDataRequest) (*federpb.DescribeSegmentIndexDataResponse, error) {
|
|
return &federpb.DescribeSegmentIndexDataResponse{
|
|
Status: merr.Status(merr.WrapErrServiceUnavailable("unimp")),
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) Connect(ctx context.Context, request *milvuspb.ConnectRequest) (*milvuspb.ConnectResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ConnectResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
|
|
db := GetCurDBNameFromContextOrDefault(ctx)
|
|
logsToBePrinted := append(connection.ZapClientInfo(request.GetClientInfo()), zap.String("db", db))
|
|
log := log.Ctx(ctx).With(logsToBePrinted...)
|
|
|
|
log.Info("connect received")
|
|
|
|
resp, err := node.mixCoord.ListDatabases(ctx, &milvuspb.ListDatabasesRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_ListDatabases),
|
|
),
|
|
})
|
|
if err == nil {
|
|
err = merr.Error(resp.GetStatus())
|
|
}
|
|
|
|
if err != nil {
|
|
log.Info("connect failed, failed to list databases", zap.Error(err))
|
|
return &milvuspb.ConnectResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
if !funcutil.SliceContain(resp.GetDbNames(), db) {
|
|
log.Info("connect failed, target database not exist")
|
|
return &milvuspb.ConnectResponse{
|
|
Status: merr.Status(merr.WrapErrDatabaseNotFound(db)),
|
|
}, nil
|
|
}
|
|
|
|
ts, err := node.tsoAllocator.AllocOne(ctx)
|
|
if err != nil {
|
|
log.Info("connect failed, failed to allocate timestamp", zap.Error(err))
|
|
return &milvuspb.ConnectResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
serverInfo := &commonpb.ServerInfo{
|
|
BuildTags: os.Getenv(metricsinfo.GitBuildTagsEnvKey),
|
|
BuildTime: os.Getenv(metricsinfo.MilvusBuildTimeEnvKey),
|
|
GitCommit: os.Getenv(metricsinfo.GitCommitEnvKey),
|
|
GoVersion: os.Getenv(metricsinfo.MilvusUsedGoVersion),
|
|
DeployMode: os.Getenv(metricsinfo.DeployModeEnvKey),
|
|
Reserved: make(map[string]string),
|
|
}
|
|
|
|
connection.GetManager().Register(ctx, int64(ts), request.GetClientInfo())
|
|
|
|
return &milvuspb.ConnectResponse{
|
|
Status: merr.Success(),
|
|
ServerInfo: serverInfo,
|
|
Identifier: int64(ts),
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) ReplicateMessage(ctx context.Context, req *milvuspb.ReplicateMessageRequest) (*milvuspb.ReplicateMessageResponse, error) {
|
|
return &milvuspb.ReplicateMessageResponse{
|
|
Status: merr.Status(merr.WrapErrServiceUnavailable("not supported in streaming mode")),
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) ListClientInfos(ctx context.Context, req *proxypb.ListClientInfosRequest) (*proxypb.ListClientInfosResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &proxypb.ListClientInfosResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
|
|
clients := connection.GetManager().List()
|
|
return &proxypb.ListClientInfosResponse{
|
|
Status: merr.Success(),
|
|
ClientInfos: clients,
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) AllocTimestamp(ctx context.Context, req *milvuspb.AllocTimestampRequest) (*milvuspb.AllocTimestampResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.AllocTimestampResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
)
|
|
log.Info("AllocTimestamp request receive")
|
|
ts, err := node.tsoAllocator.AllocOne(ctx)
|
|
if err != nil {
|
|
log.Info("AllocTimestamp failed", zap.Error(err))
|
|
return &milvuspb.AllocTimestampResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Info("AllocTimestamp request success", zap.Uint64("timestamp", ts))
|
|
|
|
return &milvuspb.AllocTimestampResponse{
|
|
Status: merr.Success(),
|
|
Timestamp: ts,
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) GetVersion(ctx context.Context, request *milvuspb.GetVersionRequest) (*milvuspb.GetVersionResponse, error) {
|
|
// TODO implement me
|
|
return &milvuspb.GetVersionResponse{
|
|
Status: merr.Success(),
|
|
}, nil
|
|
}
|
|
|
|
func (node *Proxy) ImportV2(ctx context.Context, req *internalpb.ImportRequest) (*internalpb.ImportResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &internalpb.ImportResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("collectionName", req.GetCollectionName()),
|
|
zap.String("partition name", req.GetPartitionName()),
|
|
zap.Any("files", req.GetFiles()),
|
|
zap.Any("options", req.GetOptions()),
|
|
)
|
|
|
|
resp := &internalpb.ImportResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
|
|
method := "ImportV2"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
log.Info(rpcReceived(method))
|
|
nodeID := fmt.Sprint(paramtable.GetNodeID())
|
|
|
|
it := &importTask{
|
|
ctx: ctx,
|
|
Condition: NewTaskCondition(ctx),
|
|
req: req,
|
|
node: node,
|
|
mixCoord: node.mixCoord,
|
|
resp: resp,
|
|
}
|
|
|
|
if err := node.sched.dmQueue.Enqueue(it); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
log.Info(
|
|
rpcEnqueued(method),
|
|
zap.Uint64("BeginTs", it.BeginTs()),
|
|
zap.Uint64("EndTs", it.EndTs()))
|
|
|
|
if err := it.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
zap.Uint64("BeginTs", it.BeginTs()),
|
|
zap.Uint64("EndTs", it.EndTs()))
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
|
|
metrics.ProxyReqLatency.WithLabelValues(nodeID, method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return resp, nil
|
|
}
|
|
|
|
func (node *Proxy) GetImportProgress(ctx context.Context, req *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &internalpb.GetImportProgressResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("jobID", req.GetJobID()),
|
|
)
|
|
method := "GetImportProgress"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
log.Info(rpcReceived(method))
|
|
|
|
nodeID := fmt.Sprint(paramtable.GetNodeID())
|
|
resp, err := node.mixCoord.GetImportProgress(ctx, req)
|
|
if resp.GetStatus().GetCode() != 0 || err != nil {
|
|
log.Warn("get import progress failed", zap.String("reason", resp.GetStatus().GetReason()), zap.Error(err))
|
|
}
|
|
metrics.ProxyReqLatency.WithLabelValues(nodeID, method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return resp, err
|
|
}
|
|
|
|
func (node *Proxy) ListImports(ctx context.Context, req *internalpb.ListImportsRequest) (*internalpb.ListImportsResponse, error) {
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &internalpb.ListImportsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
resp := &internalpb.ListImportsResponse{
|
|
Status: merr.Success(),
|
|
}
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("dbName", req.GetDbName()),
|
|
zap.String("collectionName", req.GetCollectionName()),
|
|
)
|
|
method := "ListImports"
|
|
tr := timerecord.NewTimeRecorder(method)
|
|
log.Info(rpcReceived(method))
|
|
|
|
nodeID := fmt.Sprint(paramtable.GetNodeID())
|
|
|
|
var (
|
|
err error
|
|
collectionID UniqueID
|
|
)
|
|
if req.GetCollectionName() != "" {
|
|
collectionID, err = globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
|
|
if err != nil {
|
|
resp.Status = merr.Status(err)
|
|
return resp, nil
|
|
}
|
|
}
|
|
resp, err = node.mixCoord.ListImports(ctx, &internalpb.ListImportsRequestInternal{
|
|
CollectionID: collectionID,
|
|
})
|
|
if resp.GetStatus().GetCode() != 0 || err != nil {
|
|
log.Warn("list imports", zap.String("reason", resp.GetStatus().GetReason()), zap.Error(err))
|
|
}
|
|
metrics.ProxyReqLatency.WithLabelValues(nodeID, method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
return resp, nil
|
|
}
|
|
|
|
// DeregisterSubLabel must add the sub-labels here if using other labels for the sub-labels
|
|
func DeregisterSubLabel(subLabel string) {
|
|
rateCol.DeregisterSubLabel(internalpb.RateType_DQLQuery.String(), subLabel)
|
|
rateCol.DeregisterSubLabel(internalpb.RateType_DQLSearch.String(), subLabel)
|
|
}
|
|
|
|
// RegisterRestRouter registers the router for the proxy
|
|
func (node *Proxy) RegisterRestRouter(router gin.IRouter) {
|
|
// Cluster request that executed by proxy
|
|
router.GET(http.ClusterInfoPath, getClusterInfo(node))
|
|
router.GET(http.ClusterConfigsPath, getConfigs(paramtable.Get().GetConfigsView()))
|
|
router.GET(http.ClusterClientsPath, getConnectedClients)
|
|
router.GET(http.ClusterDependenciesPath, getDependencies)
|
|
|
|
// Hook request that executed by proxy
|
|
router.GET(http.HookConfigsPath, getConfigs(paramtable.GetHookParams().GetAll()))
|
|
|
|
// Slow query request that executed by proxy
|
|
router.GET(http.SlowQueryPath, getSlowQuery(node))
|
|
|
|
// QueryCoord requests that are forwarded from proxy
|
|
router.GET(http.QCTargetPath, getQueryComponentMetrics(node, metricsinfo.TargetKey))
|
|
router.GET(http.QCDistPath, getQueryComponentMetrics(node, metricsinfo.DistKey))
|
|
router.GET(http.QCReplicaPath, getQueryComponentMetrics(node, metricsinfo.ReplicaKey))
|
|
router.GET(http.QCResourceGroupPath, getQueryComponentMetrics(node, metricsinfo.ResourceGroupKey))
|
|
router.GET(http.QCAllTasksPath, getQueryComponentMetrics(node, metricsinfo.AllTaskKey))
|
|
router.GET(http.QCSegmentsPath, getQueryComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInQC))
|
|
|
|
// QueryNode requests that are forwarded from querycoord
|
|
router.GET(http.QNSegmentsPath, getQueryComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInQN))
|
|
router.GET(http.QNChannelsPath, getQueryComponentMetrics(node, metricsinfo.ChannelKey))
|
|
|
|
// DataCoord requests that are forwarded from proxy
|
|
router.GET(http.DCDistPath, getDataComponentMetrics(node, metricsinfo.DistKey))
|
|
router.GET(http.DCCompactionTasksPath, getDataComponentMetrics(node, metricsinfo.CompactionTaskKey))
|
|
router.GET(http.DCImportTasksPath, getDataComponentMetrics(node, metricsinfo.ImportTaskKey))
|
|
router.GET(http.DCBuildIndexTasksPath, getDataComponentMetrics(node, metricsinfo.BuildIndexTaskKey))
|
|
router.GET(http.IndexListPath, getDataComponentMetrics(node, metricsinfo.IndexKey))
|
|
router.GET(http.DCSegmentsPath, getDataComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInDC))
|
|
|
|
// Datanode requests that are forwarded from datacoord
|
|
router.GET(http.DNSyncTasksPath, getDataComponentMetrics(node, metricsinfo.SyncTaskKey))
|
|
router.GET(http.DNSegmentsPath, getDataComponentMetrics(node, metricsinfo.SegmentKey, metricsinfo.RequestParamsInDN))
|
|
router.GET(http.DNChannelsPath, getDataComponentMetrics(node, metricsinfo.ChannelKey))
|
|
|
|
// Database requests
|
|
router.GET(http.DatabaseListPath, listDatabase(node))
|
|
router.GET(http.DatabaseDescPath, describeDatabase(node))
|
|
|
|
// Collection requests
|
|
router.GET(http.CollectionListPath, listCollection(node))
|
|
router.GET(http.CollectionDescPath, describeCollection(node))
|
|
}
|
|
|
|
func (node *Proxy) CreatePrivilegeGroup(ctx context.Context, req *milvuspb.CreatePrivilegeGroupRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreatePrivilegeGroup")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("CreatePrivilegeGroup", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := ValidatePrivilegeGroupName(req.GroupName); err != nil {
|
|
log.Warn("CreatePrivilegeGroup failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, "CreatePrivilegeGroup", "", ""), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_CreatePrivilegeGroup
|
|
|
|
result, err := node.mixCoord.CreatePrivilegeGroup(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to create privilege group", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) DropPrivilegeGroup(ctx context.Context, req *milvuspb.DropPrivilegeGroupRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-DropPrivilegeGroup")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("DropPrivilegeGroup", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := ValidatePrivilegeGroupName(req.GroupName); err != nil {
|
|
log.Warn("DropPrivilegeGroup failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, "DropPrivilegeGroup", "", ""), nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_DropPrivilegeGroup
|
|
|
|
result, err := node.mixCoord.DropPrivilegeGroup(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to drop privilege group", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) ListPrivilegeGroups(ctx context.Context, req *milvuspb.ListPrivilegeGroupsRequest) (*milvuspb.ListPrivilegeGroupsResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ListPrivilegeGroups")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole))
|
|
|
|
log.Debug("ListPrivilegeGroups")
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ListPrivilegeGroupsResponse{Status: merr.Status(err)}, nil
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_ListPrivilegeGroups
|
|
rootCoordReq := &milvuspb.ListPrivilegeGroupsRequest{
|
|
Base: commonpbutil.NewMsgBase(
|
|
commonpbutil.WithMsgType(commonpb.MsgType_ListPrivilegeGroups),
|
|
),
|
|
}
|
|
resp, err := node.mixCoord.ListPrivilegeGroups(ctx, rootCoordReq)
|
|
if err != nil {
|
|
return &milvuspb.ListPrivilegeGroupsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return resp, nil
|
|
}
|
|
|
|
func (node *Proxy) OperatePrivilegeGroup(ctx context.Context, req *milvuspb.OperatePrivilegeGroupRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-OperatePrivilegeGroup")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
log.Info("OperatePrivilegeGroup", zap.Any("req", req))
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
if err := ValidatePrivilegeGroupName(req.GroupName); err != nil {
|
|
log.Warn("OperatePrivilegeGroup failed",
|
|
zap.Error(err),
|
|
)
|
|
return getErrResponse(err, "OperatePrivilegeGroup", "", ""), nil
|
|
}
|
|
for _, priv := range req.GetPrivileges() {
|
|
if err := ValidatePrivilege(priv.Name); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
}
|
|
if req.Base == nil {
|
|
req.Base = &commonpb.MsgBase{}
|
|
}
|
|
req.Base.MsgType = commonpb.MsgType_OperatePrivilegeGroup
|
|
|
|
result, err := node.mixCoord.OperatePrivilegeGroup(ctx, req)
|
|
if err != nil {
|
|
log.Warn("fail to operate privilege group", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
return result, nil
|
|
}
|
|
|
|
func (node *Proxy) runAnalyzer(req *milvuspb.RunAnalyzerRequest) ([]*milvuspb.AnalyzerResult, error) {
|
|
analyzer, err := analyzer.NewAnalyzer(req.GetAnalyzerParams())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
defer analyzer.Destroy()
|
|
|
|
results := make([]*milvuspb.AnalyzerResult, len(req.GetPlaceholder()))
|
|
for i, text := range req.GetPlaceholder() {
|
|
stream := analyzer.NewTokenStream(string(text))
|
|
defer stream.Destroy()
|
|
|
|
results[i] = &milvuspb.AnalyzerResult{
|
|
Tokens: make([]*milvuspb.AnalyzerToken, 0),
|
|
}
|
|
|
|
for stream.Advance() {
|
|
var token *milvuspb.AnalyzerToken
|
|
if req.GetWithDetail() {
|
|
token = stream.DetailedToken()
|
|
} else {
|
|
token = &milvuspb.AnalyzerToken{Token: stream.Token()}
|
|
}
|
|
|
|
if req.GetWithHash() {
|
|
token.Hash = typeutil.HashString2LessUint32(token.GetToken())
|
|
}
|
|
results[i].Tokens = append(results[i].Tokens, token)
|
|
}
|
|
}
|
|
return results, nil
|
|
}
|
|
|
|
func (node *Proxy) RunAnalyzer(ctx context.Context, req *milvuspb.RunAnalyzerRequest) (*milvuspb.RunAnalyzerResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-RunAnalyzer")
|
|
defer sp.End()
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.RunAnalyzerResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
if len(req.Placeholder) == 0 {
|
|
return &milvuspb.RunAnalyzerResponse{
|
|
Status: merr.Status(nil),
|
|
Results: make([]*milvuspb.AnalyzerResult, 0),
|
|
}, nil
|
|
}
|
|
|
|
if req.GetCollectionName() == "" {
|
|
results, err := node.runAnalyzer(req)
|
|
if err != nil {
|
|
return &milvuspb.RunAnalyzerResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
return &milvuspb.RunAnalyzerResponse{
|
|
Status: merr.Status(nil),
|
|
Results: results,
|
|
}, nil
|
|
}
|
|
|
|
if err := validateRunAnalyzer(req); err != nil {
|
|
return &milvuspb.RunAnalyzerResponse{
|
|
Status: merr.Status(merr.WrapErrAsInputError(err)),
|
|
}, nil
|
|
}
|
|
|
|
method := "RunAnalyzer"
|
|
task := &RunAnalyzerTask{
|
|
ctx: ctx,
|
|
lb: node.lbPolicy,
|
|
Condition: NewTaskCondition(ctx),
|
|
RunAnalyzerRequest: req,
|
|
}
|
|
|
|
if err := node.sched.dqQueue.Enqueue(task); err != nil {
|
|
log.Warn(
|
|
rpcFailedToEnqueue(method),
|
|
zap.Error(err),
|
|
)
|
|
|
|
return &milvuspb.RunAnalyzerResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
if err := task.WaitToFinish(); err != nil {
|
|
log.Warn(
|
|
rpcFailedToWaitToFinish(method),
|
|
zap.Error(err),
|
|
)
|
|
|
|
return &milvuspb.RunAnalyzerResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
return task.result, nil
|
|
}
|
|
|
|
func (node *Proxy) GetQuotaMetrics(ctx context.Context, req *internalpb.GetQuotaMetricsRequest) (*internalpb.GetQuotaMetricsResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetQuotaMetrics")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx)
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &internalpb.GetQuotaMetricsResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Info("receive GetQuotaMetrics request")
|
|
|
|
metricsResp, err := node.mixCoord.GetQuotaMetrics(ctx, req)
|
|
if err != nil {
|
|
log.Warn("GetQuotaMetrics fail",
|
|
zap.Error(err))
|
|
metricsResp.Status = merr.Status(err)
|
|
return metricsResp, nil
|
|
}
|
|
err = merr.Error(metricsResp.GetStatus())
|
|
if err != nil {
|
|
metricsResp.Status = merr.Status(err)
|
|
return metricsResp, nil
|
|
}
|
|
|
|
log.Info("GetQuotaMetrics success", zap.String("metrics", metricsResp.GetMetricsInfo()))
|
|
|
|
return metricsResp, nil
|
|
}
|
|
|
|
// AddFileResource add file resource to rootcoord
|
|
func (node *Proxy) AddFileResource(ctx context.Context, req *milvuspb.AddFileResourceRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-AddFileResource")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("name", req.GetName()),
|
|
zap.String("path", req.GetPath()))
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info("receive AddFileResource request")
|
|
|
|
status, err := node.mixCoord.AddFileResource(ctx, req)
|
|
if err != nil {
|
|
log.Warn("AddFileResource fail", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
if err = merr.Error(status); err != nil {
|
|
log.Warn("AddFileResource fail", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info("AddFileResource success")
|
|
return status, nil
|
|
}
|
|
|
|
// RemoveFileResource remove file resource from rootcoord
|
|
func (node *Proxy) RemoveFileResource(ctx context.Context, req *milvuspb.RemoveFileResourceRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-RemoveFileResource")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(
|
|
zap.String("role", typeutil.ProxyRole),
|
|
zap.String("name", req.GetName()))
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info("receive RemoveFileResource request")
|
|
|
|
status, err := node.mixCoord.RemoveFileResource(ctx, req)
|
|
if err != nil {
|
|
log.Warn("RemoveFileResource fail", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
if err = merr.Error(status); err != nil {
|
|
log.Warn("RemoveFileResource fail", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
|
|
log.Info("RemoveFileResource success")
|
|
return status, nil
|
|
}
|
|
|
|
// ListFileResources list file resources from rootcoord
|
|
func (node *Proxy) ListFileResources(ctx context.Context, req *milvuspb.ListFileResourcesRequest) (*milvuspb.ListFileResourcesResponse, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-ListFileResources")
|
|
defer sp.End()
|
|
|
|
log := log.Ctx(ctx).With(zap.String("role", typeutil.ProxyRole))
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return &milvuspb.ListFileResourcesResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Info("receive ListFileResources request")
|
|
|
|
resp, err := node.mixCoord.ListFileResources(ctx, req)
|
|
if err != nil {
|
|
log.Warn("ListFileResources fail", zap.Error(err))
|
|
return &milvuspb.ListFileResourcesResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
if err = merr.Error(resp.GetStatus()); err != nil {
|
|
log.Warn("ListFileResources fail", zap.Error(err))
|
|
return &milvuspb.ListFileResourcesResponse{
|
|
Status: merr.Status(err),
|
|
}, nil
|
|
}
|
|
|
|
log.Info("ListFileResources success", zap.Int("count", len(resp.GetResources())))
|
|
return resp, nil
|
|
}
|
|
|
|
// UpdateReplicateConfiguration applies a full replacement of the current replication configuration across Milvus clusters.
|
|
func (node *Proxy) UpdateReplicateConfiguration(ctx context.Context, req *milvuspb.UpdateReplicateConfigurationRequest) (*commonpb.Status, error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-UpdateReplicateConfiguration")
|
|
defer sp.End()
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return merr.Status(err), nil
|
|
}
|
|
log.Ctx(ctx).Info("UpdateReplicateConfiguration received", replicateutil.ConfigLogFields(req.GetReplicateConfiguration())...)
|
|
err := streaming.WAL().Replicate().UpdateReplicateConfiguration(ctx, req.GetReplicateConfiguration())
|
|
if err != nil {
|
|
log.Ctx(ctx).Warn("UpdateReplicateConfiguration fail", zap.Error(err))
|
|
return merr.Status(err), nil
|
|
}
|
|
log.Ctx(ctx).Info("UpdateReplicateConfiguration success", replicateutil.ConfigLogFields(req.GetReplicateConfiguration())...)
|
|
return merr.Status(nil), nil
|
|
}
|
|
|
|
// GetReplicateInfo retrieves replication-related metadata from a target Milvus cluster.
|
|
// TODO: sheep, only get target checkpoint
|
|
func (node *Proxy) GetReplicateInfo(ctx context.Context, req *milvuspb.GetReplicateInfoRequest) (resp *milvuspb.GetReplicateInfoResponse, err error) {
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-GetReplicateInfo")
|
|
defer sp.End()
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
logger := log.Ctx(ctx).With(zap.String("sourceClusterID", req.GetSourceClusterId()))
|
|
logger.Info("GetReplicateInfo received")
|
|
defer func() {
|
|
if err != nil {
|
|
logger.Warn("GetReplicateInfo fail", zap.Error(err))
|
|
} else {
|
|
logger.Info("GetReplicateInfo success", zap.Any("checkpoints", resp.GetCheckpoints()))
|
|
}
|
|
}()
|
|
|
|
configHelper, err := streaming.WAL().Replicate().GetReplicateConfiguration(ctx)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
currentCluster := configHelper.GetCurrentCluster()
|
|
|
|
checkpoints := make([]*commonpb.ReplicateCheckpoint, 0, len(currentCluster.GetPchannels()))
|
|
for _, pchannel := range currentCluster.GetPchannels() {
|
|
checkpoint, err := streaming.WAL().Replicate().GetReplicateCheckpoint(ctx, pchannel)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
checkpoints = append(checkpoints, checkpoint.IntoProto())
|
|
}
|
|
return &milvuspb.GetReplicateInfoResponse{
|
|
Checkpoints: checkpoints,
|
|
}, nil
|
|
}
|
|
|
|
// CreateReplicateStream establishes a replication stream on the target Milvus cluster.
|
|
func (node *Proxy) CreateReplicateStream(stream milvuspb.MilvusService_CreateReplicateStreamServer) (err error) {
|
|
ctx := stream.Context()
|
|
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-CreateReplicateStream")
|
|
defer sp.End()
|
|
|
|
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
|
return err
|
|
}
|
|
|
|
log.Ctx(ctx).Info("replicate stream created")
|
|
defer func() {
|
|
if err != nil {
|
|
log.Ctx(ctx).Warn("replicate stream closed with error", zap.Error(err))
|
|
} else {
|
|
log.Ctx(ctx).Info("replicate stream closed")
|
|
}
|
|
}()
|
|
|
|
s, err := replicate.CreateReplicateServer(stream)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return s.Execute()
|
|
}
|